From eb7aad00160b1418ed96ecc83770b62ce3bfaaf0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 14 Nov 2023 11:35:54 +0100 Subject: [PATCH 001/325] Do not consider parts broken if only projections are broken --- src/Interpreters/MutationsInterpreter.cpp | 14 + src/Interpreters/MutationsInterpreter.h | 1 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 61 +++- src/Storages/MergeTree/IMergeTreeDataPart.h | 19 +- src/Storages/MergeTree/MergeTreeData.cpp | 26 +- src/Storages/MergeTree/MergeTreeData.h | 11 +- .../MergeTree/MergeTreeDataPartChecksum.h | 2 + src/Storages/MergeTree/MutateTask.cpp | 8 +- .../ReplicatedMergeTreePartCheckThread.cpp | 31 +- .../ReplicatedMergeTreePartCheckThread.h | 4 +- src/Storages/MergeTree/checkDataPart.cpp | 80 ++++- src/Storages/MergeTree/checkDataPart.h | 4 +- src/Storages/StorageMergeTree.cpp | 5 +- src/Storages/StorageReplicatedMergeTree.cpp | 3 +- src/Storages/System/StorageSystemDisks.cpp | 2 +- .../System/StorageSystemPartsBase.cpp | 8 +- src/Storages/System/StorageSystemPartsBase.h | 2 +- .../System/StorageSystemProjectionParts.cpp | 48 ++- .../StorageSystemProjectionPartsColumns.cpp | 21 +- .../02916_broken_projection.reference | 224 ++++++++++++++ .../0_stateless/02916_broken_projection.sh | 283 ++++++++++++++++++ 21 files changed, 795 insertions(+), 62 deletions(-) create mode 100644 tests/queries/0_stateless/02916_broken_projection.reference create mode 100755 tests/queries/0_stateless/02916_broken_projection.sh diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 0ace0a8b79c..a9a5d4f33d0 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -305,6 +305,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; @@ -922,6 +927,15 @@ void MutationsInterpreter::prepare(bool dry_run) materialized_indices.insert(index.name); } + /// Always rebuild broken projections. + for (const auto & projection : metadata_snapshot->getProjections()) + { + if (!source.hasBrokenProjection(projection.name)) + continue; + + materialized_projections.insert(projection.name); + } + for (const auto & projection : metadata_snapshot->getProjections()) { if (!source.hasProjection(projection.name)) diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index c53b86ddb5e..33b8021a653 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -122,6 +122,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/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 9bc72577b25..bc81758675e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -732,7 +732,23 @@ 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; + + LOG_ERROR(&Poco::Logger::get("IMergeTreeDataPart"), + "Cannot load projection {}, will consider it broken", projection.name); + + addBrokenProjectionPart(projection.name, std::move(part), getCurrentExceptionMessage(false), getCurrentExceptionCode()); + continue; + } + addProjectionPart(projection.name, std::move(part)); } } @@ -1129,7 +1145,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 */{}, /* throw_on_broken_projection */false); writeChecksums(checksums, {}); bytes_on_disk = checksums.getTotalSizeOnDisk(); @@ -2130,6 +2147,46 @@ std::optional IMergeTreeDataPart::getStreamNameForColumn( return getStreamNameOrHash(stream_name, extension, storage_); } +void IMergeTreeDataPart::addBrokenProjectionPart( + const String & projection_name, + std::shared_ptr projection_part, + const String & message, + int code) +{ + projection_part->setBrokenReason(message, code); + bool inserted = broken_projection_parts.emplace(projection_name, projection_part).second; + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Projection part {} in part {} is already added to a broken projection parts list", projection_name, name); +} + +void IMergeTreeDataPart::markProjectionPartAsBroken(const String & projection_name, const String & message, int code) const +{ + std::lock_guard lock(broken_projections_mutex); + + 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); + + broken_projection_parts.emplace(projection_name, it->second); + projection_parts.erase(it); +} + +void IMergeTreeDataPart::setBrokenReason(const String & message, int code) +{ + std::lock_guard lock(broken_projections_mutex); + is_broken = true; + exception = message; + exception_code = code; +} + +bool IMergeTreeDataPart::hasBrokenProjection(const String & projection_name) const +{ + std::lock_guard lock(broken_projections_mutex); + return broken_projection_parts.contains(projection_name); +} + 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 a9659d2f5f4..52a1541e15f 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -255,6 +255,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_projections_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; @@ -405,12 +411,20 @@ public: const std::map> & getProjectionParts() const { return projection_parts; } + const std::map> & getBrokenProjectionParts() const { return broken_projection_parts; } + MergeTreeDataPartBuilder getProjectionPartBuilder(const String & projection_name, bool is_temp_projection = false); void addProjectionPart(const String & projection_name, std::shared_ptr && projection_part); + void addBrokenProjectionPart(const String & projection_name, std::shared_ptr projection_part, const String & message, int code); + + 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); } + bool hasBrokenProjection(const String & projection_name) const; + void loadProjections(bool require_columns_checksums, bool check_consistency, bool if_not_loaded = false); /// Return set of metadata file names without checksums. For example, @@ -564,7 +578,8 @@ protected: const IMergeTreeDataPart * parent_part; String parent_part_name; - std::map> projection_parts; + mutable std::map> projection_parts; + mutable std::map> broken_projection_parts; mutable PartMetadataManagerPtr metadata_manager; @@ -678,6 +693,8 @@ private: void incrementStateMetric(MergeTreeDataPartState state) const; void decrementStateMetric(MergeTreeDataPartState state) const; + void setBrokenReason(const String & message, int code); + /// This ugly flag is needed for debug assertions only mutable bool part_is_probably_removed_from_disk = false; }; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1c0f9208fef..152c386e188 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5737,7 +5737,7 @@ MergeTreeData::getDataPartsVectorForInternalUsage(const DataPartStates & afforda } MergeTreeData::ProjectionPartsVector -MergeTreeData::getProjectionPartsVectorForInternalUsage(const DataPartStates & affordable_states, DataPartStateVector * out_states) const +MergeTreeData::getProjectionPartsVectorForInternalUsage(const DataPartStates & affordable_states, bool fill_states) const { auto lock = lockParts(); ProjectionPartsVector res; @@ -5749,14 +5749,20 @@ MergeTreeData::getProjectionPartsVectorForInternalUsage(const DataPartStates & a res.data_parts.push_back(part); for (const auto & [_, projection_part] : part->getProjectionParts()) res.projection_parts.push_back(projection_part); + for (const auto & [_, projection_part] : part->getBrokenProjectionParts()) + res.broken_projection_parts.push_back(projection_part); } } - if (out_states != nullptr) + if (fill_states) { - out_states->resize(res.projection_parts.size()); + res.projection_parts_states.resize(res.projection_parts.size()); for (size_t i = 0; i < res.projection_parts.size(); ++i) - (*out_states)[i] = res.projection_parts[i]->getParentPart()->getState(); + (res.projection_parts_states)[i] = res.projection_parts[i]->getParentPart()->getState(); + + res.broken_projection_parts_states.resize(res.broken_projection_parts.size()); + for (size_t i = 0; i < res.broken_projection_parts.size(); ++i) + (res.broken_projection_parts_states)[i] = res.broken_projection_parts[i]->getParentPart()->getState(); } return res; @@ -5809,7 +5815,7 @@ bool MergeTreeData::supportsLightweightDelete() const return true; } -MergeTreeData::ProjectionPartsVector MergeTreeData::getAllProjectionPartsVector(MergeTreeData::DataPartStateVector * out_states) const +MergeTreeData::ProjectionPartsVector MergeTreeData::getAllProjectionPartsVector(bool fill_states) const { ProjectionPartsVector res; auto lock = lockParts(); @@ -5820,11 +5826,15 @@ MergeTreeData::ProjectionPartsVector MergeTreeData::getAllProjectionPartsVector( res.projection_parts.push_back(projection_part); } - if (out_states != nullptr) + if (fill_states) { - out_states->resize(res.projection_parts.size()); + res.projection_parts_states.resize(res.projection_parts.size()); for (size_t i = 0; i < res.projection_parts.size(); ++i) - (*out_states)[i] = res.projection_parts[i]->getParentPart()->getState(); + (res.projection_parts_states)[i] = res.projection_parts[i]->getParentPart()->getState(); + + res.broken_projection_parts_states.resize(res.broken_projection_parts.size()); + for (size_t i = 0; i < res.broken_projection_parts.size(); ++i) + (res.broken_projection_parts_states)[i] = res.broken_projection_parts[i]->getParentPart()->getState(); } return res; } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 54104849fe4..4ef3b75988b 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, bool fill_states = false) const; /// Returns absolutely all parts (and snapshot of their states) @@ -496,7 +501,7 @@ public: size_t getTotalMarksCount() const; /// Same as above but only returns projection parts - ProjectionPartsVector getAllProjectionPartsVector(MergeTreeData::DataPartStateVector * out_states = nullptr) const; + ProjectionPartsVector getAllProjectionPartsVector(bool fill_states = false) const; /// Returns parts in Active state DataParts getDataPartsForInternalUsage() const; diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.h b/src/Storages/MergeTree/MergeTreeDataPartChecksum.h index 8e5e8c8c448..3595ce38db5 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 911b25de2ad..8ef1621b647 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -513,7 +513,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); @@ -1367,7 +1369,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) { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index b1875464725..4468cf8e3bf 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -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) { @@ -384,12 +387,22 @@ ReplicatedCheckResult ReplicatedMergeTreePartCheckThread::checkPartImpl(const St tryLogCurrentException(log, __PRETTY_FUNCTION__); - auto message = PreformattedMessage::create("Part {} looks broken. Removing it and will try to fetch.", part_name); - LOG_ERROR(log, message); + PreformattedMessage message; + if (is_broken_projection) + { + message = PreformattedMessage::create("Part {} has a broken projection. It will be ignored.", part_name); + LOG_DEBUG(log, message); + result.action = ReplicatedCheckResult::DoNothing; + } + else + { + 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 +432,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 +590,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 68dc6ca3d1d..26c4bfe9384 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 a75df00e8a7..74af7cbb77c 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -111,7 +111,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; @@ -120,6 +122,7 @@ static IMergeTreeDataPart::Checksums checkDataPart( */ CurrentMetrics::Increment metric_increment{CurrentMetrics::ReplicatedChecks}; + Poco::Logger * log = &Poco::Logger::get("checkDataPart"); NamesAndTypesList columns_txt; @@ -269,23 +272,68 @@ static IMergeTreeDataPart::Checksums checkDataPart( } } - for (const auto & [name, projection] : data_part->getProjectionParts()) + auto check_projection = [&](const String & name, std::shared_ptr projection) { - 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); + return; + } + + 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; + + LOG_TEST(log, "Marking projection {} as broken ({})", name, projection_file); + + if (!data_part->hasBrokenProjection(name)) + data_part->markProjectionPartAsBroken(name, getCurrentExceptionMessage(false), getCurrentExceptionCode()); + + is_broken_projection = true; + if (throw_on_broken_projection) + throw; + + projections_on_disk.erase(projection_file); + checksums_txt.remove(projection_file); + return; + } checksums_data.files[projection_file] = IMergeTreeDataPart::Checksums::Checksum( projection_checksums.getTotalSizeOnDisk(), projection_checksums.getTotalChecksumUInt128()); projections_on_disk.erase(projection_file); + }; + + auto broken_projection_parts = data_part->getBrokenProjectionParts(); /// Iterate over copy + for (const auto & [name, projection] : broken_projection_parts) + { + if (is_cancelled()) + return {}; + else + check_projection(name, projection); + } + + auto projection_parts = data_part->getProjectionParts(); /// Iterate over copy + for (const auto & [name, projection] : projection_parts) + { + if (is_cancelled()) + return {}; + else + check_projection(name, projection); } if (require_checksums && !projections_on_disk.empty()) @@ -315,7 +363,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); @@ -357,7 +407,9 @@ IMergeTreeDataPart::Checksums checkDataPart( data_part->getFileNamesWithoutChecksums(), read_settings, require_checksums, - is_cancelled); + is_cancelled, + is_broken_projection, + throw_on_broken_projection); }; try @@ -371,7 +423,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 e9a0dd5fbf3..74277616e95 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2242,11 +2242,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 */{}, /* throw_on_broken_projection */true); calculated_checksums.checkEqual(part->checksums, true); auto & part_mutable = const_cast(*part); @@ -2267,7 +2268,7 @@ std::optional StorageMergeTree::checkDataNext(DataValidationTasksPt { try { - checkDataPart(part, true); + checkDataPart(part, true, noop, /* is_cancelled */{}, /* throw_on_broken_projection */true); return CheckResult(part->name, true, ""); } catch (...) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 74821a9186c..1859fa03094 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8690,12 +8690,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/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index 23a00cc7ae5..250fcdba641 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -63,7 +63,7 @@ Pipe StorageSystemDisks::read( for (const auto & [disk_name, disk_ptr] : context->getDisksMap()) { col_name->insert(disk_name); - col_path->insert(disk_ptr->getPath()); + col_path->insert(fs::absolute(disk_ptr->getPath()).string()); col_free->insert(disk_ptr->getAvailableSpace().value_or(std::numeric_limits::max())); col_total->insert(disk_ptr->getTotalSpace().value_or(std::numeric_limits::max())); col_unreserved->insert(disk_ptr->getUnreservedSpace().value_or(std::numeric_limits::max())); diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 513af6cfc46..e97c13b1fed 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -64,7 +64,7 @@ StoragesInfo::getParts(MergeTreeData::DataPartStateVector & state, bool has_stat } MergeTreeData::ProjectionPartsVector -StoragesInfo::getProjectionParts(MergeTreeData::DataPartStateVector & state, bool has_state_column) const +StoragesInfo::getProjectionParts(bool fill_states, bool has_state_column) const { if (data->getInMemoryMetadataPtr()->projections.empty()) return {}; @@ -74,12 +74,12 @@ StoragesInfo::getProjectionParts(MergeTreeData::DataPartStateVector & state, boo { /// If has_state_column is requested, return all states. if (!has_state_column) - return data->getProjectionPartsVectorForInternalUsage({State::Active, State::Outdated}, &state); + return data->getProjectionPartsVectorForInternalUsage({State::Active, State::Outdated}, fill_states); - return data->getAllProjectionPartsVector(&state); + return data->getAllProjectionPartsVector(fill_states); } - return data->getProjectionPartsVectorForInternalUsage({State::Active}, &state); + return data->getProjectionPartsVectorForInternalUsage({State::Active}, fill_states); } StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context) diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index c3d2e64b303..e0e81f0d24d 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -25,7 +25,7 @@ struct StoragesInfo explicit operator bool() const { return storage != nullptr; } MergeTreeData::DataPartsVector getParts(MergeTreeData::DataPartStateVector & state, bool has_state_column) const; - MergeTreeData::ProjectionPartsVector getProjectionParts(MergeTreeData::DataPartStateVector & state, bool has_state_column) const; + MergeTreeData::ProjectionPartsVector getProjectionParts(bool fill_states, bool has_state_column) const; }; /** A helper class that enumerates the storages that match given query. */ diff --git a/src/Storages/System/StorageSystemProjectionParts.cpp b/src/Storages/System/StorageSystemProjectionParts.cpp index 213865a8d61..44bdb294a2d 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()}, } ) { @@ -93,15 +97,14 @@ void StorageSystemProjectionParts::processNextStorage( ContextPtr, MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) { using State = MergeTreeDataPartState; - MergeTreeData::DataPartStateVector all_parts_state; - MergeTreeData::ProjectionPartsVector all_parts = info.getProjectionParts(all_parts_state, has_state_column); - for (size_t part_number = 0; part_number < all_parts.projection_parts.size(); ++part_number) + MergeTreeData::ProjectionPartsVector all_parts = info.getProjectionParts(true, has_state_column); + auto fill_part_info = [&](size_t part_number, const MergeTreeData::DataPartsVector & parts, const MergeTreeData::DataPartStateVector & states) { - const auto & part = all_parts.projection_parts[part_number]; + const auto & part = parts[part_number]; const auto * parent_part = part->getParentPart(); chassert(parent_part); - auto part_state = all_parts_state[part_number]; + auto part_state = states[part_number]; ColumnSize columns_size = part->getTotalColumnsSize(); ColumnSize parent_columns_size = parent_part->getTotalColumnsSize(); @@ -278,10 +281,43 @@ void StorageSystemProjectionParts::processNextStorage( 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_projections_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) columns[res_index++]->insert(IMergeTreeDataPart::stateString(part_state)); + }; + + for (size_t part_number = 0; part_number < all_parts.projection_parts.size(); ++part_number) + { + auto part = all_parts.projection_parts[part_number]; + fill_part_info(part_number, all_parts.projection_parts, all_parts.projection_parts_states); + } + + for (size_t part_number = 0; part_number < all_parts.broken_projection_parts.size(); ++part_number) + { + auto part = all_parts.broken_projection_parts[part_number]; + fill_part_info(part_number, all_parts.broken_projection_parts, all_parts.broken_projection_parts_states); } } diff --git a/src/Storages/System/StorageSystemProjectionPartsColumns.cpp b/src/Storages/System/StorageSystemProjectionPartsColumns.cpp index 06becc6d91c..3f4224e46bb 100644 --- a/src/Storages/System/StorageSystemProjectionPartsColumns.cpp +++ b/src/Storages/System/StorageSystemProjectionPartsColumns.cpp @@ -103,15 +103,14 @@ void StorageSystemProjectionPartsColumns::processNextStorage( } /// Go through the list of projection parts. - MergeTreeData::DataPartStateVector all_parts_state; - MergeTreeData::ProjectionPartsVector all_parts = info.getProjectionParts(all_parts_state, has_state_column); - for (size_t part_number = 0; part_number < all_parts.projection_parts.size(); ++part_number) + MergeTreeData::ProjectionPartsVector all_parts = info.getProjectionParts(true, has_state_column); + auto fill_part_info = [&](size_t part_number, const MergeTreeData::DataPartsVector & parts, const MergeTreeData::DataPartStateVector & states) { - const auto & part = all_parts.projection_parts[part_number]; + const auto & part = parts[part_number]; const auto * parent_part = part->getParentPart(); chassert(parent_part); - auto part_state = all_parts_state[part_number]; + auto part_state = states[part_number]; auto columns_size = part->getTotalColumnsSize(); auto parent_columns_size = parent_part->getTotalColumnsSize(); @@ -260,6 +259,18 @@ void StorageSystemProjectionPartsColumns::processNextStorage( if (has_state_column) columns[res_index++]->insert(part->stateString()); } + }; + + for (size_t part_number = 0; part_number < all_parts.projection_parts.size(); ++part_number) + { + auto part = all_parts.projection_parts[part_number]; + fill_part_info(part_number, all_parts.projection_parts, all_parts.projection_parts_states); + } + + for (size_t part_number = 0; part_number < all_parts.broken_projection_parts.size(); ++part_number) + { + auto part = all_parts.broken_projection_parts[part_number]; + fill_part_info(part_number, all_parts.broken_projection_parts, all_parts.broken_projection_parts_states); } } diff --git a/tests/queries/0_stateless/02916_broken_projection.reference b/tests/queries/0_stateless/02916_broken_projection.reference new file mode 100644 index 00000000000..d0b07e081db --- /dev/null +++ b/tests/queries/0_stateless/02916_broken_projection.reference @@ -0,0 +1,224 @@ +insert new part +insert new part +insert new part +insert new part +system.parts +all_0_0_0 1 ['proj','proj_2'] +all_1_1_0 1 ['proj','proj_2'] +all_2_2_0 1 ['proj','proj_2'] +all_3_3_0 1 ['proj','proj_2'] +select from projection 'proj' +16 +12 +used projections +SELECT c FROM test WHERE d == 12 OR d == 16; ['default.test.proj'] +select from projection 'proj_2' +12 +16 +used projections +SELECT d FROM test WHERE c == 12 OR c == 16; ['default.test.proj_2'] +check table +1 +0 +broke metadata of part 'proj' (parent part: all_2_2_0) +system.parts +all_0_0_0 1 ['proj','proj_2'] +all_1_1_0 1 ['proj','proj_2'] +all_2_2_0 1 ['proj','proj_2'] +all_3_3_0 1 ['proj','proj_2'] +select from projection 'proj' +16 +12 +used projections +SELECT c FROM test WHERE d == 12 OR d == 16; ['default.test.proj'] +select from projection 'proj_2' +16 +12 +used projections +SELECT d FROM test WHERE c == 12 OR c == 16; ['default.test.proj_2'] +check table +0 +broken projections info +all_2_2_0 proj FILE_DOESNT_EXIST +check table full +all_0_0_0 1 +all_1_1_0 1 +all_3_3_0 1 +all_2_2_0 0 Part all_2_2_0 has a broken projection. It will be ignored. +0 +broke data of part 'proj_2' (parent part: all_2_2_0) +broken projections info +all_2_2_0 proj FILE_DOESNT_EXIST +system.parts +all_0_0_0 1 ['proj','proj_2'] +all_1_1_0 1 ['proj','proj_2'] +all_2_2_0 1 ['proj_2'] +all_3_3_0 1 ['proj','proj_2'] +select from projection 'proj' +16 +12 +used projections +SELECT c FROM test WHERE d == 12 OR d == 16; ['default.test.proj'] +select from projection 'proj_2' +FILE_DOESNT_EXIST +check table +0 +broken projections info +all_2_2_0 proj FILE_DOESNT_EXIST +all_2_2_0 proj_2 NO_FILE_IN_DATA_PART +system.parts +all_0_0_0 1 ['proj','proj_2'] +all_1_1_0 1 ['proj','proj_2'] +all_2_2_0 1 [] +all_3_3_0 1 ['proj','proj_2'] +select from projection 'proj' +16 +12 +used projections +SELECT c FROM test WHERE d == 12 OR d == 16; ['default.test.proj'] +select from projection 'proj_2' +16 +12 +used projections +SELECT d FROM test WHERE c == 12 OR c == 16; ['default.test.proj_2'] +check table +0 +0 +broke data of part 'proj_2' (parent part: all_3_3_0) +broken projections info +all_2_2_0 proj FILE_DOESNT_EXIST +all_2_2_0 proj_2 NO_FILE_IN_DATA_PART +insert new part +insert new part +optimize +0 +broken projections info +all_2_2_0 proj FILE_DOESNT_EXIST +all_2_2_0 proj_2 NO_FILE_IN_DATA_PART +all_3_3_0 proj_2 NO_FILE_IN_DATA_PART +system.parts +all_0_0_0 1 ['proj','proj_2'] +all_1_1_0 1 ['proj','proj_2'] +all_2_2_0 1 [] +all_3_3_0 0 ['proj'] +all_3_5_1 1 ['proj'] +all_4_4_0 0 ['proj','proj_2'] +all_5_5_0 0 ['proj','proj_2'] +select from projection 'proj' +16 +12 +used projections +SELECT c FROM test WHERE d == 12 OR d == 16; ['default.test.proj'] +select from projection 'proj_2' +16 +12 +used projections +SELECT d FROM test WHERE c == 12 OR c == 16; ['default.test.proj_2'] +check table +0 +0 +broke metadata of part 'proj' (parent part: all_1_1_0) +Detach - Attach +broken projections info +all_1_1_0 proj NO_FILE_IN_DATA_PART +all_2_2_0 proj NO_FILE_IN_DATA_PART +all_2_2_0 proj_2 FILE_DOESNT_EXIST +all_3_3_0 proj_2 FILE_DOESNT_EXIST +0 +broke data of part 'proj_2' (parent part: all_1_1_0) +Detach - Attach +broken projections info +all_1_1_0 proj NO_FILE_IN_DATA_PART +all_1_1_0 proj_2 FILE_DOESNT_EXIST +all_2_2_0 proj NO_FILE_IN_DATA_PART +all_2_2_0 proj_2 FILE_DOESNT_EXIST +all_3_3_0 proj_2 FILE_DOESNT_EXIST +system.parts +all_0_0_0 1 ['proj','proj_2'] +all_1_1_0 1 [] +all_2_2_0 1 [] +all_3_3_0 0 ['proj'] +all_3_5_1 1 ['proj'] +all_4_4_0 0 ['proj','proj_2'] +all_5_5_0 0 ['proj','proj_2'] +select from projection 'proj' +16 +12 +used projections +SELECT c FROM test WHERE d == 12 OR d == 16; ['default.test.proj'] +select from projection 'proj_2' +16 +12 +used projections +SELECT d FROM test WHERE c == 12 OR c == 16; ['default.test.proj_2'] +check table +0 +check table full +all_3_5_1 1 +all_0_0_0 1 +all_1_1_0 0 Part all_1_1_0 has a broken projection. It will be ignored. +all_2_2_0 0 Part all_2_2_0 has a broken projection. It will be ignored. +materialize projection proj +check table full +all_3_5_1_6 1 +all_0_0_0_6 1 +all_2_2_0_6 1 +all_1_1_0_6 1 +system.parts +all_0_0_0 0 ['proj','proj_2'] +all_0_0_0_6 1 ['proj','proj_2'] +all_1_1_0 0 [] +all_1_1_0_6 1 ['proj','proj_2'] +all_2_2_0 0 [] +all_2_2_0_6 1 ['proj','proj_2'] +all_3_3_0 0 ['proj'] +all_3_5_1 0 ['proj'] +all_3_5_1_6 1 ['proj'] +all_4_4_0 0 ['proj','proj_2'] +all_5_5_0 0 ['proj','proj_2'] +select from projection 'proj' +16 +12 +used projections +SELECT c FROM test WHERE d == 12 OR d == 16; ['default.test.proj'] +select from projection 'proj_2' +12 +16 +used projections +SELECT d FROM test WHERE c == 12 OR c == 16; ['default.test.proj_2'] +check table +1 +materialize projection proj_2 +check table full +all_3_5_1_7 1 +all_0_0_0_7 1 +all_2_2_0_7 1 +all_1_1_0_7 1 +system.parts +all_0_0_0 0 ['proj','proj_2'] +all_0_0_0_6 0 ['proj','proj_2'] +all_0_0_0_7 1 ['proj','proj_2'] +all_1_1_0 0 [] +all_1_1_0_6 0 ['proj','proj_2'] +all_1_1_0_7 1 ['proj','proj_2'] +all_2_2_0 0 [] +all_2_2_0_6 0 ['proj','proj_2'] +all_2_2_0_7 1 ['proj','proj_2'] +all_3_3_0 0 ['proj'] +all_3_5_1 0 ['proj'] +all_3_5_1_6 0 ['proj'] +all_3_5_1_7 1 ['proj','proj_2'] +all_4_4_0 0 ['proj','proj_2'] +all_5_5_0 0 ['proj','proj_2'] +select from projection 'proj' +12 +16 +used projections +SELECT c FROM test WHERE d == 12 OR d == 16; ['default.test.proj'] +select from projection 'proj_2' +16 +12 +used projections +SELECT d FROM test WHERE c == 12 OR c == 16; ['default.test.proj_2'] +check table +1 diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh new file mode 100755 index 00000000000..81adfe6e49d --- /dev/null +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -0,0 +1,283 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -nm -q " +DROP TABLE IF EXISTS test SYNC; +CREATE TABLE test +( + a String, + b String, + c Int32, + d Int32, + e Int32, + + PROJECTION proj + ( + SELECT c ORDER BY d + ), + PROJECTION proj_2 + ( + SELECT d ORDER BY c + ) +) +ENGINE = ReplicatedMergeTree('/test2/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/', '1') PRIMARY KEY (a) +SETTINGS min_bytes_for_wide_part = 0, + max_parts_to_merge_at_once=3, + enable_vertical_merge_algorithm=1, + vertical_merge_algorithm_min_rows_to_activate = 1, + vertical_merge_algorithm_min_columns_to_activate = 1, + vertical_merge_algorithm_min_columns_to_activate = 1; +" + +table_uuid=$($CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.tables WHERE table='test' and database=currentDatabase()") + +function random() +{ + cat /dev/urandom | LC_ALL=C tr -dc 'a-zA-Z' | fold -w ${1:-8} | head -n 1 +} + +function insert() +{ + offset=$1 + size=$2 + echo 'insert new part' + $CLICKHOUSE_CLIENT -q "INSERT INTO test SELECT number, number, number, number, number%2 FROM numbers($offset, $size);" +} + +function break_projection() +{ + part_name=$1 + parent_name=$2 + break_type=$3 + + read -r disk_name part_path <<< $($CLICKHOUSE_CLIENT -nm -q " + SELECT disk_name, path + FROM system.projection_parts + WHERE table='test' + AND database=currentDatabase() + AND active=1 + AND part_name='$part_name' + AND parent_name='$parent_name' + LIMIT 1; + ") + + path=$($CLICKHOUSE_CLIENT -q "SELECT path FROM system.disks WHERE name='$disk_name'") + + # make sure path is absolute + $CLICKHOUSE_CLIENT -q "select throwIf(substring('$path', 1, 1) != '/', 'Path is relative: $path')" || exit + + if [ "$break_type" = "data" ] + then + rm "$path/$part_path/d.bin" + rm "$path/$part_path/c.bin" + echo "broke data of part '$part_name' (parent part: $parent_name)" + else + rm "$path/$part_path/columns.txt" + echo "broke metadata of part '$part_name' (parent part: $parent_name)" + fi +} + +function broken_projections_info() +{ + echo 'broken projections info' + $CLICKHOUSE_CLIENT -q " + SELECT parent_name, name, errors.name FROM + ( + SELECT parent_name, name, exception_code + FROM system.projection_parts + WHERE table='test' + 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 +" +} + +function check() +{ + expect_broken_part="" + expected_error="" + if [ $# -ne 0 ]; then + expect_broken_part=$1 + expected_error=$2 + fi + + echo 'system.parts' + $CLICKHOUSE_CLIENT -q " + SELECT name, active, projections + FROM system.parts + WHERE table='test' AND database=currentDatabase() + ORDER BY name;" + + echo "select from projection 'proj'" + query_id=$(random 8) + + if [ "$expect_broken_part" = "proj" ] + then + $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT c FROM test WHERE d == 12;" 2>&1 | grep -o $expected_error + else + $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT c FROM test WHERE d == 12 OR d == 16;" + echo 'used projections' + $CLICKHOUSE_CLIENT -nm -q " + SYSTEM FLUSH LOGS; + SELECT query, projections FROM system.query_log WHERE query_id='$query_id' and type='QueryFinish' + " + fi + + echo "select from projection 'proj_2'" + query_id=$(random 8) + + if [ "$expect_broken_part" = "proj_2" ] + then + $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT d FROM test WHERE c == 12;" 2>&1 | grep -o $expected_error + else + $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT d FROM test WHERE c == 12 OR c == 16;" + echo 'used projections' + $CLICKHOUSE_CLIENT -nm -q " + SYSTEM FLUSH LOGS; + SELECT query, projections FROM system.query_log WHERE query_id='$query_id' and type='QueryFinish' + " + fi + + echo 'check table' + $CLICKHOUSE_CLIENT -q "CHECK TABLE test" +} + +function optimize_no_wait() +{ + echo 'optimize' + $CLICKHOUSE_CLIENT -nm -q "OPTIMIZE TABLE test SETTINGS alter_sync=0;" +} + +function reattach() +{ + echo 'Detach - Attach' + $CLICKHOUSE_CLIENT -nm -q " + DETACH TABLE test; + ATTACH TABLE test; + " +} + +function materialize_projection +{ + projection=$1 + echo "materialize projection $projection" + $CLICKHOUSE_CLIENT -q "ALTER TABLE test MATERIALIZE PROJECTION $projection SETTINGS mutations_sync=2" +} + +function check_table_full() +{ + echo 'check table full' + $CLICKHOUSE_CLIENT -q "CHECK TABLE test SETTINGS check_query_single_value_result = 0" +} + + +insert 0 5 + +insert 5 5 + +insert 10 5 + +insert 15 5 + +check + +# Break metadata file of projection 'proj' +break_projection proj all_2_2_0 metadata + +# Do select and after "check table" query. +# Select works because it does not read columns.txt. +check + +# Projection 'proj' from part all_2_2_0 will now appear in broken parts info +# because it was marked broken during "check table" query. +# TODO: try to mark it during select as well +broken_projections_info + +# Check table query will also show a list of parts which have broken projections. +check_table_full + +# Break data file of projection 'proj_2' for part all_2_2_0 +break_projection proj_2 all_2_2_0 data + +# It will not yet appear in broken projections info. +broken_projections_info + +# Select now fails with error "File doesn't exist" +check "proj_2" "FILE_DOESNT_EXIST" + +# Projection 'proj_2' from part all_2_2_0 will now appear in broken parts info. +broken_projections_info + +# Second select works, because projection is now marked as broken. +check + +# Break data file of projection 'proj_2' for part all_3_3_0 +break_projection proj_2 all_3_3_0 data + +# It will not yet appear in broken projections info. +broken_projections_info + +insert 20 5 + +insert 25 5 + +# Part all_3_3_0 has 'proj' and 'proj_2' projections, but 'proj_2' 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 'proj_2' of part all_3_3_0 and proj_2 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 'proj_2' as it will see that one part does not have it anymore in the set of valid projections. +optimize_no_wait +sleep 2 + +$CLICKHOUSE_CLIENT -nm -q " +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%proj_2.proj/c.bin%' +" + +# Projection 'proj_2' from part all_2_2_0 will now appear in broken parts info. +broken_projections_info + +check + +break_projection proj all_1_1_0 metadata + +reattach + +broken_projections_info + +break_projection proj_2 all_1_1_0 data + +reattach + +broken_projections_info + +check + +check_table_full + +materialize_projection proj + +check_table_full + +check + +materialize_projection proj_2 + +check_table_full + +check + +$CLICKHOUSE_CLIENT -nm -q " +DROP TABLE test; +" From 6c42a3fad6b58efdf91115c3b80f267f1f604c62 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 16 Nov 2023 16:43:34 +0100 Subject: [PATCH 002/325] Better --- src/Interpreters/MutationsInterpreter.cpp | 18 ++-- .../Optimizations/projectionsCommon.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 45 +++------ src/Storages/MergeTree/IMergeTreeDataPart.h | 11 +-- src/Storages/MergeTree/MergeTask.cpp | 5 +- src/Storages/MergeTree/MergeTreeData.cpp | 57 ++++++----- src/Storages/MergeTree/MergeTreeData.h | 4 +- src/Storages/MergeTree/MutateTask.cpp | 8 +- src/Storages/MergeTree/checkDataPart.cpp | 34 ++----- .../System/StorageSystemPartsBase.cpp | 8 +- src/Storages/System/StorageSystemPartsBase.h | 2 +- .../System/StorageSystemProjectionParts.cpp | 30 +++--- .../StorageSystemProjectionPartsColumns.cpp | 21 +--- .../02916_broken_projection.reference | 95 ++++++++----------- .../0_stateless/02916_broken_projection.sh | 12 +-- 15 files changed, 146 insertions(+), 206 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index a9a5d4f33d0..237bffe4a67 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -753,7 +753,7 @@ void MutationsInterpreter::prepare(bool dry_run) { mutation_kind.set(MutationKind::MUTATE_INDEX_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); @@ -927,20 +927,18 @@ void MutationsInterpreter::prepare(bool dry_run) materialized_indices.insert(index.name); } - /// Always rebuild broken projections. - for (const auto & projection : metadata_snapshot->getProjections()) - { - if (!source.hasBrokenProjection(projection.name)) - continue; - - materialized_projections.insert(projection.name); - } - for (const auto & projection : metadata_snapshot->getProjections()) { 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/Processors/QueryPlan/Optimizations/projectionsCommon.cpp b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp index c3b3449857b..9ebd5aaa32f 100644 --- a/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp +++ b/src/Processors/QueryPlan/Optimizations/projectionsCommon.cpp @@ -224,7 +224,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/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index bc81758675e..85ce112d9a1 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -745,8 +745,7 @@ void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool ch LOG_ERROR(&Poco::Logger::get("IMergeTreeDataPart"), "Cannot load projection {}, will consider it broken", projection.name); - addBrokenProjectionPart(projection.name, std::move(part), getCurrentExceptionMessage(false), getCurrentExceptionCode()); - continue; + part->setBrokenReason(getCurrentExceptionMessage(false), getCurrentExceptionCode()); } addProjectionPart(projection.name, std::move(part)); @@ -2147,44 +2146,30 @@ std::optional IMergeTreeDataPart::getStreamNameForColumn( return getStreamNameOrHash(stream_name, extension, storage_); } -void IMergeTreeDataPart::addBrokenProjectionPart( - const String & projection_name, - std::shared_ptr projection_part, - const String & message, - int code) -{ - projection_part->setBrokenReason(message, code); - bool inserted = broken_projection_parts.emplace(projection_name, projection_part).second; - if (!inserted) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Projection part {} in part {} is already added to a broken projection parts list", projection_name, name); -} - void IMergeTreeDataPart::markProjectionPartAsBroken(const String & projection_name, const String & message, int code) const { - std::lock_guard lock(broken_projections_mutex); - 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); - - broken_projection_parts.emplace(projection_name, it->second); - projection_parts.erase(it); -} - -void IMergeTreeDataPart::setBrokenReason(const String & message, int code) -{ - std::lock_guard lock(broken_projections_mutex); - is_broken = true; - exception = message; - exception_code = code; } bool IMergeTreeDataPart::hasBrokenProjection(const String & projection_name) const { - std::lock_guard lock(broken_projections_mutex); - return broken_projection_parts.contains(projection_name); + 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) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 52a1541e15f..9af2c16f1e8 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -259,7 +259,7 @@ public: mutable std::atomic is_broken {false}; mutable std::string exception; mutable int exception_code = 0; - mutable std::mutex broken_projections_mutex; + 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; @@ -411,14 +411,10 @@ public: const std::map> & getProjectionParts() const { return projection_parts; } - const std::map> & getBrokenProjectionParts() const { return broken_projection_parts; } - MergeTreeDataPartBuilder getProjectionPartBuilder(const String & projection_name, bool is_temp_projection = false); void addProjectionPart(const String & projection_name, std::shared_ptr && projection_part); - void addBrokenProjectionPart(const String & projection_name, std::shared_ptr projection_part, const String & message, int code); - 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); } @@ -427,6 +423,8 @@ public: void loadProjections(bool require_columns_checksums, bool check_consistency, 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. NameSet getFileNamesWithoutChecksums() const; @@ -579,7 +577,6 @@ protected: String parent_part_name; mutable std::map> projection_parts; - mutable std::map> broken_projection_parts; mutable PartMetadataManagerPtr metadata_manager; @@ -693,8 +690,6 @@ private: void incrementStateMetric(MergeTreeDataPartState state) const; void decrementStateMetric(MergeTreeDataPartState state) const; - void setBrokenReason(const String & message, int code); - /// This ugly flag is needed for debug assertions only mutable bool part_is_probably_removed_from_disk = false; }; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index adb1ca72e46..53ba1a57b27 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -696,8 +696,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 152c386e188..0725c3cbf32 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5737,7 +5737,7 @@ MergeTreeData::getDataPartsVectorForInternalUsage(const DataPartStates & afforda } MergeTreeData::ProjectionPartsVector -MergeTreeData::getProjectionPartsVectorForInternalUsage(const DataPartStates & affordable_states, bool fill_states) const +MergeTreeData::getProjectionPartsVectorForInternalUsage(const DataPartStates & affordable_states, DataPartStateVector * out_states) const { auto lock = lockParts(); ProjectionPartsVector res; @@ -5749,20 +5749,14 @@ MergeTreeData::getProjectionPartsVectorForInternalUsage(const DataPartStates & a res.data_parts.push_back(part); for (const auto & [_, projection_part] : part->getProjectionParts()) res.projection_parts.push_back(projection_part); - for (const auto & [_, projection_part] : part->getBrokenProjectionParts()) - res.broken_projection_parts.push_back(projection_part); } } - if (fill_states) + if (out_states != nullptr) { - res.projection_parts_states.resize(res.projection_parts.size()); + out_states->resize(res.projection_parts.size()); for (size_t i = 0; i < res.projection_parts.size(); ++i) - (res.projection_parts_states)[i] = res.projection_parts[i]->getParentPart()->getState(); - - res.broken_projection_parts_states.resize(res.broken_projection_parts.size()); - for (size_t i = 0; i < res.broken_projection_parts.size(); ++i) - (res.broken_projection_parts_states)[i] = res.broken_projection_parts[i]->getParentPart()->getState(); + (*out_states)[i] = res.projection_parts[i]->getParentPart()->getState(); } return res; @@ -5815,7 +5809,7 @@ bool MergeTreeData::supportsLightweightDelete() const return true; } -MergeTreeData::ProjectionPartsVector MergeTreeData::getAllProjectionPartsVector(bool fill_states) const +MergeTreeData::ProjectionPartsVector MergeTreeData::getAllProjectionPartsVector(MergeTreeData::DataPartStateVector * out_states) const { ProjectionPartsVector res; auto lock = lockParts(); @@ -5826,15 +5820,11 @@ MergeTreeData::ProjectionPartsVector MergeTreeData::getAllProjectionPartsVector( res.projection_parts.push_back(projection_part); } - if (fill_states) + if (out_states != nullptr) { - res.projection_parts_states.resize(res.projection_parts.size()); + out_states->resize(res.projection_parts.size()); for (size_t i = 0; i < res.projection_parts.size(); ++i) - (res.projection_parts_states)[i] = res.projection_parts[i]->getParentPart()->getState(); - - res.broken_projection_parts_states.resize(res.broken_projection_parts.size()); - for (size_t i = 0; i < res.broken_projection_parts.size(); ++i) - (res.broken_projection_parts_states)[i] = res.broken_projection_parts[i]->getParentPart()->getState(); + (*out_states)[i] = res.projection_parts[i]->getParentPart()->getState(); } return res; } @@ -7050,8 +7040,7 @@ std::pair MergeTreeData::cloneAn } } - auto projections = src_part->getProjectionParts(); - for (const auto & [name, projection_part] : projections) + for (const auto & [name, projection_part] : src_part->getProjectionParts()) { const auto & projection_storage = projection_part->getDataPartStorage(); for (auto it = projection_storage.iterate(); it->isValid(); it->next()) @@ -7654,21 +7643,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 = [](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(left_projection_parts); + remove_broken_parts(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 4ef3b75988b..18087c6b059 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -489,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, bool fill_states = false) const; + const DataPartStates & affordable_states, MergeTreeData::DataPartStateVector * out_states) const; /// Returns absolutely all parts (and snapshot of their states) @@ -501,7 +501,7 @@ public: size_t getTotalMarksCount() const; /// Same as above but only returns projection parts - ProjectionPartsVector getAllProjectionPartsVector(bool fill_states = false) const; + ProjectionPartsVector getAllProjectionPartsVector(MergeTreeData::DataPartStateVector * out_states = nullptr) const; /// Returns parts in Active state DataParts getDataPartsForInternalUsage() const; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 8ef1621b647..6a1ceec1cd3 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -514,8 +514,8 @@ static std::set getProjectionsToRecalculate( bool need_recalculate = materialized_projections.contains(projection.name) || (!is_full_part_storage - && (source_part->hasProjection(projection.name) - || source_part->hasBrokenProjection(projection.name))); + && source_part->hasProjection(projection.name) + && !source_part->hasBrokenProjection(projection.name)); if (need_recalculate) projections_to_recalc.insert(&projection); @@ -1370,8 +1370,8 @@ private: bool need_recalculate = ctx->materialized_projections.contains(projection.name) || (!is_full_part_storage - && (ctx->source_part->hasProjection(projection.name) - || ctx->source_part->hasBrokenProjection(projection.name))); + && ctx->source_part->hasProjection(projection.name) + && !ctx->source_part->hasBrokenProjection(projection.name)); if (need_recalculate) { diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 74af7cbb77c..8feabf344b5 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -272,14 +272,16 @@ static IMergeTreeDataPart::Checksums checkDataPart( } } - auto check_projection = [&](const String & name, std::shared_ptr projection) + for (const auto & [name, projection] : data_part->getProjectionParts()) { + if (is_cancelled()) + return {}; + auto projection_file = name + ".proj"; if (!throw_on_broken_projection && projection->is_broken) { projections_on_disk.erase(projection_file); checksums_txt.remove(projection_file); - return; } IMergeTreeDataPart::Checksums projection_checksums; @@ -297,10 +299,11 @@ static IMergeTreeDataPart::Checksums checkDataPart( if (isRetryableException(std::current_exception())) throw; - LOG_TEST(log, "Marking projection {} as broken ({})", name, projection_file); - - if (!data_part->hasBrokenProjection(name)) - data_part->markProjectionPartAsBroken(name, getCurrentExceptionMessage(false), getCurrentExceptionCode()); + 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) @@ -308,7 +311,6 @@ static IMergeTreeDataPart::Checksums checkDataPart( projections_on_disk.erase(projection_file); checksums_txt.remove(projection_file); - return; } checksums_data.files[projection_file] = IMergeTreeDataPart::Checksums::Checksum( @@ -316,24 +318,6 @@ static IMergeTreeDataPart::Checksums checkDataPart( projection_checksums.getTotalChecksumUInt128()); projections_on_disk.erase(projection_file); - }; - - auto broken_projection_parts = data_part->getBrokenProjectionParts(); /// Iterate over copy - for (const auto & [name, projection] : broken_projection_parts) - { - if (is_cancelled()) - return {}; - else - check_projection(name, projection); - } - - auto projection_parts = data_part->getProjectionParts(); /// Iterate over copy - for (const auto & [name, projection] : projection_parts) - { - if (is_cancelled()) - return {}; - else - check_projection(name, projection); } if (require_checksums && !projections_on_disk.empty()) diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index e97c13b1fed..513af6cfc46 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -64,7 +64,7 @@ StoragesInfo::getParts(MergeTreeData::DataPartStateVector & state, bool has_stat } MergeTreeData::ProjectionPartsVector -StoragesInfo::getProjectionParts(bool fill_states, bool has_state_column) const +StoragesInfo::getProjectionParts(MergeTreeData::DataPartStateVector & state, bool has_state_column) const { if (data->getInMemoryMetadataPtr()->projections.empty()) return {}; @@ -74,12 +74,12 @@ StoragesInfo::getProjectionParts(bool fill_states, bool has_state_column) const { /// If has_state_column is requested, return all states. if (!has_state_column) - return data->getProjectionPartsVectorForInternalUsage({State::Active, State::Outdated}, fill_states); + return data->getProjectionPartsVectorForInternalUsage({State::Active, State::Outdated}, &state); - return data->getAllProjectionPartsVector(fill_states); + return data->getAllProjectionPartsVector(&state); } - return data->getProjectionPartsVectorForInternalUsage({State::Active}, fill_states); + return data->getProjectionPartsVectorForInternalUsage({State::Active}, &state); } StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context) diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index e0e81f0d24d..c3d2e64b303 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -25,7 +25,7 @@ struct StoragesInfo explicit operator bool() const { return storage != nullptr; } MergeTreeData::DataPartsVector getParts(MergeTreeData::DataPartStateVector & state, bool has_state_column) const; - MergeTreeData::ProjectionPartsVector getProjectionParts(bool fill_states, bool has_state_column) const; + MergeTreeData::ProjectionPartsVector getProjectionParts(MergeTreeData::DataPartStateVector & state, bool has_state_column) const; }; /** A helper class that enumerates the storages that match given query. */ diff --git a/src/Storages/System/StorageSystemProjectionParts.cpp b/src/Storages/System/StorageSystemProjectionParts.cpp index 44bdb294a2d..3dbe6823dac 100644 --- a/src/Storages/System/StorageSystemProjectionParts.cpp +++ b/src/Storages/System/StorageSystemProjectionParts.cpp @@ -97,14 +97,15 @@ void StorageSystemProjectionParts::processNextStorage( ContextPtr, MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) { using State = MergeTreeDataPartState; - MergeTreeData::ProjectionPartsVector all_parts = info.getProjectionParts(true, has_state_column); - auto fill_part_info = [&](size_t part_number, const MergeTreeData::DataPartsVector & parts, const MergeTreeData::DataPartStateVector & states) + MergeTreeData::DataPartStateVector all_parts_state; + MergeTreeData::ProjectionPartsVector all_parts = info.getProjectionParts(all_parts_state, has_state_column); + for (size_t part_number = 0; part_number < all_parts.projection_parts.size(); ++part_number) { - const auto & part = parts[part_number]; + const auto & part = all_parts.projection_parts[part_number]; const auto * parent_part = part->getParentPart(); chassert(parent_part); - auto part_state = states[part_number]; + auto part_state = all_parts_state[part_number]; ColumnSize columns_size = part->getTotalColumnsSize(); ColumnSize parent_columns_size = parent_part->getTotalColumnsSize(); @@ -275,7 +276,12 @@ 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); @@ -287,7 +293,7 @@ void StorageSystemProjectionParts::processNextStorage( if (part->is_broken) { - std::lock_guard lock(part->broken_projections_mutex); + 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++]) @@ -306,18 +312,6 @@ void StorageSystemProjectionParts::processNextStorage( /// Do not use part->getState*, it can be changed from different thread if (has_state_column) columns[res_index++]->insert(IMergeTreeDataPart::stateString(part_state)); - }; - - for (size_t part_number = 0; part_number < all_parts.projection_parts.size(); ++part_number) - { - auto part = all_parts.projection_parts[part_number]; - fill_part_info(part_number, all_parts.projection_parts, all_parts.projection_parts_states); - } - - for (size_t part_number = 0; part_number < all_parts.broken_projection_parts.size(); ++part_number) - { - auto part = all_parts.broken_projection_parts[part_number]; - fill_part_info(part_number, all_parts.broken_projection_parts, all_parts.broken_projection_parts_states); } } diff --git a/src/Storages/System/StorageSystemProjectionPartsColumns.cpp b/src/Storages/System/StorageSystemProjectionPartsColumns.cpp index 3f4224e46bb..06becc6d91c 100644 --- a/src/Storages/System/StorageSystemProjectionPartsColumns.cpp +++ b/src/Storages/System/StorageSystemProjectionPartsColumns.cpp @@ -103,14 +103,15 @@ void StorageSystemProjectionPartsColumns::processNextStorage( } /// Go through the list of projection parts. - MergeTreeData::ProjectionPartsVector all_parts = info.getProjectionParts(true, has_state_column); - auto fill_part_info = [&](size_t part_number, const MergeTreeData::DataPartsVector & parts, const MergeTreeData::DataPartStateVector & states) + MergeTreeData::DataPartStateVector all_parts_state; + MergeTreeData::ProjectionPartsVector all_parts = info.getProjectionParts(all_parts_state, has_state_column); + for (size_t part_number = 0; part_number < all_parts.projection_parts.size(); ++part_number) { - const auto & part = parts[part_number]; + const auto & part = all_parts.projection_parts[part_number]; const auto * parent_part = part->getParentPart(); chassert(parent_part); - auto part_state = states[part_number]; + auto part_state = all_parts_state[part_number]; auto columns_size = part->getTotalColumnsSize(); auto parent_columns_size = parent_part->getTotalColumnsSize(); @@ -259,18 +260,6 @@ void StorageSystemProjectionPartsColumns::processNextStorage( if (has_state_column) columns[res_index++]->insert(part->stateString()); } - }; - - for (size_t part_number = 0; part_number < all_parts.projection_parts.size(); ++part_number) - { - auto part = all_parts.projection_parts[part_number]; - fill_part_info(part_number, all_parts.projection_parts, all_parts.projection_parts_states); - } - - for (size_t part_number = 0; part_number < all_parts.broken_projection_parts.size(); ++part_number) - { - auto part = all_parts.broken_projection_parts[part_number]; - fill_part_info(part_number, all_parts.broken_projection_parts, all_parts.broken_projection_parts_states); } } diff --git a/tests/queries/0_stateless/02916_broken_projection.reference b/tests/queries/0_stateless/02916_broken_projection.reference index d0b07e081db..62966036eed 100644 --- a/tests/queries/0_stateless/02916_broken_projection.reference +++ b/tests/queries/0_stateless/02916_broken_projection.reference @@ -8,15 +8,15 @@ all_1_1_0 1 ['proj','proj_2'] all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] select from projection 'proj' -16 12 +16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16; ['default.test.proj'] +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['default.test.proj'] select from projection 'proj_2' 12 16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16; ['default.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['default.test.proj_2'] check table 1 0 @@ -27,23 +27,20 @@ all_1_1_0 1 ['proj','proj_2'] all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] select from projection 'proj' -16 12 +16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16; ['default.test.proj'] +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['default.test.proj'] select from projection 'proj_2' -16 12 +16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16; ['default.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['default.test.proj_2'] check table 0 broken projections info all_2_2_0 proj FILE_DOESNT_EXIST check table full -all_0_0_0 1 -all_1_1_0 1 -all_3_3_0 1 all_2_2_0 0 Part all_2_2_0 has a broken projection. It will be ignored. 0 broke data of part 'proj_2' (parent part: all_2_2_0) @@ -52,13 +49,13 @@ all_2_2_0 proj FILE_DOESNT_EXIST system.parts all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] -all_2_2_0 1 ['proj_2'] +all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] select from projection 'proj' -16 12 +16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16; ['default.test.proj'] +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['default.test.proj'] select from projection 'proj_2' FILE_DOESNT_EXIST check table @@ -69,18 +66,18 @@ all_2_2_0 proj_2 NO_FILE_IN_DATA_PART system.parts all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] -all_2_2_0 1 [] +all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] select from projection 'proj' -16 12 +16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16; ['default.test.proj'] +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['default.test.proj'] select from projection 'proj_2' -16 12 +16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16; ['default.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['default.test.proj_2'] check table 0 0 @@ -99,21 +96,21 @@ all_3_3_0 proj_2 NO_FILE_IN_DATA_PART system.parts all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] -all_2_2_0 1 [] -all_3_3_0 0 ['proj'] +all_2_2_0 1 ['proj','proj_2'] +all_3_3_0 0 ['proj','proj_2'] all_3_5_1 1 ['proj'] all_4_4_0 0 ['proj','proj_2'] all_5_5_0 0 ['proj','proj_2'] select from projection 'proj' -16 12 +16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16; ['default.test.proj'] +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['default.test.proj'] select from projection 'proj_2' -16 12 +16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16; ['default.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['default.test.proj_2'] check table 0 0 @@ -135,76 +132,66 @@ all_2_2_0 proj_2 FILE_DOESNT_EXIST all_3_3_0 proj_2 FILE_DOESNT_EXIST system.parts all_0_0_0 1 ['proj','proj_2'] -all_1_1_0 1 [] -all_2_2_0 1 [] -all_3_3_0 0 ['proj'] +all_1_1_0 1 ['proj','proj_2'] +all_2_2_0 1 ['proj','proj_2'] +all_3_3_0 0 ['proj','proj_2'] all_3_5_1 1 ['proj'] all_4_4_0 0 ['proj','proj_2'] all_5_5_0 0 ['proj','proj_2'] select from projection 'proj' -16 12 +16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16; ['default.test.proj'] +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['default.test.proj'] select from projection 'proj_2' -16 12 +16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16; ['default.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['default.test.proj_2'] check table 0 check table full -all_3_5_1 1 -all_0_0_0 1 -all_1_1_0 0 Part all_1_1_0 has a broken projection. It will be ignored. all_2_2_0 0 Part all_2_2_0 has a broken projection. It will be ignored. +all_1_1_0 0 Part all_1_1_0 has a broken projection. It will be ignored. materialize projection proj check table full -all_3_5_1_6 1 -all_0_0_0_6 1 -all_2_2_0_6 1 -all_1_1_0_6 1 system.parts all_0_0_0 0 ['proj','proj_2'] all_0_0_0_6 1 ['proj','proj_2'] -all_1_1_0 0 [] +all_1_1_0 0 ['proj','proj_2'] all_1_1_0_6 1 ['proj','proj_2'] -all_2_2_0 0 [] +all_2_2_0 0 ['proj','proj_2'] all_2_2_0_6 1 ['proj','proj_2'] -all_3_3_0 0 ['proj'] +all_3_3_0 0 ['proj','proj_2'] all_3_5_1 0 ['proj'] all_3_5_1_6 1 ['proj'] all_4_4_0 0 ['proj','proj_2'] all_5_5_0 0 ['proj','proj_2'] select from projection 'proj' -16 12 +16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16; ['default.test.proj'] +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['default.test.proj'] select from projection 'proj_2' 12 16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16; ['default.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['default.test.proj_2'] check table 1 materialize projection proj_2 check table full -all_3_5_1_7 1 -all_0_0_0_7 1 -all_2_2_0_7 1 -all_1_1_0_7 1 system.parts all_0_0_0 0 ['proj','proj_2'] all_0_0_0_6 0 ['proj','proj_2'] all_0_0_0_7 1 ['proj','proj_2'] -all_1_1_0 0 [] +all_1_1_0 0 ['proj','proj_2'] all_1_1_0_6 0 ['proj','proj_2'] all_1_1_0_7 1 ['proj','proj_2'] -all_2_2_0 0 [] +all_2_2_0 0 ['proj','proj_2'] all_2_2_0_6 0 ['proj','proj_2'] all_2_2_0_7 1 ['proj','proj_2'] -all_3_3_0 0 ['proj'] +all_3_3_0 0 ['proj','proj_2'] all_3_5_1 0 ['proj'] all_3_5_1_6 0 ['proj'] all_3_5_1_7 1 ['proj','proj_2'] @@ -214,11 +201,11 @@ select from projection 'proj' 12 16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16; ['default.test.proj'] +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['default.test.proj'] select from projection 'proj_2' -16 12 +16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16; ['default.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['default.test.proj_2'] check table 1 diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index 81adfe6e49d..4748506d9cf 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -23,7 +23,7 @@ CREATE TABLE test SELECT d ORDER BY c ) ) -ENGINE = ReplicatedMergeTree('/test2/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/', '1') PRIMARY KEY (a) +ENGINE = ReplicatedMergeTree('/test3/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/', '1') PRIMARY KEY (a) SETTINGS min_bytes_for_wide_part = 0, max_parts_to_merge_at_once=3, enable_vertical_merge_algorithm=1, @@ -119,9 +119,9 @@ function check() if [ "$expect_broken_part" = "proj" ] then - $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT c FROM test WHERE d == 12;" 2>&1 | grep -o $expected_error + $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT c FROM test WHERE d == 12 ORDER BY c;" 2>&1 | grep -o $expected_error else - $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT c FROM test WHERE d == 12 OR d == 16;" + $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c;" echo 'used projections' $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; @@ -134,9 +134,9 @@ function check() if [ "$expect_broken_part" = "proj_2" ] then - $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT d FROM test WHERE c == 12;" 2>&1 | grep -o $expected_error + $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT d FROM test WHERE c == 12 ORDER BY d;" 2>&1 | grep -o $expected_error else - $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT d FROM test WHERE c == 12 OR c == 16;" + $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d;" echo 'used projections' $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; @@ -173,7 +173,7 @@ function materialize_projection function check_table_full() { echo 'check table full' - $CLICKHOUSE_CLIENT -q "CHECK TABLE test SETTINGS check_query_single_value_result = 0" + $CLICKHOUSE_CLIENT -q "CHECK TABLE test SETTINGS check_query_single_value_result = 0" | grep "broken" } From 8ea4e302a50db872a798c6cd39c6f5edb255ec49 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 16 Nov 2023 19:43:32 +0100 Subject: [PATCH 003/325] Fix style check --- .../0_stateless/02916_broken_projection.sh | 31 ++++++++++++++++--- 1 file changed, 26 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index 4748506d9cf..bf0ec61fd76 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# shellcheck disable=SC2046 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -125,7 +126,7 @@ function check() echo 'used projections' $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; - SELECT query, projections FROM system.query_log WHERE query_id='$query_id' and type='QueryFinish' + SELECT query, projections FROM system.query_log WHERE current_database=currentDatabase() AND query_id='$query_id' AND type='QueryFinish' " fi @@ -140,7 +141,7 @@ function check() echo 'used projections' $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; - SELECT query, projections FROM system.query_log WHERE query_id='$query_id' and type='QueryFinish' + SELECT query, projections FROM system.query_log WHERE current_database=currentDatabase() AND query_id='$query_id' AND type='QueryFinish' " fi @@ -148,10 +149,20 @@ function check() $CLICKHOUSE_CLIENT -q "CHECK TABLE test" } -function optimize_no_wait() +function optimize() { + final=$1 + no_wait=$2 + echo 'optimize' - $CLICKHOUSE_CLIENT -nm -q "OPTIMIZE TABLE test SETTINGS alter_sync=0;" + query="OPTIMIZE TABLE test" + + if [ $final -eq 1 ]; then + query="$query FINAL" + if [ $no_wait -eq 1 ]; then + query="$query SETTINGS alter_sync=0" + + $CLICKHOUSE_CLIENT -nm -q $query } function reattach() @@ -234,7 +245,7 @@ insert 25 5 # 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 'proj_2' as it will see that one part does not have it anymore in the set of valid projections. -optimize_no_wait +optimize 0 1 sleep 2 $CLICKHOUSE_CLIENT -nm -q " @@ -276,6 +287,16 @@ materialize_projection proj_2 check_table_full +break_projection proj all_3_5_1_7 data + +insert 30 5 + +optimize 1 0 + +insert 35 5 + +optimize 1 0 + check $CLICKHOUSE_CLIENT -nm -q " From 42b2fe9adcf4596e8e36231068911c5dbdc4948f Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 Nov 2023 13:21:35 +0100 Subject: [PATCH 004/325] Fxi --- src/Common/ErrorCodes.cpp | 1 + .../ReplicatedMergeTreePartCheckThread.cpp | 10 +- src/Storages/MergeTree/checkDataPart.cpp | 17 +++- .../02916_broken_projection.reference | 93 ++++++++++--------- .../0_stateless/02916_broken_projection.sh | 39 ++++---- 5 files changed, 94 insertions(+), 66 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 28f8e6c6021..9c3aab5ad01 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -586,6 +586,7 @@ M(704, CANNOT_USE_QUERY_CACHE_WITH_NONDETERMINISTIC_FUNCTIONS) \ M(705, TABLE_NOT_EMPTY) \ M(706, LIBSSH_ERROR) \ + M(707, BROKEN_PROJECTION) \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index 4468cf8e3bf..ba4d4869025 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(); @@ -385,17 +385,19 @@ ReplicatedCheckResult ReplicatedMergeTreePartCheckThread::checkPartImpl(const St if (isRetryableException(std::current_exception())) throw; - tryLogCurrentException(log, __PRETTY_FUNCTION__); - PreformattedMessage message; if (is_broken_projection) { - message = PreformattedMessage::create("Part {} has a broken projection. It will be ignored.", part_name); + message = PreformattedMessage::create( + "Part {} has a broken projections. It will be ignored. Broken projections info: \n{}", + part_name, getCurrentExceptionMessage(false)); LOG_DEBUG(log, message); result.action = ReplicatedCheckResult::DoNothing; } else { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + message = PreformattedMessage::create("Part {} looks broken. Removing it and will try to fetch.", part_name); LOG_ERROR(log, message); result.action = ReplicatedCheckResult::TryFetchMissing; diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 8feabf344b5..3bb6f763c8b 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -42,6 +42,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; } @@ -272,6 +273,7 @@ static IMergeTreeDataPart::Checksums checkDataPart( } } + std::string broken_projections_message; for (const auto & [name, projection] : data_part->getProjectionParts()) { if (is_cancelled()) @@ -307,7 +309,15 @@ static IMergeTreeDataPart::Checksums checkDataPart( is_broken_projection = true; if (throw_on_broken_projection) - throw; + { + 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); @@ -320,6 +330,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.data()); + } + if (require_checksums && !projections_on_disk.empty()) { throw Exception(ErrorCodes::UNEXPECTED_FILE_IN_DATA_PART, diff --git a/tests/queries/0_stateless/02916_broken_projection.reference b/tests/queries/0_stateless/02916_broken_projection.reference index 62966036eed..aee18a21fb8 100644 --- a/tests/queries/0_stateless/02916_broken_projection.reference +++ b/tests/queries/0_stateless/02916_broken_projection.reference @@ -7,42 +7,40 @@ all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] -select from projection 'proj' +select from projection 'proj', expect error: 12 16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['default.test.proj'] -select from projection 'proj_2' +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['test.test.proj'] +select from projection 'proj_2', expect error: 12 16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['default.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['test.test.proj_2'] check table 1 -0 broke metadata of part 'proj' (parent part: all_2_2_0) system.parts all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] -select from projection 'proj' +select from projection 'proj', expect error: 12 16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['default.test.proj'] -select from projection 'proj_2' +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['test.test.proj'] +select from projection 'proj_2', expect error: 12 16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['default.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['test.test.proj_2'] check table 0 broken projections info all_2_2_0 proj FILE_DOESNT_EXIST check table full -all_2_2_0 0 Part all_2_2_0 has a broken projection. It will be ignored. -0 +all_2_2_0 0 Part all_2_2_0 has a broken projections. It will be ignored. Broken projections info: \nCode: 707. DB::Exception: Part all_2_2_0 has a broken projection proj (error: Code: 107. DB::ErrnoException: Cannot open file /var/lib/clickhouse/store/521/521986ec-2fef-42c8-a402-83f937689286/all_2_2_0/proj.proj/columns.txt, errno: 2, strerror: No such file or directory. (FILE_DOESNT_EXIST) (version 23.11.1.1)). (BROKEN_PROJECTION) (version 23.11.1.1) broke data of part 'proj_2' (parent part: all_2_2_0) broken projections info all_2_2_0 proj FILE_DOESNT_EXIST @@ -51,13 +49,12 @@ all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] -select from projection 'proj' +select from projection 'proj', expect error: proj_2 12 16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['default.test.proj'] -select from projection 'proj_2' -FILE_DOESNT_EXIST +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['test.test.proj'] +select from projection 'proj_2', expect error: proj_2 check table 0 broken projections info @@ -68,19 +65,18 @@ all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] -select from projection 'proj' +select from projection 'proj', expect error: 12 16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['default.test.proj'] -select from projection 'proj_2' +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['test.test.proj'] +select from projection 'proj_2', expect error: 12 16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['default.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['test.test.proj_2'] check table 0 -0 broke data of part 'proj_2' (parent part: all_3_3_0) broken projections info all_2_2_0 proj FILE_DOESNT_EXIST @@ -88,6 +84,7 @@ all_2_2_0 proj_2 NO_FILE_IN_DATA_PART insert new part insert new part optimize +OPTIMIZE TABLE test SETTINGS alter_sync=0 0 broken projections info all_2_2_0 proj FILE_DOESNT_EXIST @@ -101,19 +98,18 @@ all_3_3_0 0 ['proj','proj_2'] all_3_5_1 1 ['proj'] all_4_4_0 0 ['proj','proj_2'] all_5_5_0 0 ['proj','proj_2'] -select from projection 'proj' +select from projection 'proj', expect error: 12 16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['default.test.proj'] -select from projection 'proj_2' +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['test.test.proj'] +select from projection 'proj_2', expect error: 12 16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['default.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['test.test.proj_2'] check table 0 -0 broke metadata of part 'proj' (parent part: all_1_1_0) Detach - Attach broken projections info @@ -121,7 +117,6 @@ all_1_1_0 proj NO_FILE_IN_DATA_PART all_2_2_0 proj NO_FILE_IN_DATA_PART all_2_2_0 proj_2 FILE_DOESNT_EXIST all_3_3_0 proj_2 FILE_DOESNT_EXIST -0 broke data of part 'proj_2' (parent part: all_1_1_0) Detach - Attach broken projections info @@ -138,21 +133,21 @@ all_3_3_0 0 ['proj','proj_2'] all_3_5_1 1 ['proj'] all_4_4_0 0 ['proj','proj_2'] all_5_5_0 0 ['proj','proj_2'] -select from projection 'proj' +select from projection 'proj', expect error: 12 16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['default.test.proj'] -select from projection 'proj_2' +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['test.test.proj'] +select from projection 'proj_2', expect error: 12 16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['default.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['test.test.proj_2'] check table 0 check table full -all_2_2_0 0 Part all_2_2_0 has a broken projection. It will be ignored. -all_1_1_0 0 Part all_1_1_0 has a broken projection. It will be ignored. +all_2_2_0 0 Part all_2_2_0 has a broken projections. It will be ignored. Broken projections info: \nCode: 707. DB::Exception: Part all_2_2_0 has a broken projection proj (error: Code: 107. DB::ErrnoException: Cannot open file /var/lib/clickhouse/store/521/521986ec-2fef-42c8-a402-83f937689286/all_2_2_0/proj.proj/columns.txt, errno: 2, strerror: No such file or directory. (FILE_DOESNT_EXIST) (version 23.11.1.1))\nPart all_2_2_0 has a broken projection proj_2 (error: Code: 226. DB::Exception: There is no file for column \'c\' in data part \'proj_2\'. (NO_FILE_IN_DATA_PART) (version 23.11.1.1)). (BROKEN_PROJECTION) (version 23.11.1.1) +all_1_1_0 0 Part all_1_1_0 has a broken projections. It will be ignored. Broken projections info: \nCode: 707. DB::Exception: Part all_1_1_0 has a broken projection proj (error: Code: 107. DB::ErrnoException: Cannot open file /var/lib/clickhouse/store/521/521986ec-2fef-42c8-a402-83f937689286/all_1_1_0/proj.proj/columns.txt, errno: 2, strerror: No such file or directory. (FILE_DOESNT_EXIST) (version 23.11.1.1))\nPart all_1_1_0 has a broken projection proj_2 (error: Code: 226. DB::Exception: There is no file for column \'c\' in data part \'proj_2\'. (NO_FILE_IN_DATA_PART) (version 23.11.1.1)). (BROKEN_PROJECTION) (version 23.11.1.1) materialize projection proj check table full system.parts @@ -167,45 +162,55 @@ all_3_5_1 0 ['proj'] all_3_5_1_6 1 ['proj'] all_4_4_0 0 ['proj','proj_2'] all_5_5_0 0 ['proj','proj_2'] -select from projection 'proj' +select from projection 'proj', expect error: 12 16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['default.test.proj'] -select from projection 'proj_2' +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['test.test.proj'] +select from projection 'proj_2', expect error: 12 16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['default.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['test.test.proj_2'] check table 1 materialize projection proj_2 check table full +broke data of part 'proj' (parent part: all_3_5_1_7) +insert new part +optimize +OPTIMIZE TABLE test FINAL +insert new part +optimize +OPTIMIZE TABLE test FINAL system.parts all_0_0_0 0 ['proj','proj_2'] all_0_0_0_6 0 ['proj','proj_2'] -all_0_0_0_7 1 ['proj','proj_2'] +all_0_0_0_7 0 ['proj','proj_2'] +all_0_8_2_7 1 ['proj_2'] all_1_1_0 0 ['proj','proj_2'] all_1_1_0_6 0 ['proj','proj_2'] -all_1_1_0_7 1 ['proj','proj_2'] +all_1_1_0_7 0 ['proj','proj_2'] all_2_2_0 0 ['proj','proj_2'] all_2_2_0_6 0 ['proj','proj_2'] -all_2_2_0_7 1 ['proj','proj_2'] +all_2_2_0_7 0 ['proj','proj_2'] all_3_3_0 0 ['proj','proj_2'] all_3_5_1 0 ['proj'] all_3_5_1_6 0 ['proj'] -all_3_5_1_7 1 ['proj','proj_2'] +all_3_5_1_7 0 ['proj','proj_2'] all_4_4_0 0 ['proj','proj_2'] all_5_5_0 0 ['proj','proj_2'] -select from projection 'proj' +all_8_8_0 0 ['proj','proj_2'] +all_9_9_0 1 ['proj','proj_2'] +select from projection 'proj', expect error: 12 16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['default.test.proj'] -select from projection 'proj_2' +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['test.test.proj_2'] +select from projection 'proj_2', expect error: 12 16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['default.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['test.test.proj_2'] check table 1 diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index bf0ec61fd76..bf382624787 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -24,7 +24,7 @@ CREATE TABLE test SELECT d ORDER BY c ) ) -ENGINE = ReplicatedMergeTree('/test3/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/', '1') PRIMARY KEY (a) +ENGINE = ReplicatedMergeTree('/test4/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/', '1') PRIMARY KEY (a) SETTINGS min_bytes_for_wide_part = 0, max_parts_to_merge_at_once=3, enable_vertical_merge_algorithm=1, @@ -65,18 +65,13 @@ function break_projection() LIMIT 1; ") - path=$($CLICKHOUSE_CLIENT -q "SELECT path FROM system.disks WHERE name='$disk_name'") - - # make sure path is absolute - $CLICKHOUSE_CLIENT -q "select throwIf(substring('$path', 1, 1) != '/', 'Path is relative: $path')" || exit - if [ "$break_type" = "data" ] then - rm "$path/$part_path/d.bin" - rm "$path/$part_path/c.bin" + rm "$part_path/d.bin" + rm "$part_path/c.bin" echo "broke data of part '$part_name' (parent part: $parent_name)" else - rm "$path/$part_path/columns.txt" + rm "$part_path/columns.txt" echo "broke metadata of part '$part_name' (parent part: $parent_name)" fi } @@ -115,12 +110,12 @@ function check() WHERE table='test' AND database=currentDatabase() ORDER BY name;" - echo "select from projection 'proj'" + echo "select from projection 'proj', expect error: $expect_broken_part" query_id=$(random 8) if [ "$expect_broken_part" = "proj" ] then - $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT c FROM test WHERE d == 12 ORDER BY c;" 2>&1 | grep -o $expected_error + $CLICKHOUSE_CLIENT --optimize_use_projections 1 --send_logs_level 'fatal' --query_id $query_id -q "SELECT c FROM test WHERE d == 12 ORDER BY c;" 2>&1 | grep -o $expected_error else $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c;" echo 'used projections' @@ -130,12 +125,12 @@ function check() " fi - echo "select from projection 'proj_2'" + echo "select from projection 'proj_2', expect error: $expect_broken_part" query_id=$(random 8) if [ "$expect_broken_part" = "proj_2" ] then - $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT d FROM test WHERE c == 12 ORDER BY d;" 2>&1 | grep -o $expected_error + $CLICKHOUSE_CLIENT --optimize_use_projections 1 --send_logs_level 'fatal' --query_id $query_id -q "SELECT d FROM test WHERE c == 12 ORDER BY d;" 2>&1 | grep -o $expected_error else $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d;" echo 'used projections' @@ -146,7 +141,9 @@ function check() fi echo 'check table' - $CLICKHOUSE_CLIENT -q "CHECK TABLE test" + $CLICKHOUSE_CLIENT -nm -q " + SET send_logs_level='fatal'; + CHECK TABLE test;" } function optimize() @@ -159,16 +156,21 @@ function optimize() if [ $final -eq 1 ]; then query="$query FINAL" + fi if [ $no_wait -eq 1 ]; then query="$query SETTINGS alter_sync=0" + fi - $CLICKHOUSE_CLIENT -nm -q $query + echo $query + + $CLICKHOUSE_CLIENT -q "$query" } function reattach() { echo 'Detach - Attach' $CLICKHOUSE_CLIENT -nm -q " + SET send_logs_level='fatal'; DETACH TABLE test; ATTACH TABLE test; " @@ -184,7 +186,10 @@ function materialize_projection function check_table_full() { echo 'check table full' - $CLICKHOUSE_CLIENT -q "CHECK TABLE test SETTINGS check_query_single_value_result = 0" | grep "broken" + $CLICKHOUSE_CLIENT -nm -q " + SET send_logs_level='fatal'; + CHECK TABLE test SETTINGS check_query_single_value_result = 0; +" | grep "broken" } @@ -300,5 +305,5 @@ optimize 1 0 check $CLICKHOUSE_CLIENT -nm -q " -DROP TABLE test; +DROP TABLE test SYNC; " From bcc87c01771414806fca705b5c9b5e0e925dea5f Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 Nov 2023 17:17:36 +0100 Subject: [PATCH 005/325] Better test --- .../ReplicatedMergeTreePartCheckThread.cpp | 3 +- src/Storages/System/StorageSystemDisks.cpp | 2 +- .../02916_broken_projection.reference | 43 +++++++++---------- .../0_stateless/02916_broken_projection.sh | 27 ++++++++---- 4 files changed, 42 insertions(+), 33 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index ba4d4869025..d058113e134 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -388,8 +388,9 @@ ReplicatedCheckResult ReplicatedMergeTreePartCheckThread::checkPartImpl(const St PreformattedMessage message; if (is_broken_projection) { + WriteBufferFromOwnString wb; message = PreformattedMessage::create( - "Part {} has a broken projections. It will be ignored. Broken projections info: \n{}", + "Part {} has a broken projections. It will be ignored. Broken projections info: {}", part_name, getCurrentExceptionMessage(false)); LOG_DEBUG(log, message); result.action = ReplicatedCheckResult::DoNothing; diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index 250fcdba641..23a00cc7ae5 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -63,7 +63,7 @@ Pipe StorageSystemDisks::read( for (const auto & [disk_name, disk_ptr] : context->getDisksMap()) { col_name->insert(disk_name); - col_path->insert(fs::absolute(disk_ptr->getPath()).string()); + col_path->insert(disk_ptr->getPath()); col_free->insert(disk_ptr->getAvailableSpace().value_or(std::numeric_limits::max())); col_total->insert(disk_ptr->getTotalSpace().value_or(std::numeric_limits::max())); col_unreserved->insert(disk_ptr->getUnreservedSpace().value_or(std::numeric_limits::max())); diff --git a/tests/queries/0_stateless/02916_broken_projection.reference b/tests/queries/0_stateless/02916_broken_projection.reference index aee18a21fb8..1b84ca96840 100644 --- a/tests/queries/0_stateless/02916_broken_projection.reference +++ b/tests/queries/0_stateless/02916_broken_projection.reference @@ -11,12 +11,12 @@ select from projection 'proj', expect error: 12 16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['test.test.proj'] +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj select from projection 'proj_2', expect error: 12 16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['test.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 1 broke metadata of part 'proj' (parent part: all_2_2_0) @@ -29,18 +29,18 @@ select from projection 'proj', expect error: 12 16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['test.test.proj'] +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj select from projection 'proj_2', expect error: 12 16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['test.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 0 broken projections info all_2_2_0 proj FILE_DOESNT_EXIST -check table full -all_2_2_0 0 Part all_2_2_0 has a broken projections. It will be ignored. Broken projections info: \nCode: 707. DB::Exception: Part all_2_2_0 has a broken projection proj (error: Code: 107. DB::ErrnoException: Cannot open file /var/lib/clickhouse/store/521/521986ec-2fef-42c8-a402-83f937689286/all_2_2_0/proj.proj/columns.txt, errno: 2, strerror: No such file or directory. (FILE_DOESNT_EXIST) (version 23.11.1.1)). (BROKEN_PROJECTION) (version 23.11.1.1) +check table full (all_2_2_0) +all_2_2_0 broke data of part 'proj_2' (parent part: all_2_2_0) broken projections info all_2_2_0 proj FILE_DOESNT_EXIST @@ -53,7 +53,7 @@ select from projection 'proj', expect error: proj_2 12 16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['test.test.proj'] +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj select from projection 'proj_2', expect error: proj_2 check table 0 @@ -69,12 +69,12 @@ select from projection 'proj', expect error: 12 16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['test.test.proj'] +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj select from projection 'proj_2', expect error: 12 16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['test.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 0 broke data of part 'proj_2' (parent part: all_3_3_0) @@ -102,12 +102,12 @@ select from projection 'proj', expect error: 12 16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['test.test.proj'] +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj select from projection 'proj_2', expect error: 12 16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['test.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 0 broke metadata of part 'proj' (parent part: all_1_1_0) @@ -137,19 +137,18 @@ select from projection 'proj', expect error: 12 16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['test.test.proj'] +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj select from projection 'proj_2', expect error: 12 16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['test.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 0 -check table full -all_2_2_0 0 Part all_2_2_0 has a broken projections. It will be ignored. Broken projections info: \nCode: 707. DB::Exception: Part all_2_2_0 has a broken projection proj (error: Code: 107. DB::ErrnoException: Cannot open file /var/lib/clickhouse/store/521/521986ec-2fef-42c8-a402-83f937689286/all_2_2_0/proj.proj/columns.txt, errno: 2, strerror: No such file or directory. (FILE_DOESNT_EXIST) (version 23.11.1.1))\nPart all_2_2_0 has a broken projection proj_2 (error: Code: 226. DB::Exception: There is no file for column \'c\' in data part \'proj_2\'. (NO_FILE_IN_DATA_PART) (version 23.11.1.1)). (BROKEN_PROJECTION) (version 23.11.1.1) -all_1_1_0 0 Part all_1_1_0 has a broken projections. It will be ignored. Broken projections info: \nCode: 707. DB::Exception: Part all_1_1_0 has a broken projection proj (error: Code: 107. DB::ErrnoException: Cannot open file /var/lib/clickhouse/store/521/521986ec-2fef-42c8-a402-83f937689286/all_1_1_0/proj.proj/columns.txt, errno: 2, strerror: No such file or directory. (FILE_DOESNT_EXIST) (version 23.11.1.1))\nPart all_1_1_0 has a broken projection proj_2 (error: Code: 226. DB::Exception: There is no file for column \'c\' in data part \'proj_2\'. (NO_FILE_IN_DATA_PART) (version 23.11.1.1)). (BROKEN_PROJECTION) (version 23.11.1.1) +check table full (all_1_1_0) +all_1_1_0 materialize projection proj -check table full +check table full () system.parts all_0_0_0 0 ['proj','proj_2'] all_0_0_0_6 1 ['proj','proj_2'] @@ -166,16 +165,16 @@ select from projection 'proj', expect error: 12 16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['test.test.proj'] +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj select from projection 'proj_2', expect error: 12 16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['test.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 1 materialize projection proj_2 -check table full +check table full () broke data of part 'proj' (parent part: all_3_5_1_7) insert new part optimize @@ -206,11 +205,11 @@ select from projection 'proj', expect error: 12 16 used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; ['test.test.proj_2'] +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj_2 select from projection 'proj_2', expect error: 12 16 used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; ['test.test.proj_2'] +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 1 diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index bf382624787..a522de42c89 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -121,7 +121,7 @@ function check() echo 'used projections' $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; - SELECT query, projections FROM system.query_log WHERE current_database=currentDatabase() AND query_id='$query_id' AND type='QueryFinish' + SELECT query, splitByChar('.', arrayJoin(projections))[-1] FROM system.query_log WHERE current_database=currentDatabase() AND query_id='$query_id' AND type='QueryFinish' " fi @@ -136,7 +136,7 @@ function check() echo 'used projections' $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; - SELECT query, projections FROM system.query_log WHERE current_database=currentDatabase() AND query_id='$query_id' AND type='QueryFinish' + SELECT query, splitByChar('.', arrayJoin(projections))[-1] FROM system.query_log WHERE current_database=currentDatabase() AND query_id='$query_id' AND type='QueryFinish' " fi @@ -185,11 +185,20 @@ function materialize_projection function check_table_full() { - echo 'check table full' - $CLICKHOUSE_CLIENT -nm -q " - SET send_logs_level='fatal'; - CHECK TABLE test SETTINGS check_query_single_value_result = 0; -" | grep "broken" + echo "check table full ($1)" + expect_broken_part=$1 + if [ "$expect_broken_part" = "" ] + then + $CLICKHOUSE_CLIENT -nm -q " + SET send_logs_level='fatal'; + CHECK TABLE test SETTINGS check_query_single_value_result = 0; + " | grep "broken" + else + $CLICKHOUSE_CLIENT -nm -q " + SET send_logs_level='fatal'; + CHECK TABLE test SETTINGS check_query_single_value_result = 0; + " | grep "broken" | grep -o $expect_broken_part | head -n 1 + fi } @@ -216,7 +225,7 @@ check broken_projections_info # Check table query will also show a list of parts which have broken projections. -check_table_full +check_table_full "all_2_2_0" # Break data file of projection 'proj_2' for part all_2_2_0 break_projection proj_2 all_2_2_0 data @@ -280,7 +289,7 @@ broken_projections_info check -check_table_full +check_table_full all_1_1_0 materialize_projection proj From e8d99cb29654645c5a89d6cb15856b48a55d7bdf Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 22 Nov 2023 12:34:31 +0100 Subject: [PATCH 006/325] Fix style check --- tests/queries/0_stateless/02916_broken_projection.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index a522de42c89..6ed92e2e06e 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -54,8 +54,8 @@ function break_projection() parent_name=$2 break_type=$3 - read -r disk_name part_path <<< $($CLICKHOUSE_CLIENT -nm -q " - SELECT disk_name, path + read -r part_path <<< $($CLICKHOUSE_CLIENT -nm -q " + SELECT path FROM system.projection_parts WHERE table='test' AND database=currentDatabase() From a57e612cf2ef657801cdeefb8410caf5cab804a2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Nov 2023 16:08:49 +0100 Subject: [PATCH 007/325] Fxi tests --- src/Storages/StorageMergeTree.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 74277616e95..84b48bb650b 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2247,7 +2247,7 @@ std::optional StorageMergeTree::checkDataNext(DataValidationTasksPt { try { - auto calculated_checksums = checkDataPart(part, false, noop, /* is_cancelled */{}, /* throw_on_broken_projection */true); + 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); @@ -2268,7 +2268,7 @@ std::optional StorageMergeTree::checkDataNext(DataValidationTasksPt { try { - checkDataPart(part, true, noop, /* is_cancelled */{}, /* throw_on_broken_projection */true); + checkDataPart(part, true, noop, /* is_cancelled */[]{ return false; }, /* throw_on_broken_projection */true); return CheckResult(part->name, true, ""); } catch (...) From 8ebbc8d85dc3f1e37d109ddb1ad1a05a55283a79 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Nov 2023 18:37:40 +0100 Subject: [PATCH 008/325] Update 02117_show_create_table_system.reference --- .../0_stateless/02117_show_create_table_system.reference | 3 +++ 1 file changed, 3 insertions(+) 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 9ed905a0df8..e122de8ef6c 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -685,6 +685,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 b4dab194954845b76d1ce9a6bf8b18dded059d74 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 Nov 2023 12:42:09 +0100 Subject: [PATCH 009/325] Fix test --- .../0_stateless/02916_broken_projection.reference | 1 + tests/queries/0_stateless/02916_broken_projection.sh | 12 +++++++++--- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02916_broken_projection.reference b/tests/queries/0_stateless/02916_broken_projection.reference index 1b84ca96840..1f072e207a7 100644 --- a/tests/queries/0_stateless/02916_broken_projection.reference +++ b/tests/queries/0_stateless/02916_broken_projection.reference @@ -55,6 +55,7 @@ select from projection 'proj', expect error: proj_2 used projections SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj select from projection 'proj_2', expect error: proj_2 +FILE_DOESNT_EXIST check table 0 broken projections info diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index 6ed92e2e06e..80805330577 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -115,7 +115,10 @@ function check() if [ "$expect_broken_part" = "proj" ] then - $CLICKHOUSE_CLIENT --optimize_use_projections 1 --send_logs_level 'fatal' --query_id $query_id -q "SELECT c FROM test WHERE d == 12 ORDER BY c;" 2>&1 | grep -o $expected_error + $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -nm -q " +SET send_logs_level='fatal'; +SELECT c FROM test WHERE d == 12 ORDER BY c; +" 2>&1 | grep -oF "$expected_error" else $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c;" echo 'used projections' @@ -130,7 +133,10 @@ function check() if [ "$expect_broken_part" = "proj_2" ] then - $CLICKHOUSE_CLIENT --optimize_use_projections 1 --send_logs_level 'fatal' --query_id $query_id -q "SELECT d FROM test WHERE c == 12 ORDER BY d;" 2>&1 | grep -o $expected_error + $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -nm -q " +SET send_logs_level='fatal'; +SELECT d FROM test WHERE c == 12 ORDER BY d; +" 2>&1 | grep -oF "$expected_error" else $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d;" echo 'used projections' @@ -234,7 +240,7 @@ break_projection proj_2 all_2_2_0 data broken_projections_info # Select now fails with error "File doesn't exist" -check "proj_2" "FILE_DOESNT_EXIST" +check "proj_2" FILE_DOESNT_EXIST # Projection 'proj_2' from part all_2_2_0 will now appear in broken parts info. broken_projections_info From a6972e7c90fd8ff775855cac13f47f9cd46b2da1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 28 Nov 2023 10:22:10 +0100 Subject: [PATCH 010/325] Fxi --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 85ce112d9a1..be665a64f1c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1145,7 +1145,7 @@ void IMergeTreeDataPart::loadChecksums(bool require) LOG_WARNING(storage.log, "Checksums for part {} not found. Will calculate them from data on disk.", name); bool noop; - checksums = checkDataPart(shared_from_this(), false, noop, /* is_cancelled */{}, /* throw_on_broken_projection */false); + checksums = checkDataPart(shared_from_this(), false, noop, /* is_cancelled */[]{ return false; }, /* throw_on_broken_projection */false); writeChecksums(checksums, {}); bytes_on_disk = checksums.getTotalSizeOnDisk(); From 0e11eeaea546dd41231a4f180b877ada1291a23d Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 30 Nov 2023 13:52:08 +0100 Subject: [PATCH 011/325] Allow to backup and restore parts with broken projections --- src/Backups/BackupSettings.cpp | 2 + src/Backups/BackupSettings.h | 6 + .../MergeTree/DataPartStorageOnDiskBase.cpp | 35 +- .../MergeTree/DataPartStorageOnDiskBase.h | 4 +- src/Storages/MergeTree/IDataPartStorage.h | 4 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 15 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 3 +- src/Storages/MergeTree/MergeTreeData.cpp | 9 +- .../02916_broken_projection.reference | 226 +++++++++- .../0_stateless/02916_broken_projection.sh | 426 +++++++++++++----- 10 files changed, 588 insertions(+), 142 deletions(-) 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/Storages/MergeTree/DataPartStorageOnDiskBase.cpp b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp index 7fc8187aee5..6e5cbdde355 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDiskBase.cpp @@ -334,7 +334,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; @@ -376,7 +378,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; @@ -384,8 +386,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) { @@ -410,6 +414,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 1826e84c28d..6176a13c27b 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 072cb29626e..b3a6ab203d5 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -222,7 +222,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 be665a64f1c..940b3991067 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -647,13 +647,14 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks loadIndex(); /// Must be called after loadIndexGranularity as it uses the value of `index_granularity` 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 */); + has_broken_projections = !loadProjections(require_columns_checksums, check_consistency, false /* if_not_loaded */); } - if (check_consistency) + if (check_consistency && !has_broken_projections) checkConsistency(require_columns_checksums); loadDefaultCompressionCodec(); @@ -715,9 +716,10 @@ 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) +bool IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool check_consistency, bool if_not_loaded) { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); + bool has_broken_projection = false; for (const auto & projection : metadata_snapshot->projections) { auto path = projection.name + ".proj"; @@ -742,16 +744,19 @@ void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool ch if (isRetryableException(std::current_exception())) throw; + auto message = getCurrentExceptionMessage(true); LOG_ERROR(&Poco::Logger::get("IMergeTreeDataPart"), - "Cannot load projection {}, will consider it broken", projection.name); + "Cannot load projection {}, will consider it broken. Reason: {}", projection.name, message); - part->setBrokenReason(getCurrentExceptionMessage(false), getCurrentExceptionCode()); + has_broken_projection = true; + part->setBrokenReason(message, getCurrentExceptionCode()); } addProjectionPart(projection.name, std::move(part)); } } } + return has_broken_projection; } void IMergeTreeDataPart::loadIndexGranularity() diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 9af2c16f1e8..6e276284f4c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -421,7 +421,8 @@ public: bool hasBrokenProjection(const String & projection_name) const; - void loadProjections(bool require_columns_checksums, bool check_consistency, bool if_not_loaded = false); + /// Return true, if all projections were loaded successfully and none was marked as broken. + bool loadProjections(bool require_columns_checksums, bool check_consistency, bool if_not_loaded = false); void setBrokenReason(const String & message, int code) const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8683e4293e9..c95aee88aee 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5113,7 +5113,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; @@ -5125,7 +5125,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) @@ -5138,7 +5139,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) diff --git a/tests/queries/0_stateless/02916_broken_projection.reference b/tests/queries/0_stateless/02916_broken_projection.reference index 1f072e207a7..4c4901ae99f 100644 --- a/tests/queries/0_stateless/02916_broken_projection.reference +++ b/tests/queries/0_stateless/02916_broken_projection.reference @@ -19,6 +19,7 @@ used projections SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 1 +0 broke metadata of part 'proj' (parent part: all_2_2_0) system.parts all_0_0_0 1 ['proj','proj_2'] @@ -39,8 +40,9 @@ check table 0 broken projections info all_2_2_0 proj FILE_DOESNT_EXIST -check table full (all_2_2_0) +check table full (test - all_2_2_0) all_2_2_0 +0 broke data of part 'proj_2' (parent part: all_2_2_0) broken projections info all_2_2_0 proj FILE_DOESNT_EXIST @@ -78,6 +80,7 @@ used projections SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 0 +0 broke data of part 'proj_2' (parent part: all_3_3_0) broken projections info all_2_2_0 proj FILE_DOESNT_EXIST @@ -111,6 +114,7 @@ used projections SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 0 +0 broke metadata of part 'proj' (parent part: all_1_1_0) Detach - Attach broken projections info @@ -118,6 +122,7 @@ all_1_1_0 proj NO_FILE_IN_DATA_PART all_2_2_0 proj NO_FILE_IN_DATA_PART all_2_2_0 proj_2 FILE_DOESNT_EXIST all_3_3_0 proj_2 FILE_DOESNT_EXIST +0 broke data of part 'proj_2' (parent part: all_1_1_0) Detach - Attach broken projections info @@ -146,10 +151,10 @@ used projections SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 0 -check table full (all_1_1_0) +check table full (test - all_1_1_0) all_1_1_0 materialize projection proj -check table full () +check table full (test - ) system.parts all_0_0_0 0 ['proj','proj_2'] all_0_0_0_6 1 ['proj','proj_2'] @@ -175,7 +180,8 @@ SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 1 materialize projection proj_2 -check table full () +check table full (test - ) +0 broke data of part 'proj' (parent part: all_3_5_1_7) insert new part optimize @@ -214,3 +220,215 @@ used projections SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 1 +insert new part +insert new part +insert new part +insert new part +system.parts +all_0_0_0 1 ['proj','proj_2'] +all_1_1_0 1 ['proj','proj_2'] +select from projection 'proj', expect error: +used projections +SELECT c FROM test2 WHERE d == 12 OR d == 16 ORDER BY c; proj +select from projection 'proj_2', expect error: +used projections +SELECT d FROM test2 WHERE c == 12 OR c == 16 ORDER BY d; proj_2 +check table +1 +system.parts +all_1_1_0 1 ['proj','proj_2'] +select from projection 'proj', expect error: +used projections +SELECT c FROM test2_replica WHERE d == 12 OR d == 16 ORDER BY c; proj +select from projection 'proj_2', expect error: +used projections +SELECT d FROM test2_replica WHERE c == 12 OR c == 16 ORDER BY d; proj_2 +check table +1 +0 +broke data of part 'proj' (parent part: all_0_0_0) +check table full (test2 - all_0_0_0) +all_0_0_0 +system.parts +all_0_0_0 1 ['proj','proj_2'] +all_1_1_0 1 ['proj','proj_2'] +select from projection 'proj', expect error: +used projections +SELECT c FROM test2 WHERE d == 12 OR d == 16 ORDER BY c; proj +select from projection 'proj_2', expect error: +used projections +SELECT d FROM test2 WHERE c == 12 OR c == 16 ORDER BY d; proj_2 +check table +0 +broke data of part 'all_0_0_0' +check table full (test2 - all_0_0_0) +all_0_0_0 +system.parts +all_0_0_0 0 ['proj','proj_2'] +all_1_1_0 1 ['proj','proj_2'] +select from projection 'proj', expect error: +used projections +SELECT c FROM test2 WHERE d == 12 OR d == 16 ORDER BY c; proj +select from projection 'proj_2', expect error: +used projections +SELECT d FROM test2 WHERE c == 12 OR c == 16 ORDER BY d; proj_2 +check table +1 +system.parts +all_0_0_0 1 ['proj','proj_2'] +all_1_1_0 1 ['proj','proj_2'] +select from projection 'proj', expect error: +used projections +SELECT c FROM test2 WHERE d == 12 OR d == 16 ORDER BY c; proj +select from projection 'proj_2', expect error: +used projections +SELECT d FROM test2 WHERE c == 12 OR c == 16 ORDER BY d; proj_2 +check table +1 +insert new part +insert new part +insert new part +insert new part +system.parts +all_0_0_0 1 ['proj','proj_2'] +all_1_1_0 1 ['proj','proj_2'] +all_2_2_0 1 ['proj','proj_2'] +all_3_3_0 1 ['proj','proj_2'] +select from projection 'proj', expect error: +12 +16 +used projections +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj +select from projection 'proj_2', expect error: +12 +16 +used projections +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 +check table +1 +0 +broke data of part 'proj' (parent part: all_2_2_0) +system.parts +all_0_0_0 1 ['proj','proj_2'] +all_1_1_0 1 ['proj','proj_2'] +all_2_2_0 1 ['proj','proj_2'] +all_3_3_0 1 ['proj','proj_2'] +select from projection 'proj', expect error: proj +FILE_DOESNT_EXIST +select from projection 'proj_2', expect error: proj +12 +16 +used projections +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 +check table +0 +broken projections info +all_2_2_0 proj NO_FILE_IN_DATA_PART +BACKUP_CREATED +RESTORED +system.parts +all_0_0_0 1 ['proj','proj_2'] +all_1_1_0 1 ['proj','proj_2'] +all_2_2_0 1 ['proj','proj_2'] +all_3_3_0 1 ['proj','proj_2'] +select from projection 'proj', expect error: proj +FILE_DOESNT_EXIST +select from projection 'proj_2', expect error: proj +12 +16 +used projections +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 +check table +0 +broken projections info +all_2_2_0 proj NO_FILE_IN_DATA_PART +0 +broke all data of part 'proj' (parent part: all_2_2_0) +system.parts +all_0_0_0 1 ['proj','proj_2'] +all_1_1_0 1 ['proj','proj_2'] +all_2_2_0 1 ['proj','proj_2'] +all_3_3_0 1 ['proj','proj_2'] +select from projection 'proj', expect error: proj +select from projection 'proj_2', expect error: proj +12 +16 +used projections +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 +check table +0 +broken projections info +all_2_2_0 proj NO_FILE_IN_DATA_PART +FILE_DOESNT_EXIST +materialize projection proj +system.parts +all_0_0_0 0 ['proj','proj_2'] +all_0_0_0_4 1 ['proj','proj_2'] +all_1_1_0 0 ['proj','proj_2'] +all_1_1_0_4 1 ['proj','proj_2'] +all_2_2_0 0 ['proj','proj_2'] +all_2_2_0_4 1 ['proj','proj_2'] +all_3_3_0 0 ['proj','proj_2'] +all_3_3_0_4 1 ['proj','proj_2'] +select from projection 'proj', expect error: proj +select from projection 'proj_2', expect error: proj +12 +16 +used projections +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 +check table +1 +broken projections info +all_2_2_0 proj NO_FILE_IN_DATA_PART +BACKUP_CREATED +RESTORED +system.parts +all_0_0_0 1 ['proj','proj_2'] +all_1_1_0 1 ['proj','proj_2'] +all_2_2_0 1 ['proj','proj_2'] +all_3_3_0 1 ['proj','proj_2'] +select from projection 'proj', expect error: proj +select from projection 'proj_2', expect error: proj +12 +16 +used projections +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 +check table +1 +0 +broke all data of part 'proj' (parent part: all_2_2_0) +system.parts +all_0_0_0 1 ['proj','proj_2'] +all_1_1_0 1 ['proj','proj_2'] +all_2_2_0 1 ['proj','proj_2'] +all_3_3_0 1 ['proj','proj_2'] +select from projection 'proj', expect error: proj +select from projection 'proj_2', expect error: proj +12 +16 +used projections +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 +check table +0 +broken projections info +all_2_2_0 proj FILE_DOESNT_EXIST +BACKUP_CREATED +RESTORED +system.parts +all_0_0_0 1 ['proj','proj_2'] +all_1_1_0 1 ['proj','proj_2'] +all_2_2_0 1 ['proj_2'] +all_3_3_0 1 ['proj','proj_2'] +select from projection 'proj', expect error: +12 +16 +used projections +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj +select from projection 'proj_2', expect error: +12 +16 +used projections +SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 +check table +0 +broken projections info diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index 80805330577..1555139e16f 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -5,35 +5,40 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " -DROP TABLE IF EXISTS test SYNC; -CREATE TABLE test -( - a String, - b String, - c Int32, - d Int32, - e Int32, +function create_table() +{ + test_id=$1 + name=$2 + replica=$3 + $CLICKHOUSE_CLIENT -nm -q " + DROP TABLE IF EXISTS $name SYNC; + CREATE TABLE $name + ( + a String, + b String, + c Int64, + d Int64, + e Int64, - PROJECTION proj - ( - SELECT c ORDER BY d - ), - PROJECTION proj_2 - ( - SELECT d ORDER BY c + PROJECTION proj + ( + SELECT c ORDER BY d + ), + PROJECTION proj_2 + ( + SELECT d ORDER BY c + ) ) -) -ENGINE = ReplicatedMergeTree('/test4/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/', '1') PRIMARY KEY (a) -SETTINGS min_bytes_for_wide_part = 0, - max_parts_to_merge_at_once=3, - enable_vertical_merge_algorithm=1, - vertical_merge_algorithm_min_rows_to_activate = 1, - vertical_merge_algorithm_min_columns_to_activate = 1, - vertical_merge_algorithm_min_columns_to_activate = 1; -" - -table_uuid=$($CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.tables WHERE table='test' and database=currentDatabase()") + ENGINE = ReplicatedMergeTree('/test_broken_projection_24_$test_id/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/', '$replica') ORDER BY a + SETTINGS min_bytes_for_wide_part = 0, + max_parts_to_merge_at_once=3, + enable_vertical_merge_algorithm=1, + vertical_merge_algorithm_min_rows_to_activate = 1, + vertical_merge_algorithm_min_columns_to_activate = 1, + vertical_merge_algorithm_min_columns_to_activate = 1, + compress_primary_key=0; + " +} function random() { @@ -42,49 +47,88 @@ function random() function insert() { - offset=$1 - size=$2 + table=$1 + offset=$2 + size=$3 echo 'insert new part' - $CLICKHOUSE_CLIENT -q "INSERT INTO test SELECT number, number, number, number, number%2 FROM numbers($offset, $size);" + $CLICKHOUSE_CLIENT -q "INSERT INTO $table SELECT number, number, number, number, number%2 FROM numbers($offset, $size);" } function break_projection() { - part_name=$1 - parent_name=$2 - break_type=$3 + table=$1 + part_name=$2 + parent_name=$3 + break_type=$4 read -r part_path <<< $($CLICKHOUSE_CLIENT -nm -q " SELECT path FROM system.projection_parts - WHERE table='test' + WHERE table='$table' AND database=currentDatabase() AND active=1 AND part_name='$part_name' AND parent_name='$parent_name' + ORDER BY modification_time DESC LIMIT 1; ") + $CLICKHOUSE_CLIENT -q "select throwIf(substring('$part_path', 1, 1) != '/', 'Path is relative: $part_path')" || exit + if [ "$break_type" = "data" ] then rm "$part_path/d.bin" rm "$part_path/c.bin" echo "broke data of part '$part_name' (parent part: $parent_name)" - else + fi + if [ "$break_type" = "metadata" ] + then rm "$part_path/columns.txt" echo "broke metadata of part '$part_name' (parent part: $parent_name)" fi + if [ "$break_type" = "part" ] + then + rm -r "$part_path" + echo "broke all data of part '$part_name' (parent part: $parent_name)" + fi +} + +function break_part() +{ + table=$1 + part_name=$2 + + read -r part_path <<< $($CLICKHOUSE_CLIENT -nm -q " + SELECT path + FROM system.parts + WHERE table='$table' + AND database=currentDatabase() + AND active=1 + AND part_name='$part_name' + ORDER BY modification_time DESC + LIMIT 1; + ") + + if [ "$part_path" = "" ] + then + echo "Part path is empty" + exit + fi + + rm $part_path/columns.txt + echo "broke data of part '$part_name'" } function broken_projections_info() { + table=$1 echo 'broken projections info' $CLICKHOUSE_CLIENT -q " SELECT parent_name, name, errors.name FROM ( SELECT parent_name, name, exception_code FROM system.projection_parts - WHERE table='test' + WHERE table='$table' AND database=currentDatabase() AND is_broken = 1 ) AS parts_info @@ -96,18 +140,19 @@ function broken_projections_info() function check() { + table=$1 expect_broken_part="" expected_error="" - if [ $# -ne 0 ]; then - expect_broken_part=$1 - expected_error=$2 + if [ $# -gt 1 ]; then + expect_broken_part=$2 + expected_error=$3 fi echo 'system.parts' $CLICKHOUSE_CLIENT -q " SELECT name, active, projections FROM system.parts - WHERE table='test' AND database=currentDatabase() + WHERE table='$table' AND database=currentDatabase() ORDER BY name;" echo "select from projection 'proj', expect error: $expect_broken_part" @@ -117,10 +162,10 @@ function check() then $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -nm -q " SET send_logs_level='fatal'; -SELECT c FROM test WHERE d == 12 ORDER BY c; +SELECT c FROM $table WHERE d == 12 ORDER BY c; " 2>&1 | grep -oF "$expected_error" else - $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c;" + $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT c FROM $table WHERE d == 12 OR d == 16 ORDER BY c;" echo 'used projections' $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; @@ -135,10 +180,10 @@ SELECT c FROM test WHERE d == 12 ORDER BY c; then $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -nm -q " SET send_logs_level='fatal'; -SELECT d FROM test WHERE c == 12 ORDER BY d; +SELECT d FROM $table WHERE c == 12 ORDER BY d; " 2>&1 | grep -oF "$expected_error" else - $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d;" + $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT d FROM $table WHERE c == 12 OR c == 16 ORDER BY d;" echo 'used projections' $CLICKHOUSE_CLIENT -nm -q " SYSTEM FLUSH LOGS; @@ -149,7 +194,7 @@ SELECT d FROM test WHERE c == 12 ORDER BY d; echo 'check table' $CLICKHOUSE_CLIENT -nm -q " SET send_logs_level='fatal'; - CHECK TABLE test;" + CHECK TABLE $table;" } function optimize() @@ -184,141 +229,274 @@ function reattach() function materialize_projection { - projection=$1 + table=$1 + projection=$2 echo "materialize projection $projection" - $CLICKHOUSE_CLIENT -q "ALTER TABLE test MATERIALIZE PROJECTION $projection SETTINGS mutations_sync=2" + $CLICKHOUSE_CLIENT -q "ALTER TABLE $table MATERIALIZE PROJECTION $projection SETTINGS mutations_sync=2" } function check_table_full() { - echo "check table full ($1)" - expect_broken_part=$1 + table=$1 + expect_broken_part=$2 + echo "check table full ($1 - $2)" if [ "$expect_broken_part" = "" ] then $CLICKHOUSE_CLIENT -nm -q " SET send_logs_level='fatal'; - CHECK TABLE test SETTINGS check_query_single_value_result = 0; + CHECK TABLE $table SETTINGS check_query_single_value_result = 0; " | grep "broken" else $CLICKHOUSE_CLIENT -nm -q " SET send_logs_level='fatal'; - CHECK TABLE test SETTINGS check_query_single_value_result = 0; + CHECK TABLE $table SETTINGS check_query_single_value_result = 0; " | grep "broken" | grep -o $expect_broken_part | head -n 1 fi } +function test1() +{ + create_table test1 test 1 -insert 0 5 + table_uuid=$($CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.tables WHERE table='test' and database=currentDatabase()") -insert 5 5 + insert test 0 5 -insert 10 5 + insert test 5 5 -insert 15 5 + insert test 10 5 -check + insert test 15 5 -# Break metadata file of projection 'proj' -break_projection proj all_2_2_0 metadata + check test -# Do select and after "check table" query. -# Select works because it does not read columns.txt. -check + # Break metadata file of projection 'proj' + break_projection test proj all_2_2_0 metadata -# Projection 'proj' from part all_2_2_0 will now appear in broken parts info -# because it was marked broken during "check table" query. -# TODO: try to mark it during select as well -broken_projections_info + # Do select and after "check table" query. + # Select works because it does not read columns.txt. + check test -# Check table query will also show a list of parts which have broken projections. -check_table_full "all_2_2_0" + # Projection 'proj' from part all_2_2_0 will now appear in broken parts info + # because it was marked broken during "check table" query. + # TODO: try to mark it during select as well + broken_projections_info test -# Break data file of projection 'proj_2' for part all_2_2_0 -break_projection proj_2 all_2_2_0 data + # Check table query will also show a list of parts which have broken projections. + check_table_full test "all_2_2_0" -# It will not yet appear in broken projections info. -broken_projections_info + # Break data file of projection 'proj_2' for part all_2_2_0 + break_projection test proj_2 all_2_2_0 data -# Select now fails with error "File doesn't exist" -check "proj_2" FILE_DOESNT_EXIST + # It will not yet appear in broken projections info. + broken_projections_info test -# Projection 'proj_2' from part all_2_2_0 will now appear in broken parts info. -broken_projections_info + # Select now fails with error "File doesn't exist" + check test "proj_2" FILE_DOESNT_EXIST -# Second select works, because projection is now marked as broken. -check + # Projection 'proj_2' from part all_2_2_0 will now appear in broken parts info. + broken_projections_info test -# Break data file of projection 'proj_2' for part all_3_3_0 -break_projection proj_2 all_3_3_0 data + # Second select works, because projection is now marked as broken. + check test -# It will not yet appear in broken projections info. -broken_projections_info + # Break data file of projection 'proj_2' for part all_3_3_0 + break_projection test proj_2 all_3_3_0 data -insert 20 5 + # It will not yet appear in broken projections info. + broken_projections_info test -insert 25 5 + insert test 20 5 -# Part all_3_3_0 has 'proj' and 'proj_2' projections, but 'proj_2' 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 'proj_2' of part all_3_3_0 and proj_2 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 'proj_2' as it will see that one part does not have it anymore in the set of valid projections. -optimize 0 1 -sleep 2 + insert test 25 5 -$CLICKHOUSE_CLIENT -nm -q " -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%proj_2.proj/c.bin%' -" + # Part all_3_3_0 has 'proj' and 'proj_2' projections, but 'proj_2' 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 'proj_2' of part all_3_3_0 and proj_2 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 'proj_2' as it will see that one part does not have it anymore in the set of valid projections. + optimize 0 1 + sleep 2 -# Projection 'proj_2' from part all_2_2_0 will now appear in broken parts info. -broken_projections_info + $CLICKHOUSE_CLIENT -nm -q " + 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%proj_2.proj/c.bin%' + " -check + # Projection 'proj_2' from part all_2_2_0 will now appear in broken parts info. + broken_projections_info test -break_projection proj all_1_1_0 metadata + check test -reattach + break_projection test proj all_1_1_0 metadata -broken_projections_info + reattach -break_projection proj_2 all_1_1_0 data + broken_projections_info test -reattach + break_projection test proj_2 all_1_1_0 data -broken_projections_info + reattach -check + broken_projections_info test -check_table_full all_1_1_0 + check test -materialize_projection proj + check_table_full test all_1_1_0 -check_table_full + materialize_projection test proj -check + check_table_full test -materialize_projection proj_2 + check test -check_table_full + materialize_projection test proj_2 -break_projection proj all_3_5_1_7 data + check_table_full test -insert 30 5 + break_projection test proj all_3_5_1_7 data -optimize 1 0 + insert test 30 5 -insert 35 5 + optimize 1 0 -optimize 1 0 + insert test 35 5 -check + optimize 1 0 -$CLICKHOUSE_CLIENT -nm -q " -DROP TABLE test SYNC; -" + check test +} + +function test2() +{ + create_table test2 test2 1 + + insert test2 0 5 + + insert test2 5 5 + + insert test 10 5 + + insert test 15 5 + + check test2 + + create_table test2 test2_replica 2 + + check test2_replica + + break_projection test2 proj all_0_0_0 data + + check_table_full test2 all_0_0_0 + + check test2 + + break_part test2 all_0_0_0 + + check_table_full test2 all_0_0_0 + + check test2 + + $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA test2;" + + check test2 +} + +CLICKHOUSE_TEST_UNIQUE_NAME="gghhhhhhhhhhhhhhhhhhh" +function test3() +{ + create_table test3 test 1 + + insert test 0 5 + + insert test 5 5 + + insert test 10 5 + + insert test 15 5 + + check test + + break_projection test proj all_2_2_0 data + + check test proj FILE_DOESNT_EXIST + + broken_projections_info test + + ${CLICKHOUSE_CLIENT} -nm --query " + backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}') settings check_projection_parts=false; + " | grep -o "BACKUP_CREATED" + + ${CLICKHOUSE_CLIENT} -nm --stacktrace --query " + drop table test sync; + restore table ${CLICKHOUSE_DATABASE}.test from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); + " | grep -o "RESTORED" + + check test proj FILE_DOESNT_EXIST + + broken_projections_info test + + break_projection test proj all_2_2_0 part + + check test proj + + broken_projections_info test + + ${CLICKHOUSE_CLIENT} -nm --query " + set send_logs_level='fatal'; + backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_2') + " 2>&1 | grep -o "FILE_DOESNT_EXIST" + + materialize_projection test proj + + check test proj + + broken_projections_info test + + ${CLICKHOUSE_CLIENT} -nm --query " + backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_3') + " | grep -o "BACKUP_CREATED" + + ${CLICKHOUSE_CLIENT} -nm --stacktrace --query " + drop table test sync; + restore table ${CLICKHOUSE_DATABASE}.test from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_3'); + " | grep -o "RESTORED" + + check test proj + + break_projection test proj all_2_2_0 part + + check test proj FILE_DOESNT_EXIST + + broken_projections_info test + + ${CLICKHOUSE_CLIENT} -nm --query " + backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_2') + settings check_projection_parts=false, allow_backup_broken_projections=true; + " | grep -o "BACKUP_CREATED" + + ${CLICKHOUSE_CLIENT} -nm --stacktrace --query " + drop table test sync; + restore table ${CLICKHOUSE_DATABASE}.test from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_2'); + " | grep -o "RESTORED" + + check test + + broken_projections_info test +} + +test1 +test2 +test3 + + +#$CLICKHOUSE_CLIENT -nm -q " +#DROP TABLE test SYNC; +#DROP TABLE test2 SYNC; +#DROP TABLE test2_replica SYNC; +#" From 6632589d72ed270626e012c86a78a8f0c8411fb3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 30 Nov 2023 13:54:22 +0100 Subject: [PATCH 012/325] Review fix --- src/Storages/MergeTree/MergeTreeData.cpp | 6 +++--- tests/queries/0_stateless/02916_broken_projection.sh | 10 +++++----- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c95aee88aee..1ba4153bc3e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7669,7 +7669,7 @@ MovePartsOutcome MergeTreeData::moveParts(const CurrentlyMovingPartsTaggerPtr & bool MergeTreeData::partsContainSameProjections(const DataPartPtr & left, const DataPartPtr & right, String & out_reason) { - auto remove_broken_parts = [](auto & parts) + auto remove_broken_parts_from_consideration = [](auto & parts) { std::set broken_projection_parts; for (const auto & [name, part] : parts) @@ -7684,8 +7684,8 @@ bool MergeTreeData::partsContainSameProjections(const DataPartPtr & left, const auto left_projection_parts = left->getProjectionParts(); auto right_projection_parts = right->getProjectionParts(); - remove_broken_parts(left_projection_parts); - remove_broken_parts(right_projection_parts); + 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()) { diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index 1555139e16f..60b21216d1a 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -495,8 +495,8 @@ test2 test3 -#$CLICKHOUSE_CLIENT -nm -q " -#DROP TABLE test SYNC; -#DROP TABLE test2 SYNC; -#DROP TABLE test2_replica SYNC; -#" +$CLICKHOUSE_CLIENT -nm -q " +DROP TABLE IF EXISTS test SYNC; +DROP TABLE IF EXISTS test2 SYNC; +DROP TABLE IF EXISTS test2_replica SYNC; +" From caf4dc7e14e594da3c254822b345b79c57e76d19 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 1 Dec 2023 12:21:47 +0100 Subject: [PATCH 013/325] Fix style check --- src/Common/ErrorCodes.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index b0ed754536d..57aa82f3639 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -587,7 +587,7 @@ M(705, TABLE_NOT_EMPTY) \ M(706, LIBSSH_ERROR) \ M(707, GCP_ERROR) \ - M(708, ILLEGAL_STATISTIC) \ + M(708, ILLEGAL_STATISTIC) \ M(709, BROKEN_PROJECTION) \ \ M(999, KEEPER_EXCEPTION) \ From f609c44eb83fc769ba9e8fc5875bbc10e3e17b9b Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 1 Dec 2023 13:38:28 +0100 Subject: [PATCH 014/325] Update 02916_broken_projection.sh --- tests/queries/0_stateless/02916_broken_projection.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index 60b21216d1a..bd141d1a122 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -459,11 +459,13 @@ function test3() broken_projections_info test ${CLICKHOUSE_CLIENT} -nm --query " + set send_logs_level='fatal'; backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_3') " | grep -o "BACKUP_CREATED" ${CLICKHOUSE_CLIENT} -nm --stacktrace --query " drop table test sync; + set send_logs_level='fatal'; restore table ${CLICKHOUSE_DATABASE}.test from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_3'); " | grep -o "RESTORED" @@ -476,12 +478,14 @@ function test3() broken_projections_info test ${CLICKHOUSE_CLIENT} -nm --query " + set send_logs_level='fatal'; backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_2') settings check_projection_parts=false, allow_backup_broken_projections=true; " | grep -o "BACKUP_CREATED" ${CLICKHOUSE_CLIENT} -nm --stacktrace --query " drop table test sync; + set send_logs_level='fatal'; restore table ${CLICKHOUSE_DATABASE}.test from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_2'); " | grep -o "RESTORED" From 2b903003b4795eb3768fec3f84ec8321fa5485f6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 4 Dec 2023 13:21:18 +0100 Subject: [PATCH 015/325] Update reference --- .../0_stateless/02916_broken_projection.reference | 8 +++----- tests/queries/0_stateless/02916_broken_projection.sh | 9 ++++----- 2 files changed, 7 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02916_broken_projection.reference b/tests/queries/0_stateless/02916_broken_projection.reference index 4c4901ae99f..acd1b87eb30 100644 --- a/tests/queries/0_stateless/02916_broken_projection.reference +++ b/tests/queries/0_stateless/02916_broken_projection.reference @@ -332,16 +332,14 @@ all_1_1_0 1 ['proj','proj_2'] all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] select from projection 'proj', expect error: proj -FILE_DOESNT_EXIST select from projection 'proj_2', expect error: proj 12 16 used projections SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table -0 +1 broken projections info -all_2_2_0 proj NO_FILE_IN_DATA_PART 0 broke all data of part 'proj' (parent part: all_2_2_0) system.parts @@ -358,7 +356,7 @@ SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 0 broken projections info -all_2_2_0 proj NO_FILE_IN_DATA_PART +all_2_2_0 proj FILE_DOESNT_EXIST FILE_DOESNT_EXIST materialize projection proj system.parts @@ -379,7 +377,7 @@ SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 1 broken projections info -all_2_2_0 proj NO_FILE_IN_DATA_PART +all_2_2_0 proj FILE_DOESNT_EXIST BACKUP_CREATED RESTORED system.parts diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index bd141d1a122..7315cf5ce61 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -29,7 +29,7 @@ function create_table() SELECT d ORDER BY c ) ) - ENGINE = ReplicatedMergeTree('/test_broken_projection_24_$test_id/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/', '$replica') ORDER BY a + ENGINE = ReplicatedMergeTree('/test_broken_projection_$test_id/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/', '$replica') ORDER BY a SETTINGS min_bytes_for_wide_part = 0, max_parts_to_merge_at_once=3, enable_vertical_merge_algorithm=1, @@ -407,7 +407,6 @@ function test2() check test2 } -CLICKHOUSE_TEST_UNIQUE_NAME="gghhhhhhhhhhhhhhhhhhh" function test3() { create_table test3 test 1 @@ -437,7 +436,7 @@ function test3() restore table ${CLICKHOUSE_DATABASE}.test from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); " | grep -o "RESTORED" - check test proj FILE_DOESNT_EXIST + check test proj broken_projections_info test @@ -479,14 +478,14 @@ function test3() ${CLICKHOUSE_CLIENT} -nm --query " set send_logs_level='fatal'; - backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_2') + backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_4') settings check_projection_parts=false, allow_backup_broken_projections=true; " | grep -o "BACKUP_CREATED" ${CLICKHOUSE_CLIENT} -nm --stacktrace --query " drop table test sync; set send_logs_level='fatal'; - restore table ${CLICKHOUSE_DATABASE}.test from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_2'); + restore table ${CLICKHOUSE_DATABASE}.test from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_4'); " | grep -o "RESTORED" check test From b77a6073aea98c7c5f5fcc28492a34e801d11b6b Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 4 Dec 2023 16:57:09 +0100 Subject: [PATCH 016/325] Fix test --- .../02916_broken_projection.reference | 90 +++++++++++-------- .../0_stateless/02916_broken_projection.sh | 36 ++++---- 2 files changed, 70 insertions(+), 56 deletions(-) diff --git a/tests/queries/0_stateless/02916_broken_projection.reference b/tests/queries/0_stateless/02916_broken_projection.reference index acd1b87eb30..b7764a6434e 100644 --- a/tests/queries/0_stateless/02916_broken_projection.reference +++ b/tests/queries/0_stateless/02916_broken_projection.reference @@ -7,12 +7,12 @@ all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] -select from projection 'proj', expect error: +select from projection 'proj' 12 16 used projections SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2', expect error: +select from projection 'proj_2' 12 16 used projections @@ -26,12 +26,12 @@ all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] -select from projection 'proj', expect error: +select from projection 'proj' 12 16 used projections SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2', expect error: +select from projection 'proj_2' 12 16 used projections @@ -51,7 +51,7 @@ all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] -select from projection 'proj', expect error: proj_2 +select from projection 'proj' 12 16 used projections @@ -68,12 +68,12 @@ all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] -select from projection 'proj', expect error: +select from projection 'proj' 12 16 used projections SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2', expect error: +select from projection 'proj_2' 12 16 used projections @@ -102,12 +102,12 @@ all_3_3_0 0 ['proj','proj_2'] all_3_5_1 1 ['proj'] all_4_4_0 0 ['proj','proj_2'] all_5_5_0 0 ['proj','proj_2'] -select from projection 'proj', expect error: +select from projection 'proj' 12 16 used projections SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2', expect error: +select from projection 'proj_2' 12 16 used projections @@ -139,12 +139,12 @@ all_3_3_0 0 ['proj','proj_2'] all_3_5_1 1 ['proj'] all_4_4_0 0 ['proj','proj_2'] all_5_5_0 0 ['proj','proj_2'] -select from projection 'proj', expect error: +select from projection 'proj' 12 16 used projections SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2', expect error: +select from projection 'proj_2' 12 16 used projections @@ -167,12 +167,12 @@ all_3_5_1 0 ['proj'] all_3_5_1_6 1 ['proj'] all_4_4_0 0 ['proj','proj_2'] all_5_5_0 0 ['proj','proj_2'] -select from projection 'proj', expect error: +select from projection 'proj' 12 16 used projections SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2', expect error: +select from projection 'proj_2' 12 16 used projections @@ -208,12 +208,12 @@ all_4_4_0 0 ['proj','proj_2'] all_5_5_0 0 ['proj','proj_2'] all_8_8_0 0 ['proj','proj_2'] all_9_9_0 1 ['proj','proj_2'] -select from projection 'proj', expect error: +select from projection 'proj' 12 16 used projections SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj_2 -select from projection 'proj_2', expect error: +select from projection 'proj_2' 12 16 used projections @@ -227,20 +227,19 @@ insert new part system.parts all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] -select from projection 'proj', expect error: +select from projection 'proj' used projections SELECT c FROM test2 WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2', expect error: +select from projection 'proj_2' used projections SELECT d FROM test2 WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 1 system.parts -all_1_1_0 1 ['proj','proj_2'] -select from projection 'proj', expect error: +select from projection 'proj' used projections SELECT c FROM test2_replica WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2', expect error: +select from projection 'proj_2' used projections SELECT d FROM test2_replica WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table @@ -252,10 +251,10 @@ all_0_0_0 system.parts all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] -select from projection 'proj', expect error: +select from projection 'proj' used projections SELECT c FROM test2 WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2', expect error: +select from projection 'proj_2' used projections SELECT d FROM test2 WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table @@ -266,10 +265,10 @@ all_0_0_0 system.parts all_0_0_0 0 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] -select from projection 'proj', expect error: +select from projection 'proj' used projections SELECT c FROM test2 WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2', expect error: +select from projection 'proj_2' used projections SELECT d FROM test2 WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table @@ -277,10 +276,10 @@ check table system.parts all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] -select from projection 'proj', expect error: +select from projection 'proj' used projections SELECT c FROM test2 WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2', expect error: +select from projection 'proj_2' used projections SELECT d FROM test2 WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table @@ -294,12 +293,12 @@ all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] -select from projection 'proj', expect error: +select from projection 'proj' 12 16 used projections SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2', expect error: +select from projection 'proj_2' 12 16 used projections @@ -315,7 +314,7 @@ all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] select from projection 'proj', expect error: proj FILE_DOESNT_EXIST -select from projection 'proj_2', expect error: proj +select from projection 'proj_2' 12 16 used projections @@ -331,8 +330,12 @@ all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] -select from projection 'proj', expect error: proj -select from projection 'proj_2', expect error: proj +select from projection 'proj' +12 +16 +used projections +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj +select from projection 'proj_2' 12 16 used projections @@ -348,7 +351,8 @@ all_1_1_0 1 ['proj','proj_2'] all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] select from projection 'proj', expect error: proj -select from projection 'proj_2', expect error: proj +STD_EXCEPTION +select from projection 'proj_2' 12 16 used projections @@ -368,8 +372,12 @@ all_2_2_0 0 ['proj','proj_2'] all_2_2_0_4 1 ['proj','proj_2'] all_3_3_0 0 ['proj','proj_2'] all_3_3_0_4 1 ['proj','proj_2'] -select from projection 'proj', expect error: proj -select from projection 'proj_2', expect error: proj +select from projection 'proj' +12 +16 +used projections +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj +select from projection 'proj_2' 12 16 used projections @@ -385,8 +393,12 @@ all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] -select from projection 'proj', expect error: proj -select from projection 'proj_2', expect error: proj +select from projection 'proj' +12 +16 +used projections +SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj +select from projection 'proj_2' 12 16 used projections @@ -401,7 +413,7 @@ all_1_1_0 1 ['proj','proj_2'] all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] select from projection 'proj', expect error: proj -select from projection 'proj_2', expect error: proj +select from projection 'proj_2' 12 16 used projections @@ -417,12 +429,12 @@ all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] all_2_2_0 1 ['proj_2'] all_3_3_0 1 ['proj','proj_2'] -select from projection 'proj', expect error: +select from projection 'proj' 12 16 used projections SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2', expect error: +select from projection 'proj_2' 12 16 used projections diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index 7315cf5ce61..eeea512f14a 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -29,7 +29,7 @@ function create_table() SELECT d ORDER BY c ) ) - ENGINE = ReplicatedMergeTree('/test_broken_projection_$test_id/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/', '$replica') ORDER BY a + ENGINE = ReplicatedMergeTree('/test_broken_projection_32_$test_id/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/', '$replica') ORDER BY a SETTINGS min_bytes_for_wide_part = 0, max_parts_to_merge_at_once=3, enable_vertical_merge_algorithm=1, @@ -155,39 +155,41 @@ function check() WHERE table='$table' AND database=currentDatabase() ORDER BY name;" - echo "select from projection 'proj', expect error: $expect_broken_part" query_id=$(random 8) if [ "$expect_broken_part" = "proj" ] then + echo "select from projection 'proj', expect error: $expect_broken_part" $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -nm -q " -SET send_logs_level='fatal'; -SELECT c FROM $table WHERE d == 12 ORDER BY c; -" 2>&1 | grep -oF "$expected_error" + SET send_logs_level='fatal'; + SELECT c FROM $table WHERE d == 12 ORDER BY c; + " 2>&1 | grep -oF "$expected_error" else + echo "select from projection 'proj'" $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT c FROM $table WHERE d == 12 OR d == 16 ORDER BY c;" echo 'used projections' $CLICKHOUSE_CLIENT -nm -q " - SYSTEM FLUSH LOGS; - SELECT query, splitByChar('.', arrayJoin(projections))[-1] FROM system.query_log WHERE current_database=currentDatabase() AND query_id='$query_id' AND type='QueryFinish' + SYSTEM FLUSH LOGS; + SELECT query, splitByChar('.', arrayJoin(projections))[-1] FROM system.query_log WHERE current_database=currentDatabase() AND query_id='$query_id' AND type='QueryFinish' " fi - echo "select from projection 'proj_2', expect error: $expect_broken_part" query_id=$(random 8) if [ "$expect_broken_part" = "proj_2" ] then + echo "select from projection 'proj_2', expect error: $expect_broken_part" $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -nm -q " -SET send_logs_level='fatal'; -SELECT d FROM $table WHERE c == 12 ORDER BY d; -" 2>&1 | grep -oF "$expected_error" + SET send_logs_level='fatal'; + SELECT d FROM $table WHERE c == 12 ORDER BY d; + " 2>&1 | grep -oF "$expected_error" else + echo "select from projection 'proj_2'" $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT d FROM $table WHERE c == 12 OR c == 16 ORDER BY d;" echo 'used projections' $CLICKHOUSE_CLIENT -nm -q " - SYSTEM FLUSH LOGS; - SELECT query, splitByChar('.', arrayJoin(projections))[-1] FROM system.query_log WHERE current_database=currentDatabase() AND query_id='$query_id' AND type='QueryFinish' + SYSTEM FLUSH LOGS; + SELECT query, splitByChar('.', arrayJoin(projections))[-1] FROM system.query_log WHERE current_database=currentDatabase() AND query_id='$query_id' AND type='QueryFinish' " fi @@ -436,13 +438,13 @@ function test3() restore table ${CLICKHOUSE_DATABASE}.test from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); " | grep -o "RESTORED" - check test proj + check test broken_projections_info test break_projection test proj all_2_2_0 part - check test proj + check test proj STD_EXCEPTION broken_projections_info test @@ -453,7 +455,7 @@ function test3() materialize_projection test proj - check test proj + check test broken_projections_info test @@ -468,7 +470,7 @@ function test3() restore table ${CLICKHOUSE_DATABASE}.test from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_3'); " | grep -o "RESTORED" - check test proj + check test break_projection test proj all_2_2_0 part From 4de048904a3cbb6ff30e20b5a8defd1564f2e722 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 4 Dec 2023 19:14:06 +0100 Subject: [PATCH 017/325] Update 02916_broken_projection.sh --- tests/queries/0_stateless/02916_broken_projection.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index eeea512f14a..261342da103 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-random-merge-tree-settings # shellcheck disable=SC2046 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From d3b80ac60cdb1fa17fb8907a7a6f11afde759bab Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 4 Dec 2023 19:14:55 +0100 Subject: [PATCH 018/325] Update 02916_broken_projection.sh --- tests/queries/0_stateless/02916_broken_projection.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index 261342da103..0910ba177fb 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-random-merge-tree-settings +# Tags: long, no-random-merge-tree-settings # shellcheck disable=SC2046 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From 23bde28ac4fc18e296daf6b04283ab50ee58d025 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 8 Dec 2023 19:11:47 +0100 Subject: [PATCH 019/325] Fxi --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 6 ++---- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MutateTask.cpp | 6 ++++-- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 52310802c9d..5418bcd83f3 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -677,7 +677,7 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks if (!parent_part) { loadTTLInfos(); - has_broken_projections = !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 && !has_broken_projections) @@ -742,10 +742,9 @@ void IMergeTreeDataPart::addProjectionPart( projection_parts[projection_name] = std::move(projection_part); } -bool 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(); - bool has_broken_projection = false; for (const auto & projection : metadata_snapshot->projections) { auto path = projection.name + ".proj"; @@ -782,7 +781,6 @@ bool IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool ch } } } - return has_broken_projection; } void IMergeTreeDataPart::loadIndexGranularity() diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 050bd76121c..9812529086b 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -434,7 +434,7 @@ public: bool hasBrokenProjection(const String & projection_name) const; /// Return true, if all projections were loaded successfully and none was marked as broken. - bool loadProjections(bool require_columns_checksums, bool check_consistency, bool if_not_loaded = false); + 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; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 93b241deae7..2b0cf60a7f1 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -870,7 +870,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. @@ -1570,8 +1571,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(); From c8c4db5984bf9101478e0d1f33c3432c257ea7a0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Dec 2023 13:24:31 +0100 Subject: [PATCH 020/325] Fxi test --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 7 +++++++ .../queries/0_stateless/02916_broken_projection.reference | 7 ++++--- tests/queries/0_stateless/02916_broken_projection.sh | 4 +++- 3 files changed, 14 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 5418bcd83f3..7af49edf788 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -780,6 +780,13 @@ void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool ch 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; + } } } diff --git a/tests/queries/0_stateless/02916_broken_projection.reference b/tests/queries/0_stateless/02916_broken_projection.reference index b7764a6434e..358304de74a 100644 --- a/tests/queries/0_stateless/02916_broken_projection.reference +++ b/tests/queries/0_stateless/02916_broken_projection.reference @@ -406,7 +406,7 @@ SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 1 0 -broke all data of part 'proj' (parent part: all_2_2_0) +broke all data of part 'proj' (parent part: all_1_1_0) system.parts all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] @@ -421,13 +421,13 @@ SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 0 broken projections info -all_2_2_0 proj FILE_DOESNT_EXIST +all_1_1_0 proj FILE_DOESNT_EXIST BACKUP_CREATED RESTORED system.parts all_0_0_0 1 ['proj','proj_2'] all_1_1_0 1 ['proj','proj_2'] -all_2_2_0 1 ['proj_2'] +all_2_2_0 1 ['proj','proj_2'] all_3_3_0 1 ['proj','proj_2'] select from projection 'proj' 12 @@ -442,3 +442,4 @@ SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 0 broken projections info +all_1_1_0 proj NO_FILE_IN_DATA_PART diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index 0910ba177fb..eb68f8621a2 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -410,6 +410,8 @@ function test2() check test2 } +CLICKHOUSE_DATABASE="default" +CLICKHOUSE_TEST_UNIQUE_NAME="test123456" function test3() { create_table test3 test 1 @@ -473,7 +475,7 @@ function test3() check test - break_projection test proj all_2_2_0 part + break_projection test proj all_1_1_0 part check test proj FILE_DOESNT_EXIST From cd41802d7e5b056e0114c8ad7523f00828ad5940 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 11 Dec 2023 17:37:44 +0100 Subject: [PATCH 021/325] Update 02916_broken_projection.sh --- tests/queries/0_stateless/02916_broken_projection.sh | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index eb68f8621a2..a52570f3d52 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -410,8 +410,6 @@ function test2() check test2 } -CLICKHOUSE_DATABASE="default" -CLICKHOUSE_TEST_UNIQUE_NAME="test123456" function test3() { create_table test3 test 1 From 457032d2998a085fb9c10c0b9d536e79dbcc5dab Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Dec 2023 20:40:25 +0100 Subject: [PATCH 022/325] Disable fault injection because it breaks .reference --- tests/queries/0_stateless/02916_broken_projection.sh | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index a52570f3d52..2049610e45b 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -52,7 +52,7 @@ function insert() offset=$2 size=$3 echo 'insert new part' - $CLICKHOUSE_CLIENT -q "INSERT INTO $table SELECT number, number, number, number, number%2 FROM numbers($offset, $size);" + $CLICKHOUSE_CLIENT -q "INSERT INTO $table SELECT number, number, number, number, number%2 FROM numbers($offset, $size) SETTINGS insert_keeper_fault_injection_probability;" } function break_projection() @@ -431,11 +431,12 @@ function test3() broken_projections_info test ${CLICKHOUSE_CLIENT} -nm --query " - backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}') settings check_projection_parts=false; + backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}') settings check_projection_parts=false, backup_restore_keeper_fault_injection_probability=0.0; " | grep -o "BACKUP_CREATED" ${CLICKHOUSE_CLIENT} -nm --stacktrace --query " drop table test sync; + set backup_restore_keeper_fault_injection_probability=0.0; restore table ${CLICKHOUSE_DATABASE}.test from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); " | grep -o "RESTORED" @@ -451,6 +452,7 @@ function test3() ${CLICKHOUSE_CLIENT} -nm --query " set send_logs_level='fatal'; + set backup_restore_keeper_fault_injection_probability=0.0; backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_2') " 2>&1 | grep -o "FILE_DOESNT_EXIST" @@ -462,12 +464,14 @@ function test3() ${CLICKHOUSE_CLIENT} -nm --query " set send_logs_level='fatal'; + set backup_restore_keeper_fault_injection_probability=0.0; backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_3') " | grep -o "BACKUP_CREATED" ${CLICKHOUSE_CLIENT} -nm --stacktrace --query " drop table test sync; set send_logs_level='fatal'; + set backup_restore_keeper_fault_injection_probability=0.0; restore table ${CLICKHOUSE_DATABASE}.test from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_3'); " | grep -o "RESTORED" @@ -481,6 +485,7 @@ function test3() ${CLICKHOUSE_CLIENT} -nm --query " set send_logs_level='fatal'; + set backup_restore_keeper_fault_injection_probability=0.0; backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_4') settings check_projection_parts=false, allow_backup_broken_projections=true; " | grep -o "BACKUP_CREATED" @@ -488,6 +493,7 @@ function test3() ${CLICKHOUSE_CLIENT} -nm --stacktrace --query " drop table test sync; set send_logs_level='fatal'; + set backup_restore_keeper_fault_injection_probability=0.0; restore table ${CLICKHOUSE_DATABASE}.test from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_4'); " | grep -o "RESTORED" From 8ef2638cfce90031213bbbd595a50d584406a916 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 12 Dec 2023 14:22:14 +0100 Subject: [PATCH 023/325] Update 02916_broken_projection.sh --- tests/queries/0_stateless/02916_broken_projection.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index 2049610e45b..0418759eb26 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -52,7 +52,7 @@ function insert() offset=$2 size=$3 echo 'insert new part' - $CLICKHOUSE_CLIENT -q "INSERT INTO $table SELECT number, number, number, number, number%2 FROM numbers($offset, $size) SETTINGS insert_keeper_fault_injection_probability;" + $CLICKHOUSE_CLIENT -q "INSERT INTO $table SELECT number, number, number, number, number%2 FROM numbers($offset, $size) SETTINGS insert_keeper_fault_injection_probability=0.0;" } function break_projection() From d81edb4adf65c8c3724ec27fc83b65d5d1b3ebad Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 13 Dec 2023 12:29:28 +0100 Subject: [PATCH 024/325] Update 02916_broken_projection.sh --- tests/queries/0_stateless/02916_broken_projection.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index 0418759eb26..07495c45214 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -431,7 +431,8 @@ function test3() broken_projections_info test ${CLICKHOUSE_CLIENT} -nm --query " - backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}') settings check_projection_parts=false, backup_restore_keeper_fault_injection_probability=0.0; + set backup_restore_keeper_fault_injection_probability=0.0; + backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}') settings check_projection_parts=false; " | grep -o "BACKUP_CREATED" ${CLICKHOUSE_CLIENT} -nm --stacktrace --query " From 79432255df02f696962858347c2207dbdbf2b69f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 18 Dec 2023 12:45:24 +0100 Subject: [PATCH 025/325] Update 02916_broken_projection.sh --- tests/queries/0_stateless/02916_broken_projection.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index 07495c45214..55e613b8f3a 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-random-merge-tree-settings +# Tags: long, no-random-merge-tree-settings, no-random-settings # shellcheck disable=SC2046 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From 4bb63f0a6f066bca972b5b3754a20f0a56354b8d Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Dec 2023 13:05:15 +0100 Subject: [PATCH 026/325] Update test --- .../02916_broken_projection.reference | 124 ------------------ .../0_stateless/02916_broken_projection.sh | 16 +-- 2 files changed, 8 insertions(+), 132 deletions(-) diff --git a/tests/queries/0_stateless/02916_broken_projection.reference b/tests/queries/0_stateless/02916_broken_projection.reference index 358304de74a..d340326455a 100644 --- a/tests/queries/0_stateless/02916_broken_projection.reference +++ b/tests/queries/0_stateless/02916_broken_projection.reference @@ -2,11 +2,6 @@ insert new part insert new part insert new part insert new part -system.parts -all_0_0_0 1 ['proj','proj_2'] -all_1_1_0 1 ['proj','proj_2'] -all_2_2_0 1 ['proj','proj_2'] -all_3_3_0 1 ['proj','proj_2'] select from projection 'proj' 12 16 @@ -21,11 +16,6 @@ check table 1 0 broke metadata of part 'proj' (parent part: all_2_2_0) -system.parts -all_0_0_0 1 ['proj','proj_2'] -all_1_1_0 1 ['proj','proj_2'] -all_2_2_0 1 ['proj','proj_2'] -all_3_3_0 1 ['proj','proj_2'] select from projection 'proj' 12 16 @@ -46,11 +36,6 @@ all_2_2_0 broke data of part 'proj_2' (parent part: all_2_2_0) broken projections info all_2_2_0 proj FILE_DOESNT_EXIST -system.parts -all_0_0_0 1 ['proj','proj_2'] -all_1_1_0 1 ['proj','proj_2'] -all_2_2_0 1 ['proj','proj_2'] -all_3_3_0 1 ['proj','proj_2'] select from projection 'proj' 12 16 @@ -63,11 +48,6 @@ check table broken projections info all_2_2_0 proj FILE_DOESNT_EXIST all_2_2_0 proj_2 NO_FILE_IN_DATA_PART -system.parts -all_0_0_0 1 ['proj','proj_2'] -all_1_1_0 1 ['proj','proj_2'] -all_2_2_0 1 ['proj','proj_2'] -all_3_3_0 1 ['proj','proj_2'] select from projection 'proj' 12 16 @@ -94,14 +74,6 @@ broken projections info all_2_2_0 proj FILE_DOESNT_EXIST all_2_2_0 proj_2 NO_FILE_IN_DATA_PART all_3_3_0 proj_2 NO_FILE_IN_DATA_PART -system.parts -all_0_0_0 1 ['proj','proj_2'] -all_1_1_0 1 ['proj','proj_2'] -all_2_2_0 1 ['proj','proj_2'] -all_3_3_0 0 ['proj','proj_2'] -all_3_5_1 1 ['proj'] -all_4_4_0 0 ['proj','proj_2'] -all_5_5_0 0 ['proj','proj_2'] select from projection 'proj' 12 16 @@ -131,14 +103,6 @@ all_1_1_0 proj_2 FILE_DOESNT_EXIST all_2_2_0 proj NO_FILE_IN_DATA_PART all_2_2_0 proj_2 FILE_DOESNT_EXIST all_3_3_0 proj_2 FILE_DOESNT_EXIST -system.parts -all_0_0_0 1 ['proj','proj_2'] -all_1_1_0 1 ['proj','proj_2'] -all_2_2_0 1 ['proj','proj_2'] -all_3_3_0 0 ['proj','proj_2'] -all_3_5_1 1 ['proj'] -all_4_4_0 0 ['proj','proj_2'] -all_5_5_0 0 ['proj','proj_2'] select from projection 'proj' 12 16 @@ -155,18 +119,6 @@ check table full (test - all_1_1_0) all_1_1_0 materialize projection proj check table full (test - ) -system.parts -all_0_0_0 0 ['proj','proj_2'] -all_0_0_0_6 1 ['proj','proj_2'] -all_1_1_0 0 ['proj','proj_2'] -all_1_1_0_6 1 ['proj','proj_2'] -all_2_2_0 0 ['proj','proj_2'] -all_2_2_0_6 1 ['proj','proj_2'] -all_3_3_0 0 ['proj','proj_2'] -all_3_5_1 0 ['proj'] -all_3_5_1_6 1 ['proj'] -all_4_4_0 0 ['proj','proj_2'] -all_5_5_0 0 ['proj','proj_2'] select from projection 'proj' 12 16 @@ -189,25 +141,6 @@ OPTIMIZE TABLE test FINAL insert new part optimize OPTIMIZE TABLE test FINAL -system.parts -all_0_0_0 0 ['proj','proj_2'] -all_0_0_0_6 0 ['proj','proj_2'] -all_0_0_0_7 0 ['proj','proj_2'] -all_0_8_2_7 1 ['proj_2'] -all_1_1_0 0 ['proj','proj_2'] -all_1_1_0_6 0 ['proj','proj_2'] -all_1_1_0_7 0 ['proj','proj_2'] -all_2_2_0 0 ['proj','proj_2'] -all_2_2_0_6 0 ['proj','proj_2'] -all_2_2_0_7 0 ['proj','proj_2'] -all_3_3_0 0 ['proj','proj_2'] -all_3_5_1 0 ['proj'] -all_3_5_1_6 0 ['proj'] -all_3_5_1_7 0 ['proj','proj_2'] -all_4_4_0 0 ['proj','proj_2'] -all_5_5_0 0 ['proj','proj_2'] -all_8_8_0 0 ['proj','proj_2'] -all_9_9_0 1 ['proj','proj_2'] select from projection 'proj' 12 16 @@ -224,9 +157,6 @@ insert new part insert new part insert new part insert new part -system.parts -all_0_0_0 1 ['proj','proj_2'] -all_1_1_0 1 ['proj','proj_2'] select from projection 'proj' used projections SELECT c FROM test2 WHERE d == 12 OR d == 16 ORDER BY c; proj @@ -235,7 +165,6 @@ used projections SELECT d FROM test2 WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 1 -system.parts select from projection 'proj' used projections SELECT c FROM test2_replica WHERE d == 12 OR d == 16 ORDER BY c; proj @@ -248,9 +177,6 @@ check table broke data of part 'proj' (parent part: all_0_0_0) check table full (test2 - all_0_0_0) all_0_0_0 -system.parts -all_0_0_0 1 ['proj','proj_2'] -all_1_1_0 1 ['proj','proj_2'] select from projection 'proj' used projections SELECT c FROM test2 WHERE d == 12 OR d == 16 ORDER BY c; proj @@ -262,9 +188,6 @@ check table broke data of part 'all_0_0_0' check table full (test2 - all_0_0_0) all_0_0_0 -system.parts -all_0_0_0 0 ['proj','proj_2'] -all_1_1_0 1 ['proj','proj_2'] select from projection 'proj' used projections SELECT c FROM test2 WHERE d == 12 OR d == 16 ORDER BY c; proj @@ -273,9 +196,6 @@ used projections SELECT d FROM test2 WHERE c == 12 OR c == 16 ORDER BY d; proj_2 check table 1 -system.parts -all_0_0_0 1 ['proj','proj_2'] -all_1_1_0 1 ['proj','proj_2'] select from projection 'proj' used projections SELECT c FROM test2 WHERE d == 12 OR d == 16 ORDER BY c; proj @@ -288,11 +208,6 @@ insert new part insert new part insert new part insert new part -system.parts -all_0_0_0 1 ['proj','proj_2'] -all_1_1_0 1 ['proj','proj_2'] -all_2_2_0 1 ['proj','proj_2'] -all_3_3_0 1 ['proj','proj_2'] select from projection 'proj' 12 16 @@ -307,11 +222,6 @@ check table 1 0 broke data of part 'proj' (parent part: all_2_2_0) -system.parts -all_0_0_0 1 ['proj','proj_2'] -all_1_1_0 1 ['proj','proj_2'] -all_2_2_0 1 ['proj','proj_2'] -all_3_3_0 1 ['proj','proj_2'] select from projection 'proj', expect error: proj FILE_DOESNT_EXIST select from projection 'proj_2' @@ -325,11 +235,6 @@ broken projections info all_2_2_0 proj NO_FILE_IN_DATA_PART BACKUP_CREATED RESTORED -system.parts -all_0_0_0 1 ['proj','proj_2'] -all_1_1_0 1 ['proj','proj_2'] -all_2_2_0 1 ['proj','proj_2'] -all_3_3_0 1 ['proj','proj_2'] select from projection 'proj' 12 16 @@ -345,11 +250,6 @@ check table broken projections info 0 broke all data of part 'proj' (parent part: all_2_2_0) -system.parts -all_0_0_0 1 ['proj','proj_2'] -all_1_1_0 1 ['proj','proj_2'] -all_2_2_0 1 ['proj','proj_2'] -all_3_3_0 1 ['proj','proj_2'] select from projection 'proj', expect error: proj STD_EXCEPTION select from projection 'proj_2' @@ -363,15 +263,6 @@ broken projections info all_2_2_0 proj FILE_DOESNT_EXIST FILE_DOESNT_EXIST materialize projection proj -system.parts -all_0_0_0 0 ['proj','proj_2'] -all_0_0_0_4 1 ['proj','proj_2'] -all_1_1_0 0 ['proj','proj_2'] -all_1_1_0_4 1 ['proj','proj_2'] -all_2_2_0 0 ['proj','proj_2'] -all_2_2_0_4 1 ['proj','proj_2'] -all_3_3_0 0 ['proj','proj_2'] -all_3_3_0_4 1 ['proj','proj_2'] select from projection 'proj' 12 16 @@ -388,11 +279,6 @@ broken projections info all_2_2_0 proj FILE_DOESNT_EXIST BACKUP_CREATED RESTORED -system.parts -all_0_0_0 1 ['proj','proj_2'] -all_1_1_0 1 ['proj','proj_2'] -all_2_2_0 1 ['proj','proj_2'] -all_3_3_0 1 ['proj','proj_2'] select from projection 'proj' 12 16 @@ -407,11 +293,6 @@ check table 1 0 broke all data of part 'proj' (parent part: all_1_1_0) -system.parts -all_0_0_0 1 ['proj','proj_2'] -all_1_1_0 1 ['proj','proj_2'] -all_2_2_0 1 ['proj','proj_2'] -all_3_3_0 1 ['proj','proj_2'] select from projection 'proj', expect error: proj select from projection 'proj_2' 12 @@ -424,11 +305,6 @@ broken projections info all_1_1_0 proj FILE_DOESNT_EXIST BACKUP_CREATED RESTORED -system.parts -all_0_0_0 1 ['proj','proj_2'] -all_1_1_0 1 ['proj','proj_2'] -all_2_2_0 1 ['proj','proj_2'] -all_3_3_0 1 ['proj','proj_2'] select from projection 'proj' 12 16 diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index 55e613b8f3a..a1df5dc858d 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-random-merge-tree-settings, no-random-settings +# Tags: long, no-random-merge-tree-settings, no-random-settings, no-s3-storage # shellcheck disable=SC2046 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) @@ -149,12 +149,12 @@ function check() expected_error=$3 fi - echo 'system.parts' - $CLICKHOUSE_CLIENT -q " - SELECT name, active, projections - FROM system.parts - WHERE table='$table' AND database=currentDatabase() - ORDER BY name;" + #echo 'system.parts' + #$CLICKHOUSE_CLIENT -q " + #SELECT name, active, projections + #FROM system.parts + #WHERE table='$table' AND database=currentDatabase() + #ORDER BY name;" query_id=$(random 8) @@ -447,7 +447,7 @@ function test3() break_projection test proj all_2_2_0 part - check test proj STD_EXCEPTION + check test broken_projections_info test From 3d2e95dbf5f81185d2a091d5e58490f66ed04bef Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Dec 2023 13:49:49 +0100 Subject: [PATCH 027/325] Fix build --- src/Storages/MergeTree/checkDataPart.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index ea46b6f0d56..5b60f0a7fc2 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -332,7 +332,7 @@ static IMergeTreeDataPart::Checksums checkDataPart( if (throw_on_broken_projection && !broken_projections_message.empty()) { - throw Exception(ErrorCodes::BROKEN_PROJECTION, broken_projections_message.data()); + throw Exception(ErrorCodes::BROKEN_PROJECTION, "{}", broken_projections_message); } if (require_checksums && !projections_on_disk.empty()) From 493f938c455e9bd507d521b7974b1e7a9e7c81b2 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 28 Dec 2023 17:29:25 +0100 Subject: [PATCH 028/325] Update 02916_broken_projection.sh --- tests/queries/0_stateless/02916_broken_projection.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index a1df5dc858d..ca62d275189 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -447,7 +447,7 @@ function test3() break_projection test proj all_2_2_0 part - check test + check test proj ErrnoException broken_projections_info test From 91657185c8fc4349cb8825ac2e5d6126fddb8289 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 29 Dec 2023 13:05:15 +0100 Subject: [PATCH 029/325] Fxi --- tests/queries/0_stateless/02916_broken_projection.reference | 2 +- tests/queries/0_stateless/02916_broken_projection.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02916_broken_projection.reference b/tests/queries/0_stateless/02916_broken_projection.reference index d340326455a..beaca49f99c 100644 --- a/tests/queries/0_stateless/02916_broken_projection.reference +++ b/tests/queries/0_stateless/02916_broken_projection.reference @@ -251,7 +251,7 @@ broken projections info 0 broke all data of part 'proj' (parent part: all_2_2_0) select from projection 'proj', expect error: proj -STD_EXCEPTION +Errno select from projection 'proj_2' 12 16 diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index ca62d275189..99e54b08b74 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -447,7 +447,7 @@ function test3() break_projection test proj all_2_2_0 part - check test proj ErrnoException + check test proj Errno broken_projections_info test From e0f0100332085f3075951a6d9bf5c8d69f6d9940 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 29 Dec 2023 15:38:15 +0100 Subject: [PATCH 030/325] Update 02916_broken_projection.reference --- tests/queries/0_stateless/02916_broken_projection.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02916_broken_projection.reference b/tests/queries/0_stateless/02916_broken_projection.reference index beaca49f99c..3967215e5de 100644 --- a/tests/queries/0_stateless/02916_broken_projection.reference +++ b/tests/queries/0_stateless/02916_broken_projection.reference @@ -252,6 +252,7 @@ broken projections info broke all data of part 'proj' (parent part: all_2_2_0) select from projection 'proj', expect error: proj Errno +Errno select from projection 'proj_2' 12 16 From 10af0d406fb536917a84d23f4bacba073ea9443e Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 10 Jan 2024 16:55:58 +0100 Subject: [PATCH 031/325] Update 02916_broken_projection.sh --- tests/queries/0_stateless/02916_broken_projection.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh index 99e54b08b74..fbd26e59f6f 100755 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ b/tests/queries/0_stateless/02916_broken_projection.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-random-merge-tree-settings, no-random-settings, no-s3-storage +# Tags: long, no-random-merge-tree-settings, no-random-settings, no-s3-storage, no-parallel # shellcheck disable=SC2046 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From 1dacfc53ff97fbab6ee349c6df27b3ad2f9df1e8 Mon Sep 17 00:00:00 2001 From: Dale Mcdiarmid Date: Fri, 12 Jan 2024 17:28:45 +0000 Subject: [PATCH 032/325] weather data --- .../getting-started/example-datasets/noaa.md | 340 ++++++++++++++++++ 1 file changed, 340 insertions(+) create mode 100644 docs/en/getting-started/example-datasets/noaa.md diff --git a/docs/en/getting-started/example-datasets/noaa.md b/docs/en/getting-started/example-datasets/noaa.md new file mode 100644 index 00000000000..8d34ff8d3ee --- /dev/null +++ b/docs/en/getting-started/example-datasets/noaa.md @@ -0,0 +1,340 @@ +--- +slug: /en/getting-started/example-datasets/noaa +sidebar_label: NOAA Global Historical Climatology Network +sidebar_position: 1 +description: 2.5 billion rows of climate data for the last 120 yrs +--- + +# NOAA Global Historical Climatology Network + +This dataset contains weather measurements for the last 120 years. Each row is a measurement for a point in time and station. + +More precisely and according to the [origin of this data](https://github.com/awslabs/open-data-docs/tree/main/docs/noaa/noaa-ghcn): + +> GHCN-Daily is a dataset that contains daily observations over global land areas. It contains station-based measurements from land-based stations worldwide, about two thirds of which are for precipitation measurements only (Menne et al., 2012). GHCN-Daily is a composite of climate records from numerous sources that were merged together and subjected to a common suite of quality assurance reviews (Durre et al., 2010). The archive includes the following meteorological elements: + + - Daily maximum temperature + - Daily minimum temperature + - Temperature at the time of observation + - Precipitation (i.e., rain, melted snow) + - Snowfall + - Snow depth + - Other elements where available + +## Downloading the data + +- A [pre-prepared version](#pre-prepared-data) of the data for ClickHouse, which has been cleansed, re-structured, and enriched. This data covers the years 1900 to 2022. +- [Download the original data](#original-data) and convert to the format required by ClickHouse. Users wanting to add their own columns may wish to explore this approach. + +### Pre-prepared data + +More specifically, rows have been removed that did not fail any quality assurance checks by Noaa. The data has also been restructured from a measurement per line to a row per station id and date i.e. + +```csv +"station_id","date","tempAvg","tempMax","tempMin","precipitation","snowfall","snowDepth","percentDailySun","averageWindSpeed","maxWindSpeed","weatherType" +"AEM00041194","2022-07-30",347,0,308,0,0,0,0,0,0,0 +"AEM00041194","2022-07-31",371,413,329,0,0,0,0,0,0,0 +"AEM00041194","2022-08-01",384,427,357,0,0,0,0,0,0,0 +"AEM00041194","2022-08-02",381,424,352,0,0,0,0,0,0,0 +``` + +This is simpler to query and ensures the resulting table is less sparse. Finally, the data has also been enriched with latitude and longitude. + +This data is available in the following S3 location. Either download the data to your local filesystem (and insert using the ClickHouse client) or insert directly into ClickHouse (see [Inserting from S3](#inserting-from-s3)). + +To download: + +```bash +wget https://datasets-documentation.s3.eu-west-3.amazonaws.com/noaa/noaa_enriched.parquet +``` + +### Original data + +The following details the steps to download and transform the original data in preparation for loading into ClickHouse. + +#### Download + +To download the original data: + +```bash +for i in {1900..2023}; do wget https://noaa-ghcn-pds.s3.amazonaws.com/csv.gz/${i}.csv.gz; done +``` + +#### Sampling the data + +```bash +zcat 2021.csv.gz | head +AE000041196,20210101,TMAX,278,,,S, +AE000041196,20210101,PRCP,0,D,,S, +AE000041196,20210101,TAVG,214,H,,S, +AEM00041194,20210101,TMAX,266,,,S, +AEM00041194,20210101,TMIN,178,,,S, +AEM00041194,20210101,PRCP,0,,,S, +AEM00041194,20210101,TAVG,217,H,,S, +AEM00041217,20210101,TMAX,262,,,S, +AEM00041217,20210101,TMIN,155,,,S, +AEM00041217,20210101,TAVG,202,H,,S, +``` + +Summarizing the [format documentation](https://github.com/awslabs/open-data-docs/tree/main/docs/noaa/noaa-ghcn): + + +Summarizing the format documentation and the columns in order: + + - An 11 character station identification code. This itself encodes some useful information + - YEAR/MONTH/DAY = 8 character date in YYYYMMDD format (e.g. 19860529 = May 29, 1986) + - ELEMENT = 4 character indicator of element type. Effectively the measurement type. While there are many measurements available, we select the following: + - PRCP - Precipitation (tenths of mm) + - SNOW - Snowfall (mm) + - SNWD - Snow depth (mm) + - TMAX - Maximum temperature (tenths of degrees C) + - TAVG - Average temperature (tenths of a degrees C) + - TMIN - Minimum temperature (tenths of degrees C) + - PSUN - Daily percent of possible sunshine (percent) + - AWND - Average daily wind speed (tenths of meters per second) + - WSFG - Peak gust wind speed (tenths of meters per second) + - WT** = Weather Type where ** defines the weather type. Full list of weather types here. +- DATA VALUE = 5 character data value for ELEMENT i.e. the value of the measurement. +- M-FLAG = 1 character Measurement Flag. This has 10 possible values. Some of these values indicate questionable data accuracy. We accept data where this is set to “P” - identified as missing presumed zero, as this is only relevant to the PRCP, SNOW and SNWD measurements. +- Q-FLAG is the measurement quality flag with 14 possible values. We are only interested in data with an empty value i.e. it did not fail any quality assurance checks. +- S-FLAG is the source flag for the observation. Not useful for our analysis and ignored. +- OBS-TIME = 4-character time of observation in hour-minute format (i.e. 0700 =7:00 am). Typically not present in older data. We ignore this for our purposes. + +A measurement per line would result in a sparse table structure in ClickHouse. We should transform to a row per time and station, with measurements as columns. First, we limit the dataset to those rows without issues i.e. where `qFlag` is equal to an empty string. + +#### Clean the data + +Using [ClickHouse local](https://clickhouse.com/blog/extracting-converting-querying-local-files-with-sql-clickhouse-local) we can filter rows that represent measurements of interest and pass our quality requirements: + +```bash +clickhouse local --query "SELECT count() +FROM file('*.csv.gz', CSV, 'station_id String, date String, measurement String, value Int64, mFlag String, qFlag String, sFlag String, obsTime String') WHERE qFlag = '' AND (measurement IN ('PRCP', 'SNOW', 'SNWD', 'TMAX', 'TAVG', 'TMIN', 'PSUN', 'AWND', 'WSFG') OR startsWith(measurement, 'WT'))" + +2679264563 +``` + +With over 2.6 billion rows, this isn’t a fast query since it involves parsing all the files. On our 8 core machine, this takes around 160 seconds. + + +### Pivot data + +While the measurement per line structure can be used with ClickHouse, it will unnecessarily complicate future queries. Ideally, we need a row per station id and date, where each measurement type and associated value are a column i.e. + +```csv +"station_id","date","tempAvg","tempMax","tempMin","precipitation","snowfall","snowDepth","percentDailySun","averageWindSpeed","maxWindSpeed","weatherType" +"AEM00041194","2022-07-30",347,0,308,0,0,0,0,0,0,0 +"AEM00041194","2022-07-31",371,413,329,0,0,0,0,0,0,0 +"AEM00041194","2022-08-01",384,427,357,0,0,0,0,0,0,0 +"AEM00041194","2022-08-02",381,424,352,0,0,0,0,0,0,0 +``` + +Using ClickHouse local and a simple `GROUP BY`, we can repivot our data to this structure. To limit memory overhead, we do this one file at a time. + +```bash +for i in {1900..2022} +do +clickhouse-local --query "SELECT station_id, + toDate32(date) as date, + anyIf(value, measurement = 'TAVG') as tempAvg, + anyIf(value, measurement = 'TMAX') as tempMax, + anyIf(value, measurement = 'TMIN') as tempMin, + anyIf(value, measurement = 'PRCP') as precipitation, + anyIf(value, measurement = 'SNOW') as snowfall, + anyIf(value, measurement = 'SNWD') as snowDepth, + anyIf(value, measurement = 'PSUN') as percentDailySun, + anyIf(value, measurement = 'AWND') as averageWindSpeed, + anyIf(value, measurement = 'WSFG') as maxWindSpeed, + toUInt8OrZero(replaceOne(anyIf(measurement, startsWith(measurement, 'WT') AND value = 1), 'WT', '')) as weatherType +FROM file('$i.csv.gz', CSV, 'station_id String, date String, measurement String, value Int64, mFlag String, qFlag String, sFlag String, obsTime String') + WHERE qFlag = '' AND (measurement IN ('PRCP', 'SNOW', 'SNWD', 'TMAX', 'TAVG', 'TMIN', 'PSUN', 'AWND', 'WSFG') OR startsWith(measurement, 'WT')) +GROUP BY station_id, date +ORDER BY station_id, date FORMAT CSV" >> "noaa.csv"; +done +``` + +This query produces a single 50GB file `noaa.csv`. + +### Enriching the data + +The data has no indication of location aside from a station id, which includes a prefix country code. Ideally, each station would have a latitude and longitude associated with it. To achieve this, NOAA conveniently provides the details of each station as a separate [ghcnd-stations.txt](https://github.com/awslabs/open-data-docs/tree/main/docs/noaa/noaa-ghcn#format-of-ghcnd-stationstxt-file). This file has [several columns](https://github.com/awslabs/open-data-docs/tree/main/docs/noaa/noaa-ghcn#format-of-ghcnd-stationstxt-file), of which five are useful to our future analysis: id, latitude, longitude, elevation, and name. + +```bash +wget http://noaa-ghcn-pds.s3.amazonaws.com/ghcnd-stations.txt +``` + +```bash +clickhouse local --query "WITH stations AS (SELECT id, lat, lon, elevation, splitByString(' GSN ',name)[1] as name FROM file('ghcnd-stations.txt', Regexp, 'id String, lat Float64, lon Float64, elevation Float32, name String')) +SELECT station_id, + date, + tempAvg, + tempMax, + tempMin, + precipitation, + snowfall, + snowDepth, + percentDailySun, + averageWindSpeed, + maxWindSpeed, + weatherType, + tuple(lon, lat) as location, + elevation, + name +FROM file('noaa.csv', CSV, + 'station_id String, date Date32, tempAvg Int32, tempMax Int32, tempMin Int32, precipitation Int32, snowfall Int32, snowDepth Int32, percentDailySun Int8, averageWindSpeed Int32, maxWindSpeed Int32, weatherType UInt8') as noaa LEFT OUTER + JOIN stations ON noaa.station_id = stations.id INTO OUTFILE 'noaa_enriched.parquet' FORMAT Parquet SETTINGS format_regexp='^(.{11})\s+(\-?\d{1,2}\.\d{4})\s+(\-?\d{1,3}\.\d{1,4})\s+(\-?\d*\.\d*)\s+(.*)\s+(?:[\d]*)'" +``` +This query takes a few minutes to run and produces an 6.4GB file `noaa_enriched.parquet`. + +## Create table + +Create a MergeTree table in ClickHouse (from the ClickHouse client). + +```sql +CREATE TABLE noaa +( + `station_id` LowCardinality(String), + `date` Date32, + `tempAvg` Int32 COMMENT 'Average temperature (tenths of a degrees C)', + `tempMax` Int32 COMMENT 'Maximum temperature (tenths of degrees C)', + `tempMin` Int32 COMMENT 'Minimum temperature (tenths of degrees C)', + `precipitation` UInt32 COMMENT 'Precipitation (tenths of mm)', + `snowfall` UInt32 COMMENT 'Snowfall (mm)', + `snowDepth` UInt32 COMMENT 'Snow depth (mm)', + `percentDailySun` UInt8 COMMENT 'Daily percent of possible sunshine (percent)', + `averageWindSpeed` UInt32 COMMENT 'Average daily wind speed (tenths of meters per second)', + `maxWindSpeed` UInt32 COMMENT 'Peak gust wind speed (tenths of meters per second)', + `weatherType` Enum8('Normal' = 0, 'Fog' = 1, 'Heavy Fog' = 2, 'Thunder' = 3, 'Small Hail' = 4, 'Hail' = 5, 'Glaze' = 6, 'Dust/Ash' = 7, 'Smoke/Haze' = 8, 'Blowing/Drifting Snow' = 9, 'Tornado' = 10, 'High Winds' = 11, 'Blowing Spray' = 12, 'Mist' = 13, 'Drizzle' = 14, 'Freezing Drizzle' = 15, 'Rain' = 16, 'Freezing Rain' = 17, 'Snow' = 18, 'Unknown Precipitation' = 19, 'Ground Fog' = 21, 'Freezing Fog' = 22), + `location` Point, + `elevation` Float32, + `name` LowCardinality(String) +) ENGINE = MergeTree() ORDER BY (station_id, date); + +``` + +## Inserting into ClickHouse + +### Inserting from local file + +Data can be inserted from local file as follows (from the ClickHouse client): + +```sql +INSERT INTO noaa FROM INFILE '/noaa_enriched.parquet' +``` + +where `` represents the full path to the local file on disk. + +See [here](https://clickhouse.com/blog/real-world-data-noaa-climate-data#load-the-data) for how to speed this load up. + +### Inserting from S3 + +```sql +INSERT INTO noaa SELECT * +FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/noaa/noaa_enriched.parquet') + +``` +For how to speed this up, see our blog post on [tuning large data loads](https://clickhouse.com/blog/supercharge-your-clickhouse-data-loads-part2). + +## Sample queries + +### Highest temperature ever + +```sql +SELECT + tempMax / 10 AS maxTemp, + location, + name, + date +FROM blogs.noaa +WHERE tempMax > 500 +ORDER BY + tempMax DESC, + date ASC +LIMIT 5 + +┌─maxTemp─┬─location──────────┬─name───────────────────────────────────────────┬───────date─┐ +│ 56.7 │ (-116.8667,36.45) │ CA GREENLAND RCH │ 1913-07-10 │ +│ 56.7 │ (-115.4667,32.55) │ MEXICALI (SMN) │ 1949-08-20 │ +│ 56.7 │ (-115.4667,32.55) │ MEXICALI (SMN) │ 1949-09-18 │ +│ 56.7 │ (-115.4667,32.55) │ MEXICALI (SMN) │ 1952-07-17 │ +│ 56.7 │ (-115.4667,32.55) │ MEXICALI (SMN) │ 1952-09-04 │ +└─────────┴───────────────────┴────────────────────────────────────────────────┴────────────┘ + +5 rows in set. Elapsed: 0.514 sec. Processed 1.06 billion rows, 4.27 GB (2.06 billion rows/s., 8.29 GB/s.) +``` + +Reassuringly consistent with the [documented record](https://en.wikipedia.org/wiki/List_of_weather_records#Highest_temperatures_ever_recorded) at [Furnace Creek](https://www.google.com/maps/place/36%C2%B027'00.0%22N+116%C2%B052'00.1%22W/@36.1329666,-116.1104099,8.95z/data=!4m5!3m4!1s0x0:0xf2ed901b860f4446!8m2!3d36.45!4d-116.8667) as of 2023. + +### Best ski resorts + +Using a [list of ski resorts](https://gist.githubusercontent.com/gingerwizard/dd022f754fd128fdaf270e58fa052e35/raw/622e03c37460f17ef72907afe554cb1c07f91f23/ski_resort_stats.csv) in the united states and their respective locations, we join these against the top 1000 weather stations with the most in any month in the last 5 yrs. Sorting this join by [geoDistance](https://clickhouse.com/docs/en/sql-reference/functions/geo/coordinates/#geodistance) and restricting the results to those where the distance is less than 20km, we select the top result per resort and sort this by total snow. Note we also restrict resorts to those above 1800m, as a broad indicator of good skiing conditions. + +```sql +SELECT + resort_name, + total_snow / 1000 AS total_snow_m, + resort_location, + month_year +FROM +( + WITH resorts AS + ( + SELECT + resort_name, + state, + (lon, lat) AS resort_location, + 'US' AS code + FROM url('https://gist.githubusercontent.com/gingerwizard/dd022f754fd128fdaf270e58fa052e35/raw/622e03c37460f17ef72907afe554cb1c07f91f23/ski_resort_stats.csv', CSVWithNames) + ) + SELECT + resort_name, + highest_snow.station_id, + geoDistance(resort_location.1, resort_location.2, station_location.1, station_location.2) / 1000 AS distance_km, + highest_snow.total_snow, + resort_location, + station_location, + month_year + FROM + ( + SELECT + sum(snowfall) AS total_snow, + station_id, + any(location) AS station_location, + month_year, + substring(station_id, 1, 2) AS code + FROM noaa + WHERE (date > '2017-01-01') AND (code = 'US') AND (elevation > 1800) + GROUP BY + station_id, + toYYYYMM(date) AS month_year + ORDER BY total_snow DESC + LIMIT 1000 + ) AS highest_snow + INNER JOIN resorts ON highest_snow.code = resorts.code + WHERE distance_km < 20 + ORDER BY + resort_name ASC, + total_snow DESC + LIMIT 1 BY + resort_name, + station_id +) +ORDER BY total_snow DESC +LIMIT 5 + +┌─resort_name──────────┬─total_snow_m─┬─resort_location─┬─month_year─┐ +│ Sugar Bowl, CA │ 7.799 │ (-120.3,39.27) │ 201902 │ +│ Donner Ski Ranch, CA │ 7.799 │ (-120.34,39.31) │ 201902 │ +│ Boreal, CA │ 7.799 │ (-120.35,39.33) │ 201902 │ +│ Homewood, CA │ 4.926 │ (-120.17,39.08) │ 201902 │ +│ Alpine Meadows, CA │ 4.926 │ (-120.22,39.17) │ 201902 │ +└──────────────────────┴──────────────┴─────────────────┴────────────┘ + +5 rows in set. Elapsed: 0.750 sec. Processed 689.10 million rows, 3.20 GB (918.20 million rows/s., 4.26 GB/s.) +Peak memory usage: 67.66 MiB. +``` + +## Credits + +We would like to acknowledge the efforts of the Global Historical Climatology Network for preparing, cleansing, and distributing this data. We appreciate your efforts. + +Menne, M.J., I. Durre, B. Korzeniewski, S. McNeal, K. Thomas, X. Yin, S. Anthony, R. Ray, R.S. Vose, B.E.Gleason, and T.G. Houston, 2012: Global Historical Climatology Network - Daily (GHCN-Daily), Version 3. [indicate subset used following decimal, e.g. Version 3.25]. NOAA National Centers for Environmental Information. http://doi.org/10.7289/V5D21VHZ [17/08/2020] From 5ba6def57d0e256be75b729678fc37d4c8989f7e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jan 2024 07:29:28 +0300 Subject: [PATCH 033/325] Update noaa.md --- .../getting-started/example-datasets/noaa.md | 32 ++++++++++--------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/docs/en/getting-started/example-datasets/noaa.md b/docs/en/getting-started/example-datasets/noaa.md index 8d34ff8d3ee..bc2e9fecae1 100644 --- a/docs/en/getting-started/example-datasets/noaa.md +++ b/docs/en/getting-started/example-datasets/noaa.md @@ -11,7 +11,7 @@ This dataset contains weather measurements for the last 120 years. Each row is a More precisely and according to the [origin of this data](https://github.com/awslabs/open-data-docs/tree/main/docs/noaa/noaa-ghcn): -> GHCN-Daily is a dataset that contains daily observations over global land areas. It contains station-based measurements from land-based stations worldwide, about two thirds of which are for precipitation measurements only (Menne et al., 2012). GHCN-Daily is a composite of climate records from numerous sources that were merged together and subjected to a common suite of quality assurance reviews (Durre et al., 2010). The archive includes the following meteorological elements: +> GHCN-Daily is a dataset that contains daily observations over global land areas. It contains station-based measurements from land-based stations worldwide, about two-thirds of which are for precipitation measurements only (Menne et al., 2012). GHCN-Daily is a composite of climate records from numerous sources that were merged together and subjected to a common suite of quality assurance reviews (Durre et al., 2010). The archive includes the following meteorological elements: - Daily maximum temperature - Daily minimum temperature @@ -28,7 +28,7 @@ More precisely and according to the [origin of this data](https://github.com/aws ### Pre-prepared data -More specifically, rows have been removed that did not fail any quality assurance checks by Noaa. The data has also been restructured from a measurement per line to a row per station id and date i.e. +More specifically, rows have been removed that did not fail any quality assurance checks by Noaa. The data has also been restructured from a measurement per line to a row per station id and date, i.e. ```csv "station_id","date","tempAvg","tempMax","tempMin","precipitation","snowfall","snowDepth","percentDailySun","averageWindSpeed","maxWindSpeed","weatherType" @@ -63,17 +63,19 @@ for i in {1900..2023}; do wget https://noaa-ghcn-pds.s3.amazonaws.com/csv.gz/${i #### Sampling the data ```bash -zcat 2021.csv.gz | head -AE000041196,20210101,TMAX,278,,,S, -AE000041196,20210101,PRCP,0,D,,S, -AE000041196,20210101,TAVG,214,H,,S, -AEM00041194,20210101,TMAX,266,,,S, -AEM00041194,20210101,TMIN,178,,,S, -AEM00041194,20210101,PRCP,0,,,S, -AEM00041194,20210101,TAVG,217,H,,S, -AEM00041217,20210101,TMAX,262,,,S, -AEM00041217,20210101,TMIN,155,,,S, -AEM00041217,20210101,TAVG,202,H,,S, +$ clickhouse-local --query "SELECT * FROM '2021.csv.gz' LIMIT 10" --format PrettyCompact +┌─c1──────────┬───────c2─┬─c3───┬──c4─┬─c5───┬─c6───┬─c7─┬───c8─┐ +│ AE000041196 │ 20210101 │ TMAX │ 278 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ S │ ᴺᵁᴸᴸ │ +│ AE000041196 │ 20210101 │ PRCP │ 0 │ D │ ᴺᵁᴸᴸ │ S │ ᴺᵁᴸᴸ │ +│ AE000041196 │ 20210101 │ TAVG │ 214 │ H │ ᴺᵁᴸᴸ │ S │ ᴺᵁᴸᴸ │ +│ AEM00041194 │ 20210101 │ TMAX │ 266 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ S │ ᴺᵁᴸᴸ │ +│ AEM00041194 │ 20210101 │ TMIN │ 178 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ S │ ᴺᵁᴸᴸ │ +│ AEM00041194 │ 20210101 │ PRCP │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ S │ ᴺᵁᴸᴸ │ +│ AEM00041194 │ 20210101 │ TAVG │ 217 │ H │ ᴺᵁᴸᴸ │ S │ ᴺᵁᴸᴸ │ +│ AEM00041217 │ 20210101 │ TMAX │ 262 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ S │ ᴺᵁᴸᴸ │ +│ AEM00041217 │ 20210101 │ TMIN │ 155 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ S │ ᴺᵁᴸᴸ │ +│ AEM00041217 │ 20210101 │ TAVG │ 202 │ H │ ᴺᵁᴸᴸ │ S │ ᴺᵁᴸᴸ │ +└─────────────┴──────────┴──────┴─────┴──────┴──────┴────┴──────┘ ``` Summarizing the [format documentation](https://github.com/awslabs/open-data-docs/tree/main/docs/noaa/noaa-ghcn): @@ -88,7 +90,7 @@ Summarizing the format documentation and the columns in order: - SNOW - Snowfall (mm) - SNWD - Snow depth (mm) - TMAX - Maximum temperature (tenths of degrees C) - - TAVG - Average temperature (tenths of a degrees C) + - TAVG - Average temperature (tenths of a degree C) - TMIN - Minimum temperature (tenths of degrees C) - PSUN - Daily percent of possible sunshine (percent) - AWND - Average daily wind speed (tenths of meters per second) @@ -215,7 +217,7 @@ CREATE TABLE noaa ### Inserting from local file -Data can be inserted from local file as follows (from the ClickHouse client): +Data can be inserted from a local file as follows (from the ClickHouse client): ```sql INSERT INTO noaa FROM INFILE '/noaa_enriched.parquet' From 1c1e1512bf92c4788ce17f38cf228d4525cdb9eb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jan 2024 01:29:38 +0300 Subject: [PATCH 034/325] Update noaa.md --- docs/en/getting-started/example-datasets/noaa.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/noaa.md b/docs/en/getting-started/example-datasets/noaa.md index bc2e9fecae1..9a3ec7791b6 100644 --- a/docs/en/getting-started/example-datasets/noaa.md +++ b/docs/en/getting-started/example-datasets/noaa.md @@ -185,7 +185,7 @@ FROM file('noaa.csv', CSV, 'station_id String, date Date32, tempAvg Int32, tempMax Int32, tempMin Int32, precipitation Int32, snowfall Int32, snowDepth Int32, percentDailySun Int8, averageWindSpeed Int32, maxWindSpeed Int32, weatherType UInt8') as noaa LEFT OUTER JOIN stations ON noaa.station_id = stations.id INTO OUTFILE 'noaa_enriched.parquet' FORMAT Parquet SETTINGS format_regexp='^(.{11})\s+(\-?\d{1,2}\.\d{4})\s+(\-?\d{1,3}\.\d{1,4})\s+(\-?\d*\.\d*)\s+(.*)\s+(?:[\d]*)'" ``` -This query takes a few minutes to run and produces an 6.4GB file `noaa_enriched.parquet`. +This query takes a few minutes to run and produces a 6.4 GB file, `noaa_enriched.parquet`. ## Create table From 7bc6a858c7778911a51e4c2430125f9c3741a535 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 16 Jan 2024 19:44:55 +0100 Subject: [PATCH 035/325] Rewrite bash test to integration test --- .../test_broken_projections/__init__.py | 0 .../config.d/backups.xml | 13 + .../test_broken_projections/test.py | 492 +++++++++++++++++ .../02916_broken_projection.reference | 322 ----------- .../0_stateless/02916_broken_projection.sh | 515 ------------------ 5 files changed, 505 insertions(+), 837 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 delete mode 100644 tests/queries/0_stateless/02916_broken_projection.reference delete mode 100755 tests/queries/0_stateless/02916_broken_projection.sh 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..ca1a29817a5 --- /dev/null +++ b/tests/integration/test_broken_projections/test.py @@ -0,0 +1,492 @@ +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 = ""): + if data_prefix == "": + data_prefix = table + + node.query( + f""" + DROP TABLE IF EXISTS {table} SYNC; + CREATE TABLE {table} + ( + a String, + b String, + c Int64, + d Int64, + e Int64, + + PROJECTION proj + ( + SELECT c ORDER BY d + ), + PROJECTION proj_2 + ( + 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=3, + enable_vertical_merge_algorithm=1, + vertical_merge_algorithm_min_rows_to_activate = 1, + vertical_merge_algorithm_min_columns_to_activate = 1, + vertical_merge_algorithm_min_columns_to_activate = 1, + 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 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=""): + query_id = random_str() + + if expect_broken_part == "proj": + assert expected_error in node.query_and_get_error( + f"SELECT c FROM '{table}' WHERE d == 12 ORDER BY c" + ) + else: + node.query( + f"SELECT c FROM '{table}' WHERE d == 12 OR d == 16 ORDER BY c", + query_id=query_id, + ) + assert "proj" in node.query( + f""" + SYSTEM FLUSH LOGS; + SELECT query, splitByChar('.', arrayJoin(projections))[-1] + FROM system.query_log + WHERE current_database=currentDatabase() AND query_id='{query_id}' AND type='QueryFinish' + """ + ) + + query_id = random_str() + + if expect_broken_part == "proj_2": + assert expected_error in node.query_and_get_error( + f"SELECT d FROM '{table}' WHERE c == 12 ORDER BY d" + ) + else: + node.query( + f"SELECT d FROM '{table}' WHERE c == 12 OR c == 16 ORDER BY d", + query_id=query_id, + ) + assert "proj" in node.query( + f""" + SYSTEM FLUSH LOGS; + SELECT query, splitByChar('.', arrayJoin(projections))[-1] + FROM system.query_log + WHERE current_database=currentDatabase() AND query_id='{query_id}' AND type='QueryFinish' + """ + ) + + 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 'proj' + break_projection(node, table_name, "proj", "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 'proj' 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\tproj\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 'proj_2' for part all_2_2_0 + break_projection(node, table_name, "proj_2", "all_2_2_0", "data") + + # It will not yet appear in broken projections info. + assert "proj_2" not in get_broken_projections_info(node, table_name) + + # Select now fails with error "File doesn't exist" + check(node, table_name, 0, "proj_2", "FILE_DOESNT_EXIST") + + # Projection 'proj_2' from part all_2_2_0 will now appear in broken parts info. + assert "all_2_2_0\tproj_2\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 'proj_2' for part all_3_3_0 + break_projection(node, table_name, "proj_2", "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 'proj_2' projections, but 'proj_2' 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 'proj_2' of part all_3_3_0 and proj_2 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 'proj_2' 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%proj_2.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, "proj", "all_1_1_0", "metadata") + reattach(node, table_name) + + assert "all_1_1_0\tproj\tNO_FILE_IN_DATA_PART" in get_broken_projections_info( + node, table_name + ) + assert "Part all_1_1_0 has a broken projection proj" in check_table_full( + node, table_name + ) + + break_projection(node, table_name, "proj_2", "all_1_1_0", "data") + reattach(node, table_name) + + assert "all_1_1_0\tproj_2\tFILE_DOESNT_EXIST" in get_broken_projections_info( + node, table_name + ) + assert "Part all_1_1_0 has a broken projection proj_2" in check_table_full( + node, table_name + ) + + materialize_projection(node, table_name, "proj") + + 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, "proj", "all_0_0_0", "data") + assert "Part all_0_0_0 has a broken projection proj" 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 test_broken_projections_in_backups(cluster): + node = cluster.instances["node"] + + table_name = "test4" + 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) + + break_projection(node, table_name, "proj", "all_2_2_0", "data") + check(node, table_name, 0, "proj", "FILE_DOESNT_EXIST") + + assert "all_2_2_0\tproj\tNO_FILE_IN_DATA_PART" in get_broken_projections_info( + node, table_name + ) + + assert "BACKUP_CREATED" in node.query( + f""" + set backup_restore_keeper_fault_injection_probability=0.0; + backup table {table_name} to Disk('backups', 'b1') 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', 'b1'); + """ + ) + + check(node, table_name, 1) + + assert "" == get_broken_projections_info(node, table_name) + # TODO: add a check for what projections are loaded + + break_projection(node, table_name, "proj", "all_2_2_0", "part") + + check(node, table_name, 0, "proj", "ErrnoException") + + assert "all_2_2_0\tproj\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, "proj") + check(node, table_name, 1) + # TODO: + # assert "all_2_2_0\tproj\tFILE_DOESNT_EXIST" == get_broken_projections_info(node, table_name) + + assert "BACKUP_CREATED" in node.query( + f""" + set backup_restore_keeper_fault_injection_probability=0.0; + backup table {table_name} to Disk('backups', 'b3') 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', 'b3'); + """ + ) + check(node, table_name, 1) + + break_projection(node, table_name, "proj", "all_1_1_0", "part") + # TODO: check(node, table_name, 0, "proj", "FILE_DOESNT_EXIST") + assert "Part all_1_1_0 has a broken projection proj" in check_table_full( + node, table_name + ) + assert "all_1_1_0\tproj\tFILE_DOESNT_EXIST" == get_broken_projections_info( + node, table_name + ) + + assert "BACKUP_CREATED" in node.query( + f""" + set backup_restore_keeper_fault_injection_probability=0.0; + backup table {table_name} to Disk('backups', 'b4') 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', 'b4'); + """ + ) + check(node, table_name, 1) + assert "" == get_broken_projections_info(node, table_name) diff --git a/tests/queries/0_stateless/02916_broken_projection.reference b/tests/queries/0_stateless/02916_broken_projection.reference deleted file mode 100644 index 3967215e5de..00000000000 --- a/tests/queries/0_stateless/02916_broken_projection.reference +++ /dev/null @@ -1,322 +0,0 @@ -insert new part -insert new part -insert new part -insert new part -select from projection 'proj' -12 -16 -used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2' -12 -16 -used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -1 -0 -broke metadata of part 'proj' (parent part: all_2_2_0) -select from projection 'proj' -12 -16 -used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2' -12 -16 -used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -0 -broken projections info -all_2_2_0 proj FILE_DOESNT_EXIST -check table full (test - all_2_2_0) -all_2_2_0 -0 -broke data of part 'proj_2' (parent part: all_2_2_0) -broken projections info -all_2_2_0 proj FILE_DOESNT_EXIST -select from projection 'proj' -12 -16 -used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2', expect error: proj_2 -FILE_DOESNT_EXIST -check table -0 -broken projections info -all_2_2_0 proj FILE_DOESNT_EXIST -all_2_2_0 proj_2 NO_FILE_IN_DATA_PART -select from projection 'proj' -12 -16 -used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2' -12 -16 -used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -0 -0 -broke data of part 'proj_2' (parent part: all_3_3_0) -broken projections info -all_2_2_0 proj FILE_DOESNT_EXIST -all_2_2_0 proj_2 NO_FILE_IN_DATA_PART -insert new part -insert new part -optimize -OPTIMIZE TABLE test SETTINGS alter_sync=0 -0 -broken projections info -all_2_2_0 proj FILE_DOESNT_EXIST -all_2_2_0 proj_2 NO_FILE_IN_DATA_PART -all_3_3_0 proj_2 NO_FILE_IN_DATA_PART -select from projection 'proj' -12 -16 -used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2' -12 -16 -used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -0 -0 -broke metadata of part 'proj' (parent part: all_1_1_0) -Detach - Attach -broken projections info -all_1_1_0 proj NO_FILE_IN_DATA_PART -all_2_2_0 proj NO_FILE_IN_DATA_PART -all_2_2_0 proj_2 FILE_DOESNT_EXIST -all_3_3_0 proj_2 FILE_DOESNT_EXIST -0 -broke data of part 'proj_2' (parent part: all_1_1_0) -Detach - Attach -broken projections info -all_1_1_0 proj NO_FILE_IN_DATA_PART -all_1_1_0 proj_2 FILE_DOESNT_EXIST -all_2_2_0 proj NO_FILE_IN_DATA_PART -all_2_2_0 proj_2 FILE_DOESNT_EXIST -all_3_3_0 proj_2 FILE_DOESNT_EXIST -select from projection 'proj' -12 -16 -used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2' -12 -16 -used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -0 -check table full (test - all_1_1_0) -all_1_1_0 -materialize projection proj -check table full (test - ) -select from projection 'proj' -12 -16 -used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2' -12 -16 -used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -1 -materialize projection proj_2 -check table full (test - ) -0 -broke data of part 'proj' (parent part: all_3_5_1_7) -insert new part -optimize -OPTIMIZE TABLE test FINAL -insert new part -optimize -OPTIMIZE TABLE test FINAL -select from projection 'proj' -12 -16 -used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj_2 -select from projection 'proj_2' -12 -16 -used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -1 -insert new part -insert new part -insert new part -insert new part -select from projection 'proj' -used projections -SELECT c FROM test2 WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2' -used projections -SELECT d FROM test2 WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -1 -select from projection 'proj' -used projections -SELECT c FROM test2_replica WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2' -used projections -SELECT d FROM test2_replica WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -1 -0 -broke data of part 'proj' (parent part: all_0_0_0) -check table full (test2 - all_0_0_0) -all_0_0_0 -select from projection 'proj' -used projections -SELECT c FROM test2 WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2' -used projections -SELECT d FROM test2 WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -0 -broke data of part 'all_0_0_0' -check table full (test2 - all_0_0_0) -all_0_0_0 -select from projection 'proj' -used projections -SELECT c FROM test2 WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2' -used projections -SELECT d FROM test2 WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -1 -select from projection 'proj' -used projections -SELECT c FROM test2 WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2' -used projections -SELECT d FROM test2 WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -1 -insert new part -insert new part -insert new part -insert new part -select from projection 'proj' -12 -16 -used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2' -12 -16 -used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -1 -0 -broke data of part 'proj' (parent part: all_2_2_0) -select from projection 'proj', expect error: proj -FILE_DOESNT_EXIST -select from projection 'proj_2' -12 -16 -used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -0 -broken projections info -all_2_2_0 proj NO_FILE_IN_DATA_PART -BACKUP_CREATED -RESTORED -select from projection 'proj' -12 -16 -used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2' -12 -16 -used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -1 -broken projections info -0 -broke all data of part 'proj' (parent part: all_2_2_0) -select from projection 'proj', expect error: proj -Errno -Errno -select from projection 'proj_2' -12 -16 -used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -0 -broken projections info -all_2_2_0 proj FILE_DOESNT_EXIST -FILE_DOESNT_EXIST -materialize projection proj -select from projection 'proj' -12 -16 -used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2' -12 -16 -used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -1 -broken projections info -all_2_2_0 proj FILE_DOESNT_EXIST -BACKUP_CREATED -RESTORED -select from projection 'proj' -12 -16 -used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2' -12 -16 -used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -1 -0 -broke all data of part 'proj' (parent part: all_1_1_0) -select from projection 'proj', expect error: proj -select from projection 'proj_2' -12 -16 -used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -0 -broken projections info -all_1_1_0 proj FILE_DOESNT_EXIST -BACKUP_CREATED -RESTORED -select from projection 'proj' -12 -16 -used projections -SELECT c FROM test WHERE d == 12 OR d == 16 ORDER BY c; proj -select from projection 'proj_2' -12 -16 -used projections -SELECT d FROM test WHERE c == 12 OR c == 16 ORDER BY d; proj_2 -check table -0 -broken projections info -all_1_1_0 proj NO_FILE_IN_DATA_PART diff --git a/tests/queries/0_stateless/02916_broken_projection.sh b/tests/queries/0_stateless/02916_broken_projection.sh deleted file mode 100755 index fbd26e59f6f..00000000000 --- a/tests/queries/0_stateless/02916_broken_projection.sh +++ /dev/null @@ -1,515 +0,0 @@ -#!/usr/bin/env bash -# Tags: long, no-random-merge-tree-settings, no-random-settings, no-s3-storage, no-parallel -# shellcheck disable=SC2046 - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -function create_table() -{ - test_id=$1 - name=$2 - replica=$3 - $CLICKHOUSE_CLIENT -nm -q " - DROP TABLE IF EXISTS $name SYNC; - CREATE TABLE $name - ( - a String, - b String, - c Int64, - d Int64, - e Int64, - - PROJECTION proj - ( - SELECT c ORDER BY d - ), - PROJECTION proj_2 - ( - SELECT d ORDER BY c - ) - ) - ENGINE = ReplicatedMergeTree('/test_broken_projection_32_$test_id/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/', '$replica') ORDER BY a - SETTINGS min_bytes_for_wide_part = 0, - max_parts_to_merge_at_once=3, - enable_vertical_merge_algorithm=1, - vertical_merge_algorithm_min_rows_to_activate = 1, - vertical_merge_algorithm_min_columns_to_activate = 1, - vertical_merge_algorithm_min_columns_to_activate = 1, - compress_primary_key=0; - " -} - -function random() -{ - cat /dev/urandom | LC_ALL=C tr -dc 'a-zA-Z' | fold -w ${1:-8} | head -n 1 -} - -function insert() -{ - table=$1 - offset=$2 - size=$3 - echo 'insert new part' - $CLICKHOUSE_CLIENT -q "INSERT INTO $table SELECT number, number, number, number, number%2 FROM numbers($offset, $size) SETTINGS insert_keeper_fault_injection_probability=0.0;" -} - -function break_projection() -{ - table=$1 - part_name=$2 - parent_name=$3 - break_type=$4 - - read -r part_path <<< $($CLICKHOUSE_CLIENT -nm -q " - SELECT path - FROM system.projection_parts - WHERE table='$table' - AND database=currentDatabase() - AND active=1 - AND part_name='$part_name' - AND parent_name='$parent_name' - ORDER BY modification_time DESC - LIMIT 1; - ") - - $CLICKHOUSE_CLIENT -q "select throwIf(substring('$part_path', 1, 1) != '/', 'Path is relative: $part_path')" || exit - - if [ "$break_type" = "data" ] - then - rm "$part_path/d.bin" - rm "$part_path/c.bin" - echo "broke data of part '$part_name' (parent part: $parent_name)" - fi - if [ "$break_type" = "metadata" ] - then - rm "$part_path/columns.txt" - echo "broke metadata of part '$part_name' (parent part: $parent_name)" - fi - if [ "$break_type" = "part" ] - then - rm -r "$part_path" - echo "broke all data of part '$part_name' (parent part: $parent_name)" - fi -} - -function break_part() -{ - table=$1 - part_name=$2 - - read -r part_path <<< $($CLICKHOUSE_CLIENT -nm -q " - SELECT path - FROM system.parts - WHERE table='$table' - AND database=currentDatabase() - AND active=1 - AND part_name='$part_name' - ORDER BY modification_time DESC - LIMIT 1; - ") - - if [ "$part_path" = "" ] - then - echo "Part path is empty" - exit - fi - - rm $part_path/columns.txt - echo "broke data of part '$part_name'" -} - -function broken_projections_info() -{ - table=$1 - echo 'broken projections info' - $CLICKHOUSE_CLIENT -q " - 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 -" -} - -function check() -{ - table=$1 - expect_broken_part="" - expected_error="" - if [ $# -gt 1 ]; then - expect_broken_part=$2 - expected_error=$3 - fi - - #echo 'system.parts' - #$CLICKHOUSE_CLIENT -q " - #SELECT name, active, projections - #FROM system.parts - #WHERE table='$table' AND database=currentDatabase() - #ORDER BY name;" - - query_id=$(random 8) - - if [ "$expect_broken_part" = "proj" ] - then - echo "select from projection 'proj', expect error: $expect_broken_part" - $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -nm -q " - SET send_logs_level='fatal'; - SELECT c FROM $table WHERE d == 12 ORDER BY c; - " 2>&1 | grep -oF "$expected_error" - else - echo "select from projection 'proj'" - $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT c FROM $table WHERE d == 12 OR d == 16 ORDER BY c;" - echo 'used projections' - $CLICKHOUSE_CLIENT -nm -q " - SYSTEM FLUSH LOGS; - SELECT query, splitByChar('.', arrayJoin(projections))[-1] FROM system.query_log WHERE current_database=currentDatabase() AND query_id='$query_id' AND type='QueryFinish' - " - fi - - query_id=$(random 8) - - if [ "$expect_broken_part" = "proj_2" ] - then - echo "select from projection 'proj_2', expect error: $expect_broken_part" - $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -nm -q " - SET send_logs_level='fatal'; - SELECT d FROM $table WHERE c == 12 ORDER BY d; - " 2>&1 | grep -oF "$expected_error" - else - echo "select from projection 'proj_2'" - $CLICKHOUSE_CLIENT --optimize_use_projections 1 --query_id $query_id -q "SELECT d FROM $table WHERE c == 12 OR c == 16 ORDER BY d;" - echo 'used projections' - $CLICKHOUSE_CLIENT -nm -q " - SYSTEM FLUSH LOGS; - SELECT query, splitByChar('.', arrayJoin(projections))[-1] FROM system.query_log WHERE current_database=currentDatabase() AND query_id='$query_id' AND type='QueryFinish' - " - fi - - echo 'check table' - $CLICKHOUSE_CLIENT -nm -q " - SET send_logs_level='fatal'; - CHECK TABLE $table;" -} - -function optimize() -{ - final=$1 - no_wait=$2 - - echo 'optimize' - query="OPTIMIZE TABLE test" - - if [ $final -eq 1 ]; then - query="$query FINAL" - fi - if [ $no_wait -eq 1 ]; then - query="$query SETTINGS alter_sync=0" - fi - - echo $query - - $CLICKHOUSE_CLIENT -q "$query" -} - -function reattach() -{ - echo 'Detach - Attach' - $CLICKHOUSE_CLIENT -nm -q " - SET send_logs_level='fatal'; - DETACH TABLE test; - ATTACH TABLE test; - " -} - -function materialize_projection -{ - table=$1 - projection=$2 - echo "materialize projection $projection" - $CLICKHOUSE_CLIENT -q "ALTER TABLE $table MATERIALIZE PROJECTION $projection SETTINGS mutations_sync=2" -} - -function check_table_full() -{ - table=$1 - expect_broken_part=$2 - echo "check table full ($1 - $2)" - if [ "$expect_broken_part" = "" ] - then - $CLICKHOUSE_CLIENT -nm -q " - SET send_logs_level='fatal'; - CHECK TABLE $table SETTINGS check_query_single_value_result = 0; - " | grep "broken" - else - $CLICKHOUSE_CLIENT -nm -q " - SET send_logs_level='fatal'; - CHECK TABLE $table SETTINGS check_query_single_value_result = 0; - " | grep "broken" | grep -o $expect_broken_part | head -n 1 - fi -} - -function test1() -{ - create_table test1 test 1 - - table_uuid=$($CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.tables WHERE table='test' and database=currentDatabase()") - - insert test 0 5 - - insert test 5 5 - - insert test 10 5 - - insert test 15 5 - - check test - - # Break metadata file of projection 'proj' - break_projection test proj all_2_2_0 metadata - - # Do select and after "check table" query. - # Select works because it does not read columns.txt. - check test - - # Projection 'proj' from part all_2_2_0 will now appear in broken parts info - # because it was marked broken during "check table" query. - # TODO: try to mark it during select as well - broken_projections_info test - - # Check table query will also show a list of parts which have broken projections. - check_table_full test "all_2_2_0" - - # Break data file of projection 'proj_2' for part all_2_2_0 - break_projection test proj_2 all_2_2_0 data - - # It will not yet appear in broken projections info. - broken_projections_info test - - # Select now fails with error "File doesn't exist" - check test "proj_2" FILE_DOESNT_EXIST - - # Projection 'proj_2' from part all_2_2_0 will now appear in broken parts info. - broken_projections_info test - - # Second select works, because projection is now marked as broken. - check test - - # Break data file of projection 'proj_2' for part all_3_3_0 - break_projection test proj_2 all_3_3_0 data - - # It will not yet appear in broken projections info. - broken_projections_info test - - insert test 20 5 - - insert test 25 5 - - # Part all_3_3_0 has 'proj' and 'proj_2' projections, but 'proj_2' 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 'proj_2' of part all_3_3_0 and proj_2 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 'proj_2' as it will see that one part does not have it anymore in the set of valid projections. - optimize 0 1 - sleep 2 - - $CLICKHOUSE_CLIENT -nm -q " - 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%proj_2.proj/c.bin%' - " - - # Projection 'proj_2' from part all_2_2_0 will now appear in broken parts info. - broken_projections_info test - - check test - - break_projection test proj all_1_1_0 metadata - - reattach - - broken_projections_info test - - break_projection test proj_2 all_1_1_0 data - - reattach - - broken_projections_info test - - check test - - check_table_full test all_1_1_0 - - materialize_projection test proj - - check_table_full test - - check test - - materialize_projection test proj_2 - - check_table_full test - - break_projection test proj all_3_5_1_7 data - - insert test 30 5 - - optimize 1 0 - - insert test 35 5 - - optimize 1 0 - - check test -} - -function test2() -{ - create_table test2 test2 1 - - insert test2 0 5 - - insert test2 5 5 - - insert test 10 5 - - insert test 15 5 - - check test2 - - create_table test2 test2_replica 2 - - check test2_replica - - break_projection test2 proj all_0_0_0 data - - check_table_full test2 all_0_0_0 - - check test2 - - break_part test2 all_0_0_0 - - check_table_full test2 all_0_0_0 - - check test2 - - $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA test2;" - - check test2 -} - -function test3() -{ - create_table test3 test 1 - - insert test 0 5 - - insert test 5 5 - - insert test 10 5 - - insert test 15 5 - - check test - - break_projection test proj all_2_2_0 data - - check test proj FILE_DOESNT_EXIST - - broken_projections_info test - - ${CLICKHOUSE_CLIENT} -nm --query " - set backup_restore_keeper_fault_injection_probability=0.0; - backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}') settings check_projection_parts=false; - " | grep -o "BACKUP_CREATED" - - ${CLICKHOUSE_CLIENT} -nm --stacktrace --query " - drop table test sync; - set backup_restore_keeper_fault_injection_probability=0.0; - restore table ${CLICKHOUSE_DATABASE}.test from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}'); - " | grep -o "RESTORED" - - check test - - broken_projections_info test - - break_projection test proj all_2_2_0 part - - check test proj Errno - - broken_projections_info test - - ${CLICKHOUSE_CLIENT} -nm --query " - set send_logs_level='fatal'; - set backup_restore_keeper_fault_injection_probability=0.0; - backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_2') - " 2>&1 | grep -o "FILE_DOESNT_EXIST" - - materialize_projection test proj - - check test - - broken_projections_info test - - ${CLICKHOUSE_CLIENT} -nm --query " - set send_logs_level='fatal'; - set backup_restore_keeper_fault_injection_probability=0.0; - backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_3') - " | grep -o "BACKUP_CREATED" - - ${CLICKHOUSE_CLIENT} -nm --stacktrace --query " - drop table test sync; - set send_logs_level='fatal'; - set backup_restore_keeper_fault_injection_probability=0.0; - restore table ${CLICKHOUSE_DATABASE}.test from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_3'); - " | grep -o "RESTORED" - - check test - - break_projection test proj all_1_1_0 part - - check test proj FILE_DOESNT_EXIST - - broken_projections_info test - - ${CLICKHOUSE_CLIENT} -nm --query " - set send_logs_level='fatal'; - set backup_restore_keeper_fault_injection_probability=0.0; - backup table ${CLICKHOUSE_DATABASE}.test to Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_4') - settings check_projection_parts=false, allow_backup_broken_projections=true; - " | grep -o "BACKUP_CREATED" - - ${CLICKHOUSE_CLIENT} -nm --stacktrace --query " - drop table test sync; - set send_logs_level='fatal'; - set backup_restore_keeper_fault_injection_probability=0.0; - restore table ${CLICKHOUSE_DATABASE}.test from Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}_4'); - " | grep -o "RESTORED" - - check test - - broken_projections_info test -} - -test1 -test2 -test3 - - -$CLICKHOUSE_CLIENT -nm -q " -DROP TABLE IF EXISTS test SYNC; -DROP TABLE IF EXISTS test2 SYNC; -DROP TABLE IF EXISTS test2_replica SYNC; -" From 216769f43ea536a38b9e7d5650cdd02fae972caf Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 16 Jan 2024 18:55:23 +0000 Subject: [PATCH 036/325] Automatic style fix --- tests/integration/test_broken_projections/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py index ca1a29817a5..cc3e55402b3 100644 --- a/tests/integration/test_broken_projections/test.py +++ b/tests/integration/test_broken_projections/test.py @@ -28,7 +28,7 @@ def cluster(): cluster.shutdown() -def create_table(node, table, replica, data_prefix = ""): +def create_table(node, table, replica, data_prefix=""): if data_prefix == "": data_prefix = table From dea8b10ae972b4fc2b20dbf90d90e362bf7e4207 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 17 Jan 2024 15:39:57 +0100 Subject: [PATCH 037/325] Fix test --- tests/integration/test_broken_projections/test.py | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py index ca1a29817a5..90d82f9f010 100644 --- a/tests/integration/test_broken_projections/test.py +++ b/tests/integration/test_broken_projections/test.py @@ -425,9 +425,7 @@ def test_broken_projections_in_backups(cluster): ) check(node, table_name, 1) - assert "" == get_broken_projections_info(node, table_name) - # TODO: add a check for what projections are loaded break_projection(node, table_name, "proj", "all_2_2_0", "part") @@ -446,8 +444,6 @@ def test_broken_projections_in_backups(cluster): materialize_projection(node, table_name, "proj") check(node, table_name, 1) - # TODO: - # assert "all_2_2_0\tproj\tFILE_DOESNT_EXIST" == get_broken_projections_info(node, table_name) assert "BACKUP_CREATED" in node.query( f""" @@ -466,7 +462,6 @@ def test_broken_projections_in_backups(cluster): check(node, table_name, 1) break_projection(node, table_name, "proj", "all_1_1_0", "part") - # TODO: check(node, table_name, 0, "proj", "FILE_DOESNT_EXIST") assert "Part all_1_1_0 has a broken projection proj" in check_table_full( node, table_name ) @@ -477,7 +472,7 @@ def test_broken_projections_in_backups(cluster): assert "BACKUP_CREATED" in node.query( f""" set backup_restore_keeper_fault_injection_probability=0.0; - backup table {table_name} to Disk('backups', 'b4') settings check_projection_parts=false; + backup table {table_name} to Disk('backups', 'b4') settings check_projection_parts=false, allow_backup_broken_projections=true; """ ) @@ -488,5 +483,5 @@ def test_broken_projections_in_backups(cluster): restore table {table_name} from Disk('backups', 'b4'); """ ) - check(node, table_name, 1) - assert "" == get_broken_projections_info(node, table_name) + check(node, table_name, 0) + assert "all_1_1_0\tproj\tNO_FILE_IN_DATA_PART" == get_broken_projections_info(node, table_name) From 1e9de73bf57de1eb66007cba0fecb9f0459c973e Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 17 Jan 2024 15:40:46 +0100 Subject: [PATCH 038/325] Fix style check --- tests/integration/test_broken_projections/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py index eb8c452fff0..1b192e0df24 100644 --- a/tests/integration/test_broken_projections/test.py +++ b/tests/integration/test_broken_projections/test.py @@ -484,4 +484,6 @@ def test_broken_projections_in_backups(cluster): """ ) check(node, table_name, 0) - assert "all_1_1_0\tproj\tNO_FILE_IN_DATA_PART" == get_broken_projections_info(node, table_name) + assert "all_1_1_0\tproj\tNO_FILE_IN_DATA_PART" == get_broken_projections_info( + node, table_name + ) From 0a6331f5f756d5d6465095edac89d2a03618d773 Mon Sep 17 00:00:00 2001 From: Dale Mcdiarmid Date: Thu, 18 Jan 2024 13:18:13 +0000 Subject: [PATCH 039/325] spell-check additions --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 73b7a081797..1f6b24597da 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1,4 +1,4 @@ -personal_ws-1.1 en 2657 +personal_ws-1.1 en 2697 AArch ACLs ALTERs @@ -2016,6 +2016,7 @@ pcre performant perl persistency +personal_ws-1.1 en 2657 phpclickhouse pipelining plaintext From 8b730811efd055f1b200f277202a34258a93722e Mon Sep 17 00:00:00 2001 From: Dale McDiarmid Date: Thu, 18 Jan 2024 13:24:40 +0000 Subject: [PATCH 040/325] Update aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 73b7a081797..1f6b24597da 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1,4 +1,4 @@ -personal_ws-1.1 en 2657 +personal_ws-1.1 en 2697 AArch ACLs ALTERs @@ -2016,6 +2016,7 @@ pcre performant perl persistency +personal_ws-1.1 en 2657 phpclickhouse pipelining plaintext From a196d04a1c2d5f36ec43c2b0947916be7321037c Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 19 Jan 2024 12:04:16 +0100 Subject: [PATCH 041/325] Update test --- .../test_broken_projections/test.py | 94 +++++++++++++++++-- 1 file changed, 84 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py index 1b192e0df24..48ed10d0f87 100644 --- a/tests/integration/test_broken_projections/test.py +++ b/tests/integration/test_broken_projections/test.py @@ -159,6 +159,19 @@ def get_broken_projections_info(node, table): ).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: @@ -389,6 +402,11 @@ def test_broken_ignored_replicated(cluster): 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(cluster): node = cluster.instances["node"] @@ -400,6 +418,10 @@ def test_broken_projections_in_backups(cluster): 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, "proj", "all_2_2_0", "data") @@ -409,10 +431,23 @@ def test_broken_projections_in_backups(cluster): node, table_name ) + assert ( + "all_0_0_0\tproj\t0\n" + "all_0_0_0\tproj_2\t0\n" + "all_1_1_0\tproj\t0\n" + "all_1_1_0\tproj_2\t0\n" + "all_2_2_0\tproj\t1\n" + "all_2_2_0\tproj_2\t0\n" + "all_3_3_0\tproj\t0\n" + "all_3_3_0\tproj_2\t0" + == get_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', 'b1') settings check_projection_parts=false; + backup table {table_name} to Disk('backups', '{backup_name}') settings check_projection_parts=false; """ ) @@ -420,18 +455,30 @@ def test_broken_projections_in_backups(cluster): f""" drop table {table_name} sync; set backup_restore_keeper_fault_injection_probability=0.0; - restore table {table_name} from Disk('backups', 'b1'); + restore table {table_name} from Disk('backups', '{backup_name}'); """ ) + assert ( + "all_0_0_0\tproj\t0\n" + "all_0_0_0\tproj_2\t0\n" + "all_1_1_0\tproj\t0\n" + "all_1_1_0\tproj_2\t0\n" + "all_2_2_0\tproj\t0\n" + "all_2_2_0\tproj_2\t0\n" + "all_3_3_0\tproj\t0\n" + "all_3_3_0\tproj_2\t0" + == get_projections_info(node, table_name) + ) + check(node, table_name, 1) assert "" == get_broken_projections_info(node, table_name) - break_projection(node, table_name, "proj", "all_2_2_0", "part") + break_projection(node, table_name, "proj_2", "all_2_2_0", "part") - check(node, table_name, 0, "proj", "ErrnoException") + check(node, table_name, 0, "proj_2", "ErrnoException") - assert "all_2_2_0\tproj\tFILE_DOESNT_EXIST" == get_broken_projections_info( + assert "all_2_2_0\tproj_2\tFILE_DOESNT_EXIST" == get_broken_projections_info( node, table_name ) @@ -442,13 +489,14 @@ def test_broken_projections_in_backups(cluster): """ ) - materialize_projection(node, table_name, "proj") + materialize_projection(node, table_name, "proj_2") 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', 'b3') settings check_projection_parts=false; + backup table {table_name} to Disk('backups', '{backup_name}') settings check_projection_parts=false; """ ) @@ -456,11 +504,23 @@ def test_broken_projections_in_backups(cluster): f""" drop table {table_name} sync; set backup_restore_keeper_fault_injection_probability=0.0; - restore table {table_name} from Disk('backups', 'b3'); + restore table {table_name} from Disk('backups', '{backup_name}'); """ ) check(node, table_name, 1) + assert ( + "all_0_0_0\tproj\t0\n" + "all_0_0_0\tproj_2\t0\n" + "all_1_1_0\tproj\t0\n" + "all_1_1_0\tproj_2\t0\n" + "all_2_2_0\tproj\t0\n" + "all_2_2_0\tproj_2\t0\n" + "all_3_3_0\tproj\t0\n" + "all_3_3_0\tproj_2\t0" + == get_projections_info(node, table_name) + ) + break_projection(node, table_name, "proj", "all_1_1_0", "part") assert "Part all_1_1_0 has a broken projection proj" in check_table_full( node, table_name @@ -469,10 +529,11 @@ def test_broken_projections_in_backups(cluster): 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', 'b4') settings check_projection_parts=false, allow_backup_broken_projections=true; + backup table {table_name} to Disk('backups', '{backup_name}') settings check_projection_parts=false, allow_backup_broken_projections=true; """ ) @@ -480,9 +541,22 @@ def test_broken_projections_in_backups(cluster): f""" drop table {table_name} sync; set backup_restore_keeper_fault_injection_probability=0.0; - restore table {table_name} from Disk('backups', 'b4'); + restore table {table_name} from Disk('backups', '{backup_name}'); """ ) + + assert ( + "all_0_0_0\tproj\t0\n" + "all_0_0_0\tproj_2\t0\n" + "all_1_1_0\tproj\t1\n" + "all_1_1_0\tproj_2\t0\n" + "all_2_2_0\tproj\t0\n" + "all_2_2_0\tproj_2\t0\n" + "all_3_3_0\tproj\t0\n" + "all_3_3_0\tproj_2\t0" + == get_projections_info(node, table_name) + ) + check(node, table_name, 0) assert "all_1_1_0\tproj\tNO_FILE_IN_DATA_PART" == get_broken_projections_info( node, table_name From 7c66141e08ec203dbff908d69d929ea3bfc0995f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 19 Jan 2024 11:11:13 +0000 Subject: [PATCH 042/325] Automatic style fix --- tests/integration/test_broken_projections/test.py | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py index 48ed10d0f87..8e3978a078e 100644 --- a/tests/integration/test_broken_projections/test.py +++ b/tests/integration/test_broken_projections/test.py @@ -439,8 +439,7 @@ def test_broken_projections_in_backups(cluster): "all_2_2_0\tproj\t1\n" "all_2_2_0\tproj_2\t0\n" "all_3_3_0\tproj\t0\n" - "all_3_3_0\tproj_2\t0" - == get_projections_info(node, table_name) + "all_3_3_0\tproj_2\t0" == get_projections_info(node, table_name) ) backup_name = f"b1-{get_random_string()}" @@ -467,8 +466,7 @@ def test_broken_projections_in_backups(cluster): "all_2_2_0\tproj\t0\n" "all_2_2_0\tproj_2\t0\n" "all_3_3_0\tproj\t0\n" - "all_3_3_0\tproj_2\t0" - == get_projections_info(node, table_name) + "all_3_3_0\tproj_2\t0" == get_projections_info(node, table_name) ) check(node, table_name, 1) @@ -517,8 +515,7 @@ def test_broken_projections_in_backups(cluster): "all_2_2_0\tproj\t0\n" "all_2_2_0\tproj_2\t0\n" "all_3_3_0\tproj\t0\n" - "all_3_3_0\tproj_2\t0" - == get_projections_info(node, table_name) + "all_3_3_0\tproj_2\t0" == get_projections_info(node, table_name) ) break_projection(node, table_name, "proj", "all_1_1_0", "part") @@ -553,8 +550,7 @@ def test_broken_projections_in_backups(cluster): "all_2_2_0\tproj\t0\n" "all_2_2_0\tproj_2\t0\n" "all_3_3_0\tproj\t0\n" - "all_3_3_0\tproj_2\t0" - == get_projections_info(node, table_name) + "all_3_3_0\tproj_2\t0" == get_projections_info(node, table_name) ) check(node, table_name, 0) From caf9d8df6d789203a4e408341c9494952eb14ad2 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sat, 20 Jan 2024 14:59:13 +0100 Subject: [PATCH 043/325] Update test.py --- tests/integration/test_broken_projections/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py index 8e3978a078e..d750bb5827d 100644 --- a/tests/integration/test_broken_projections/test.py +++ b/tests/integration/test_broken_projections/test.py @@ -413,6 +413,8 @@ def test_broken_projections_in_backups(cluster): table_name = "test4" create_table(node, table_name, 1) + node.qeury("SYSTEM STOP MERGES") + insert(node, table_name, 0, 5) insert(node, table_name, 5, 5) insert(node, table_name, 10, 5) @@ -557,3 +559,4 @@ def test_broken_projections_in_backups(cluster): assert "all_1_1_0\tproj\tNO_FILE_IN_DATA_PART" == get_broken_projections_info( node, table_name ) + node.qeury("SYSTEM START MERGES") From e5c3b67f379efdd6d403be08f8bce164348663a1 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sat, 20 Jan 2024 16:10:09 +0100 Subject: [PATCH 044/325] Update test.py --- tests/integration/test_broken_projections/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py index d750bb5827d..acf0160cf1b 100644 --- a/tests/integration/test_broken_projections/test.py +++ b/tests/integration/test_broken_projections/test.py @@ -413,7 +413,7 @@ def test_broken_projections_in_backups(cluster): table_name = "test4" create_table(node, table_name, 1) - node.qeury("SYSTEM STOP MERGES") + node.query("SYSTEM STOP MERGES") insert(node, table_name, 0, 5) insert(node, table_name, 5, 5) @@ -559,4 +559,4 @@ def test_broken_projections_in_backups(cluster): assert "all_1_1_0\tproj\tNO_FILE_IN_DATA_PART" == get_broken_projections_info( node, table_name ) - node.qeury("SYSTEM START MERGES") + node.query("SYSTEM START MERGES") From 15e3a5b3961ac304a30ef211594f57bda3a2f584 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 22 Jan 2024 14:57:10 +0100 Subject: [PATCH 045/325] Try fix flaky test --- .../integration/test_broken_projections/test.py | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py index acf0160cf1b..87d910b9c77 100644 --- a/tests/integration/test_broken_projections/test.py +++ b/tests/integration/test_broken_projections/test.py @@ -28,10 +28,17 @@ def cluster(): cluster.shutdown() -def create_table(node, table, replica, data_prefix=""): +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 + else: + vertical_merge_algorithm_min_rows_to_activate = 100000 + vertical_merge_algorithm_min_columns_to_activate = 100 + node.query( f""" DROP TABLE IF EXISTS {table} SYNC; @@ -56,9 +63,8 @@ def create_table(node, table, replica, data_prefix=""): SETTINGS min_bytes_for_wide_part = 0, max_parts_to_merge_at_once=3, enable_vertical_merge_algorithm=1, - vertical_merge_algorithm_min_rows_to_activate = 1, - vertical_merge_algorithm_min_columns_to_activate = 1, - vertical_merge_algorithm_min_columns_to_activate = 1, + 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; """ ) @@ -411,7 +417,7 @@ def test_broken_projections_in_backups(cluster): node = cluster.instances["node"] table_name = "test4" - create_table(node, table_name, 1) + create_table(node, table_name, 1, aggressive_merge=False) node.query("SYSTEM STOP MERGES") From 4e5249275ed67c52d958007978c66619db22a1a5 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 22 Jan 2024 16:45:25 +0100 Subject: [PATCH 046/325] Do not resolve remote table id on initiator --- src/Storages/StorageDistributed.cpp | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 987ea4a4957..9972517bbac 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -779,18 +779,11 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, } else { - auto resolved_remote_storage_id = remote_storage_id; - // In case of cross-replication we don't know what database is used for the table. - // `storage_id.hasDatabase()` can return false only on the initiator node. - // Each shard will use the default database (in the case of cross-replication shards may have different defaults). - if (remote_storage_id.hasDatabase()) - resolved_remote_storage_id = query_context->resolveStorageID(remote_storage_id); - auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals(); auto column_names_and_types = distributed_storage_snapshot->getColumns(get_column_options); - auto storage = std::make_shared(resolved_remote_storage_id, ColumnsDescription{column_names_and_types}); + auto storage = std::make_shared(remote_storage_id, ColumnsDescription{column_names_and_types}); auto table_node = std::make_shared(std::move(storage), query_context); if (table_expression_modifiers) From bef0fcb482c4b8782a1d2e485be9f6d8ffc2dfe9 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 23 Jan 2024 14:56:24 +0100 Subject: [PATCH 047/325] Fix is_order_by_all flag in QueryNode --- src/Analyzer/QueryNode.cpp | 28 +++++++++++++++++----------- 1 file changed, 17 insertions(+), 11 deletions(-) diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 738b1ac62e8..a82fb4489b5 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -119,6 +119,9 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s if (is_group_by_all) buffer << ", is_group_by_all: " << is_group_by_all; + if (is_order_by_all) + buffer << ", is_order_by_all: " << is_order_by_all; + std::string group_by_type; if (is_group_by_with_rollup) group_by_type = "rollup"; @@ -257,6 +260,7 @@ bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const is_group_by_with_cube == rhs_typed.is_group_by_with_cube && is_group_by_with_grouping_sets == rhs_typed.is_group_by_with_grouping_sets && is_group_by_all == rhs_typed.is_group_by_all && + is_order_by_all == rhs_typed.is_order_by_all && cte_name == rhs_typed.cte_name && projection_columns == rhs_typed.projection_columns && settings_changes == rhs_typed.settings_changes; @@ -288,6 +292,7 @@ void QueryNode::updateTreeHashImpl(HashState & state) const state.update(is_group_by_with_cube); state.update(is_group_by_with_grouping_sets); state.update(is_group_by_all); + state.update(is_order_by_all); state.update(settings_changes.size()); @@ -306,18 +311,19 @@ QueryTreeNodePtr QueryNode::cloneImpl() const { auto result_query_node = std::make_shared(context); - result_query_node->is_subquery = is_subquery; - result_query_node->is_cte = is_cte; - result_query_node->is_distinct = is_distinct; - result_query_node->is_limit_with_ties = is_limit_with_ties; - result_query_node->is_group_by_with_totals = is_group_by_with_totals; - result_query_node->is_group_by_with_rollup = is_group_by_with_rollup; - result_query_node->is_group_by_with_cube = is_group_by_with_cube; + result_query_node->is_subquery = is_subquery; + result_query_node->is_cte = is_cte; + result_query_node->is_distinct = is_distinct; + result_query_node->is_limit_with_ties = is_limit_with_ties; + result_query_node->is_group_by_with_totals = is_group_by_with_totals; + result_query_node->is_group_by_with_rollup = is_group_by_with_rollup; + result_query_node->is_group_by_with_cube = is_group_by_with_cube; result_query_node->is_group_by_with_grouping_sets = is_group_by_with_grouping_sets; - result_query_node->is_group_by_all = is_group_by_all; - result_query_node->cte_name = cte_name; - result_query_node->projection_columns = projection_columns; - result_query_node->settings_changes = settings_changes; + result_query_node->is_group_by_all = is_group_by_all; + result_query_node->is_order_by_all = is_order_by_all; + result_query_node->cte_name = cte_name; + result_query_node->projection_columns = projection_columns; + result_query_node->settings_changes = settings_changes; return result_query_node; } From 799a94081ba7587ec47d85554bdbb458ffb1436d Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 23 Jan 2024 11:40:42 -0300 Subject: [PATCH 048/325] Revert "Revert "Allow to attach partition from table with different partition expression when destination partition expression doesn't re-partition"" This reverts commit a1c83e2f51117a69d484c7ae7884c3bc5dd98129. --- .../statements/alter/partition.md | 2 +- src/Interpreters/MonotonicityCheckVisitor.h | 102 +++- src/Interpreters/applyFunction.cpp | 43 ++ src/Interpreters/applyFunction.h | 16 + src/Parsers/queryToString.cpp | 5 + src/Parsers/queryToString.h | 1 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 37 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 + src/Storages/MergeTree/KeyCondition.cpp | 83 +-- src/Storages/MergeTree/MergeTreeData.cpp | 350 +++++-------- src/Storages/MergeTree/MergeTreeData.h | 18 + .../MergeTree/MergeTreeDataPartCloner.cpp | 320 ++++++++++++ .../MergeTree/MergeTreeDataPartCloner.h | 43 ++ src/Storages/MergeTree/MergeTreePartition.cpp | 39 ++ src/Storages/MergeTree/MergeTreePartition.h | 10 +- ...ergeTreePartitionCompatibilityVerifier.cpp | 91 ++++ .../MergeTreePartitionCompatibilityVerifier.h | 30 ++ ...TreePartitionGlobalMinMaxIdxCalculator.cpp | 25 + ...geTreePartitionGlobalMinMaxIdxCalculator.h | 24 + src/Storages/StorageMergeTree.cpp | 93 +++- src/Storages/StorageReplicatedMergeTree.cpp | 135 ++++- .../__init__.py | 0 .../configs/remote_servers.xml | 17 + .../test.py | 214 ++++++++ ...artition_different_partition_exp.reference | 467 +++++++++++++++++ ...tach_partition_different_partition_exp.sql | 485 ++++++++++++++++++ 26 files changed, 2310 insertions(+), 341 deletions(-) create mode 100644 src/Interpreters/applyFunction.cpp create mode 100644 src/Interpreters/applyFunction.h create mode 100644 src/Storages/MergeTree/MergeTreeDataPartCloner.cpp create mode 100644 src/Storages/MergeTree/MergeTreeDataPartCloner.h create mode 100644 src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.cpp create mode 100644 src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.h create mode 100644 src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.cpp create mode 100644 src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.h create mode 100644 tests/integration/test_attach_partition_distinct_expression_replicated/__init__.py create mode 100644 tests/integration/test_attach_partition_distinct_expression_replicated/configs/remote_servers.xml create mode 100644 tests/integration/test_attach_partition_distinct_expression_replicated/test.py create mode 100644 tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.reference create mode 100644 tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.sql diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 114b8d5ffe3..5659a0565c5 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -112,7 +112,7 @@ Note that: For the query to run successfully, the following conditions must be met: - Both tables must have the same structure. -- Both tables must have the same partition key, the same order by key and the same primary key. +- Both tables must have the same order by key and the same primary key. - Both tables must have the same indices and projections. - Both tables must have the same storage policy. diff --git a/src/Interpreters/MonotonicityCheckVisitor.h b/src/Interpreters/MonotonicityCheckVisitor.h index cc386825024..4e71bd56851 100644 --- a/src/Interpreters/MonotonicityCheckVisitor.h +++ b/src/Interpreters/MonotonicityCheckVisitor.h @@ -1,13 +1,17 @@ #pragma once #include +#include #include +#include #include #include -#include #include +#include +#include #include #include +#include #include #include #include @@ -33,6 +37,8 @@ public: ASTIdentifier * identifier = nullptr; DataTypePtr arg_data_type = {}; + Range range = Range::createWholeUniverse(); + void reject() { monotonicity.is_monotonic = false; } bool isRejected() const { return !monotonicity.is_monotonic; } @@ -97,13 +103,30 @@ public: if (data.isRejected()) return; - /// TODO: monotonicity for functions of several arguments - if (!ast_function.arguments || ast_function.arguments->children.size() != 1) + /// Monotonicity check only works for functions that contain at most two arguments and one of them must be a constant. + if (!ast_function.arguments) { data.reject(); return; } + auto arguments_size = ast_function.arguments->children.size(); + + if (arguments_size == 0 || arguments_size > 2) + { + data.reject(); + return; + } + else if (arguments_size == 2) + { + /// If the function has two arguments, then one of them must be a constant. + if (!ast_function.arguments->children[0]->as() && !ast_function.arguments->children[1]->as()) + { + data.reject(); + return; + } + } + if (!data.canOptimize(ast_function)) { data.reject(); @@ -124,14 +147,33 @@ public: return; } - ColumnsWithTypeAndName args; - args.emplace_back(data.arg_data_type, "tmp"); - auto function_base = function->build(args); + auto function_arguments = getFunctionArguments(ast_function, data); + + auto function_base = function->build(function_arguments); if (function_base && function_base->hasInformationAboutMonotonicity()) { bool is_positive = data.monotonicity.is_positive; - data.monotonicity = function_base->getMonotonicityForRange(*data.arg_data_type, Field(), Field()); + data.monotonicity = function_base->getMonotonicityForRange(*data.arg_data_type, data.range.left, data.range.right); + + auto & key_range = data.range; + + /// If we apply function to open interval, we can get empty intervals in result. + /// E.g. for ('2020-01-03', '2020-01-20') after applying 'toYYYYMM' we will get ('202001', '202001'). + /// To avoid this we make range left and right included. + /// Any function that treats NULL specially is not monotonic. + /// Thus we can safely use isNull() as an -Inf/+Inf indicator here. + if (!key_range.left.isNull()) + { + key_range.left = applyFunction(function_base, data.arg_data_type, key_range.left); + key_range.left_included = true; + } + + if (!key_range.right.isNull()) + { + key_range.right = applyFunction(function_base, data.arg_data_type, key_range.right); + key_range.right_included = true; + } if (!is_positive) data.monotonicity.is_positive = !data.monotonicity.is_positive; @@ -143,13 +185,53 @@ public: static bool needChildVisit(const ASTPtr & parent, const ASTPtr &) { - /// Currently we check monotonicity only for single-argument functions. - /// Although, multi-argument functions with all but one constant arguments can also be monotonic. + /// Multi-argument functions with all but one constant arguments can be monotonic. if (const auto * func = typeid_cast(parent.get())) - return func->arguments->children.size() < 2; + return func->arguments->children.size() <= 2; return true; } + + static ColumnWithTypeAndName extractLiteralColumnAndTypeFromAstLiteral(const ASTLiteral * literal) + { + ColumnWithTypeAndName result; + + result.type = applyVisitor(FieldToDataType(), literal->value); + result.column = result.type->createColumnConst(0, literal->value); + + return result; + } + + static ColumnsWithTypeAndName getFunctionArguments(const ASTFunction & ast_function, const Data & data) + { + ColumnsWithTypeAndName args; + + auto arguments_size = ast_function.arguments->children.size(); + + chassert(arguments_size == 1 || arguments_size == 2); + + if (arguments_size == 2) + { + if (ast_function.arguments->children[0]->as()) + { + const auto * literal = ast_function.arguments->children[0]->as(); + args.push_back(extractLiteralColumnAndTypeFromAstLiteral(literal)); + args.emplace_back(data.arg_data_type, "tmp"); + } + else + { + const auto * literal = ast_function.arguments->children[1]->as(); + args.emplace_back(data.arg_data_type, "tmp"); + args.push_back(extractLiteralColumnAndTypeFromAstLiteral(literal)); + } + } + else + { + args.emplace_back(data.arg_data_type, "tmp"); + } + + return args; + } }; using MonotonicityCheckVisitor = ConstInDepthNodeVisitor; diff --git a/src/Interpreters/applyFunction.cpp b/src/Interpreters/applyFunction.cpp new file mode 100644 index 00000000000..a53f14f0381 --- /dev/null +++ b/src/Interpreters/applyFunction.cpp @@ -0,0 +1,43 @@ +#include + +#include +#include + +namespace DB +{ + +static Field applyFunctionForField(const FunctionBasePtr & func, const DataTypePtr & arg_type, const Field & arg_value) +{ + ColumnsWithTypeAndName columns{ + {arg_type->createColumnConst(1, arg_value), arg_type, "x"}, + }; + + auto col = func->execute(columns, func->getResultType(), 1); + return (*col)[0]; +} + +FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field) +{ + /// Fallback for fields without block reference. + if (field.isExplicit()) + return applyFunctionForField(func, current_type, field); + + String result_name = "_" + func->getName() + "_" + toString(field.column_idx); + const auto & columns = field.columns; + size_t result_idx = columns->size(); + + for (size_t i = 0; i < result_idx; ++i) + if ((*columns)[i].name == result_name) + result_idx = i; + + if (result_idx == columns->size()) + { + ColumnsWithTypeAndName args{(*columns)[field.column_idx]}; + field.columns->emplace_back(ColumnWithTypeAndName{nullptr, func->getResultType(), result_name}); + (*columns)[result_idx].column = func->execute(args, (*columns)[result_idx].type, columns->front().column->size()); + } + + return {field.columns, field.row_idx, result_idx}; +} + +} diff --git a/src/Interpreters/applyFunction.h b/src/Interpreters/applyFunction.h new file mode 100644 index 00000000000..9b8ae43a53c --- /dev/null +++ b/src/Interpreters/applyFunction.h @@ -0,0 +1,16 @@ +#pragma once + +#include + +namespace DB +{ +struct FieldRef; + +class IFunctionBase; +class IDataType; + +using DataTypePtr = std::shared_ptr; +using FunctionBasePtr = std::shared_ptr; + +FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field); +} diff --git a/src/Parsers/queryToString.cpp b/src/Parsers/queryToString.cpp index 9721aa1f128..4a1903393f6 100644 --- a/src/Parsers/queryToString.cpp +++ b/src/Parsers/queryToString.cpp @@ -3,6 +3,11 @@ namespace DB { + String queryToStringNullable(const ASTPtr & query) + { + return query ? queryToString(query) : ""; + } + String queryToString(const ASTPtr & query) { return queryToString(*query); diff --git a/src/Parsers/queryToString.h b/src/Parsers/queryToString.h index 873de218293..3acd560b1e2 100644 --- a/src/Parsers/queryToString.h +++ b/src/Parsers/queryToString.h @@ -6,4 +6,5 @@ namespace DB { String queryToString(const ASTPtr & query); String queryToString(const IAST & query); + String queryToStringNullable(const ASTPtr & query); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 87f23b0da2a..f3057a8254f 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -81,6 +81,7 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Par auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key); size_t minmax_idx_size = minmax_column_types.size(); + hyperrectangle.clear(); hyperrectangle.reserve(minmax_idx_size); for (size_t i = 0; i < minmax_idx_size; ++i) { @@ -104,6 +105,39 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Par initialized = true; } +Block IMergeTreeDataPart::MinMaxIndex::getBlock(const MergeTreeData & data) const +{ + if (!initialized) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to get block from uninitialized MinMax index."); + + Block block; + + const auto metadata_snapshot = data.getInMemoryMetadataPtr(); + const auto & partition_key = metadata_snapshot->getPartitionKey(); + + const auto minmax_column_names = data.getMinMaxColumnsNames(partition_key); + const auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key); + const auto minmax_idx_size = minmax_column_types.size(); + + for (size_t i = 0; i < minmax_idx_size; ++i) + { + const auto & data_type = minmax_column_types[i]; + const auto & column_name = minmax_column_names[i]; + + const auto column = data_type->createColumn(); + + const auto min_val = hyperrectangle.at(i).left; + const auto max_val = hyperrectangle.at(i).right; + + column->insert(min_val); + column->insert(max_val); + + block.insert(ColumnWithTypeAndName(column->getPtr(), data_type, column_name)); + } + + return block; +} + IMergeTreeDataPart::MinMaxIndex::WrittenFiles IMergeTreeDataPart::MinMaxIndex::store( const MergeTreeData & data, IDataPartStorage & part_storage, Checksums & out_checksums) const { @@ -185,8 +219,7 @@ void IMergeTreeDataPart::MinMaxIndex::merge(const MinMaxIndex & other) if (!initialized) { - hyperrectangle = other.hyperrectangle; - initialized = true; + *this = other; } else { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 640a1f1d0a3..29f0f54d419 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -336,6 +336,7 @@ public: } void load(const MergeTreeData & data, const PartMetadataManagerPtr & manager); + Block getBlock(const MergeTreeData & data) const; using WrittenFiles = std::vector>; diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index d5922ae1bc2..e5bcb11091f 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1,36 +1,37 @@ -#include -#include -#include +#include +#include #include #include #include #include +#include #include -#include #include -#include -#include -#include -#include -#include -#include -#include +#include #include +#include #include -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include +#include #include -#include +#include +#include +#include +#include +#include #include #include #include -#include -#include +#include +#include +#include #include +#include +#include +#include #include #include @@ -836,21 +837,6 @@ bool KeyCondition::getConstant(const ASTPtr & expr, Block & block_with_constants return node.tryGetConstant(out_value, out_type); } - -static Field applyFunctionForField( - const FunctionBasePtr & func, - const DataTypePtr & arg_type, - const Field & arg_value) -{ - ColumnsWithTypeAndName columns - { - { arg_type->createColumnConst(1, arg_value), arg_type, "x" }, - }; - - auto col = func->execute(columns, func->getResultType(), 1); - return (*col)[0]; -} - /// The case when arguments may have types different than in the primary key. static std::pair applyFunctionForFieldOfUnknownType( const FunctionBasePtr & func, @@ -890,33 +876,6 @@ static std::pair applyBinaryFunctionForFieldOfUnknownType( return {std::move(result), std::move(return_type)}; } - -static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field) -{ - /// Fallback for fields without block reference. - if (field.isExplicit()) - return applyFunctionForField(func, current_type, field); - - String result_name = "_" + func->getName() + "_" + toString(field.column_idx); - const auto & columns = field.columns; - size_t result_idx = columns->size(); - - for (size_t i = 0; i < result_idx; ++i) - { - if ((*columns)[i].name == result_name) - result_idx = i; - } - - if (result_idx == columns->size()) - { - ColumnsWithTypeAndName args{(*columns)[field.column_idx]}; - field.columns->emplace_back(ColumnWithTypeAndName {nullptr, func->getResultType(), result_name}); - (*columns)[result_idx].column = func->execute(args, (*columns)[result_idx].type, columns->front().column->size()); - } - - return {field.columns, field.row_idx, result_idx}; -} - /** When table's key has expression with these functions from a column, * and when a column in a query is compared with a constant, such as: * CREATE TABLE (x String) ORDER BY toDate(x) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 61332a4ff38..c3e348a549a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8,21 +8,6 @@ #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include #include @@ -43,19 +28,20 @@ #include #include #include -#include -#include #include #include #include #include #include #include +#include +#include #include +#include #include -#include -#include #include +#include +#include #include #include #include @@ -64,26 +50,41 @@ #include #include #include -#include #include #include #include #include #include #include +#include #include #include +#include #include #include #include #include #include +#include #include #include -#include #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include @@ -197,6 +198,50 @@ namespace ErrorCodes extern const int LIMIT_EXCEEDED; } +static size_t getPartitionAstFieldsCount(const ASTPartition & partition_ast, ASTPtr partition_value_ast) +{ + if (partition_ast.fields_count.has_value()) + return *partition_ast.fields_count; + + if (partition_value_ast->as()) + return 1; + + const auto * tuple_ast = partition_value_ast->as(); + + if (!tuple_ast) + { + throw Exception( + ErrorCodes::INVALID_PARTITION_VALUE, "Expected literal or tuple for partition key, got {}", partition_value_ast->getID()); + } + + if (tuple_ast->name != "tuple") + { + if (!isFunctionCast(tuple_ast)) + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); + + if (tuple_ast->arguments->as()->children.empty()) + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); + + auto first_arg = tuple_ast->arguments->as()->children.at(0); + if (const auto * inner_tuple = first_arg->as(); inner_tuple && inner_tuple->name == "tuple") + { + const auto * arguments_ast = tuple_ast->arguments->as(); + return arguments_ast ? arguments_ast->children.size() : 0; + } + else if (const auto * inner_literal_tuple = first_arg->as(); inner_literal_tuple) + { + return inner_literal_tuple->value.getType() == Field::Types::Tuple ? inner_literal_tuple->value.safeGet().size() : 1; + } + + throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); + } + else + { + const auto * arguments_ast = tuple_ast->arguments->as(); + return arguments_ast ? arguments_ast->children.size() : 0; + } +} + static void checkSuspiciousIndices(const ASTFunction * index_function) { std::unordered_set unique_index_expression_hashes; @@ -4854,7 +4899,7 @@ void MergeTreeData::removePartContributionToColumnAndSecondaryIndexSizes(const D } void MergeTreeData::checkAlterPartitionIsPossible( - const PartitionCommands & commands, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & settings, ContextPtr local_context) const + const PartitionCommands & commands, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & settings, ContextPtr) const { for (const auto & command : commands) { @@ -4882,7 +4927,15 @@ void MergeTreeData::checkAlterPartitionIsPossible( throw DB::Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Only support DROP/DETACH PARTITION ALL currently"); } else - getPartitionIDFromQuery(command.partition, local_context); + { + // The below `getPartitionIDFromQuery` call will not work for attach / replace because it assumes the partition expressions + // are the same and deliberately uses this storage. Later on, `MergeTreeData::replaceFrom` is called, and it makes the right + // call to `getPartitionIDFromQuery` using source storage. + // Note: `PartitionCommand::REPLACE_PARTITION` is used both for `REPLACE PARTITION` and `ATTACH PARTITION FROM` queries. + // But not for `ATTACH PARTITION` queries. + if (command.type != PartitionCommand::REPLACE_PARTITION) + getPartitionIDFromQuery(command.partition, getContext()); + } } } } @@ -5616,69 +5669,8 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc MergeTreePartInfo::validatePartitionID(partition_ast.id->clone(), format_version); return partition_ast.id->as()->value.safeGet(); } - size_t partition_ast_fields_count = 0; ASTPtr partition_value_ast = partition_ast.value->clone(); - if (!partition_ast.fields_count.has_value()) - { - if (partition_value_ast->as()) - { - partition_ast_fields_count = 1; - } - else if (const auto * tuple_ast = partition_value_ast->as()) - { - if (tuple_ast->name != "tuple") - { - if (isFunctionCast(tuple_ast)) - { - if (tuple_ast->arguments->as()->children.empty()) - { - throw Exception( - ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); - } - auto first_arg = tuple_ast->arguments->as()->children.at(0); - if (const auto * inner_tuple = first_arg->as(); inner_tuple && inner_tuple->name == "tuple") - { - const auto * arguments_ast = tuple_ast->arguments->as(); - if (arguments_ast) - partition_ast_fields_count = arguments_ast->children.size(); - else - partition_ast_fields_count = 0; - } - else if (const auto * inner_literal_tuple = first_arg->as(); inner_literal_tuple) - { - if (inner_literal_tuple->value.getType() == Field::Types::Tuple) - partition_ast_fields_count = inner_literal_tuple->value.safeGet().size(); - else - partition_ast_fields_count = 1; - } - else - { - throw Exception( - ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); - } - } - else - throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, "Expected tuple for complex partition key, got {}", tuple_ast->name); - } - else - { - const auto * arguments_ast = tuple_ast->arguments->as(); - if (arguments_ast) - partition_ast_fields_count = arguments_ast->children.size(); - else - partition_ast_fields_count = 0; - } - } - else - { - throw Exception( - ErrorCodes::INVALID_PARTITION_VALUE, "Expected literal or tuple for partition key, got {}", partition_value_ast->getID()); - } - } - else - { - partition_ast_fields_count = *partition_ast.fields_count; - } + auto partition_ast_fields_count = getPartitionAstFieldsCount(partition_ast, partition_value_ast); if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { @@ -7014,23 +7006,35 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & sour if (my_snapshot->getColumns().getAllPhysical().sizeOfDifference(src_snapshot->getColumns().getAllPhysical())) throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Tables have different structure"); - auto query_to_string = [] (const ASTPtr & ast) - { - return ast ? queryToString(ast) : ""; - }; - - if (query_to_string(my_snapshot->getSortingKeyAST()) != query_to_string(src_snapshot->getSortingKeyAST())) + if (queryToStringNullable(my_snapshot->getSortingKeyAST()) != queryToStringNullable(src_snapshot->getSortingKeyAST())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different ordering"); - if (query_to_string(my_snapshot->getPartitionKeyAST()) != query_to_string(src_snapshot->getPartitionKeyAST())) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key"); - if (format_version != src_data->format_version) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different format_version"); - if (query_to_string(my_snapshot->getPrimaryKeyAST()) != query_to_string(src_snapshot->getPrimaryKeyAST())) + if (queryToStringNullable(my_snapshot->getPrimaryKeyAST()) != queryToStringNullable(src_snapshot->getPrimaryKeyAST())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different primary key"); + const auto is_a_subset_of = [](const auto & lhs, const auto & rhs) + { + if (lhs.size() > rhs.size()) + return false; + + const auto rhs_set = NameSet(rhs.begin(), rhs.end()); + for (const auto & lhs_element : lhs) + if (!rhs_set.contains(lhs_element)) + return false; + + return true; + }; + + if (!is_a_subset_of(my_snapshot->getColumnsRequiredForPartitionKey(), src_snapshot->getColumnsRequiredForPartitionKey())) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Destination table partition expression columns must be a subset of source table partition expression columns"); + } + const auto check_definitions = [](const auto & my_descriptions, const auto & src_descriptions) { if (my_descriptions.size() != src_descriptions.size()) @@ -7071,128 +7075,56 @@ std::pair MergeTreeData::cloneAn const ReadSettings & read_settings, const WriteSettings & write_settings) { - /// Check that the storage policy contains the disk where the src_part is located. - bool does_storage_policy_allow_same_disk = false; - for (const DiskPtr & disk : getStoragePolicy()->getDisks()) - { - if (disk->getName() == src_part->getDataPartStorage().getDiskName()) - { - does_storage_policy_allow_same_disk = true; - break; - } - } - if (!does_storage_policy_allow_same_disk) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Could not clone and load part {} because disk does not belong to storage policy", - quoteString(src_part->getDataPartStorage().getFullPath())); + return MergeTreeDataPartCloner::clone( + this, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, require_part_metadata, params, read_settings, write_settings); +} - String dst_part_name = src_part->getNewName(dst_part_info); - String tmp_dst_part_name = tmp_part_prefix + dst_part_name; - auto temporary_directory_lock = getTemporaryPartDirectoryHolder(tmp_dst_part_name); +std::pair MergeTreeData::cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( + const MergeTreeData::DataPartPtr & src_part, + const MergeTreePartition & new_partition, + const String & partition_id, + const IMergeTreeDataPart::MinMaxIndex & min_max_index, + const String & tmp_part_prefix, + const StorageMetadataPtr & my_metadata_snapshot, + const IDataPartStorage::ClonePartParams & clone_params, + ContextPtr local_context, + Int64 min_block, + Int64 max_block +) +{ + MergeTreePartInfo dst_part_info(partition_id, min_block, max_block, src_part->info.level); - /// Why it is needed if we only hardlink files? - auto reservation = src_part->getDataPartStorage().reserve(src_part->getBytesOnDisk()); - auto src_part_storage = src_part->getDataPartStoragePtr(); + return MergeTreeDataPartCloner::cloneWithDistinctPartitionExpression( + this, + src_part, + my_metadata_snapshot, + dst_part_info, + tmp_part_prefix, + local_context->getReadSettings(), + local_context->getWriteSettings(), + new_partition, + min_max_index, + false, + clone_params); +} - scope_guard src_flushed_tmp_dir_lock; - MergeTreeData::MutableDataPartPtr src_flushed_tmp_part; +std::pair MergeTreeData::createPartitionAndMinMaxIndexFromSourcePart( + const MergeTreeData::DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr local_context) +{ + const auto & src_data = src_part->storage; - /// If source part is in memory, flush it to disk and clone it already in on-disk format - /// Protect tmp dir from removing by cleanup thread with src_flushed_tmp_dir_lock - /// Construct src_flushed_tmp_part in order to delete part with its directory at destructor - if (auto src_part_in_memory = asInMemoryPart(src_part)) - { - auto flushed_part_path = *src_part_in_memory->getRelativePathForPrefix(tmp_part_prefix); + auto metadata_manager = std::make_shared(src_part.get()); + IMergeTreeDataPart::MinMaxIndex min_max_index; - auto tmp_src_part_file_name = fs::path(tmp_dst_part_name).filename(); - src_flushed_tmp_dir_lock = src_part->storage.getTemporaryPartDirectoryHolder(tmp_src_part_file_name); + min_max_index.load(src_data, metadata_manager); - auto flushed_part_storage = src_part_in_memory->flushToDisk(flushed_part_path, metadata_snapshot); + MergeTreePartition new_partition; - src_flushed_tmp_part = MergeTreeDataPartBuilder(*this, src_part->name, flushed_part_storage) - .withPartInfo(src_part->info) - .withPartFormatFromDisk() - .build(); + new_partition.create(metadata_snapshot, min_max_index.getBlock(src_data), 0u, local_context); - src_flushed_tmp_part->is_temp = true; - src_part_storage = flushed_part_storage; - } - - String with_copy; - if (params.copy_instead_of_hardlink) - with_copy = " (copying data)"; - - auto dst_part_storage = src_part_storage->freeze( - relative_data_path, - tmp_dst_part_name, - read_settings, - write_settings, - /* save_metadata_callback= */ {}, - params); - - if (params.metadata_version_to_write.has_value()) - { - chassert(!params.keep_metadata_version); - auto out_metadata = dst_part_storage->writeFile(IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, getContext()->getWriteSettings()); - writeText(metadata_snapshot->getMetadataVersion(), *out_metadata); - out_metadata->finalize(); - if (getSettings()->fsync_after_insert) - out_metadata->sync(); - } - - LOG_DEBUG(log, "Clone{} part {} to {}{}", - src_flushed_tmp_part ? " flushed" : "", - src_part_storage->getFullPath(), - std::string(fs::path(dst_part_storage->getFullRootPath()) / tmp_dst_part_name), - with_copy); - - auto dst_data_part = MergeTreeDataPartBuilder(*this, dst_part_name, dst_part_storage) - .withPartFormatFromDisk() - .build(); - - if (!params.copy_instead_of_hardlink && params.hardlinked_files) - { - params.hardlinked_files->source_part_name = src_part->name; - params.hardlinked_files->source_table_shared_id = src_part->storage.getTableSharedID(); - - for (auto it = src_part->getDataPartStorage().iterate(); it->isValid(); it->next()) - { - if (!params.files_to_copy_instead_of_hardlinks.contains(it->name()) - && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED - && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) - { - params.hardlinked_files->hardlinks_from_source_part.insert(it->name()); - } - } - - auto projections = src_part->getProjectionParts(); - for (const auto & [name, projection_part] : projections) - { - const auto & projection_storage = projection_part->getDataPartStorage(); - for (auto it = projection_storage.iterate(); it->isValid(); it->next()) - { - auto file_name_with_projection_prefix = fs::path(projection_storage.getPartDirectory()) / it->name(); - if (!params.files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix) - && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED - && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) - { - params.hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix); - } - } - } - } - - /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. - TransactionID tid = params.txn ? params.txn->tid : Tx::PrehistoricTID; - dst_data_part->version.setCreationTID(tid, nullptr); - dst_data_part->storeVersionMetadata(); - - dst_data_part->is_temp = true; - - dst_data_part->loadColumnsChecksumsIndexes(require_part_metadata, true); - dst_data_part->modification_time = dst_part_storage->getLastModified().epochTime(); - return std::make_pair(dst_data_part, std::move(temporary_directory_lock)); + return {new_partition, min_max_index}; } String MergeTreeData::getFullPathOnDisk(const DiskPtr & disk) const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index f0dbaf0e307..9c433e11b84 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -231,6 +231,7 @@ public: } }; + using DataParts = std::set; using MutableDataParts = std::set; using DataPartsVector = std::vector; @@ -848,6 +849,23 @@ public: const ReadSettings & read_settings, const WriteSettings & write_settings); + std::pair cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( + const MergeTreeData::DataPartPtr & src_part, + const MergeTreePartition & new_partition, + const String & partition_id, + const IMergeTreeDataPart::MinMaxIndex & min_max_index, + const String & tmp_part_prefix, + const StorageMetadataPtr & my_metadata_snapshot, + const IDataPartStorage::ClonePartParams & clone_params, + ContextPtr local_context, + Int64 min_block, + Int64 max_block); + + static std::pair createPartitionAndMinMaxIndexFromSourcePart( + const MergeTreeData::DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr local_context); + virtual std::vector getMutationsStatus() const = 0; /// Returns true if table can create new parts with adaptive granularity diff --git a/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp b/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp new file mode 100644 index 00000000000..78cb9aa0624 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp @@ -0,0 +1,320 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} + +static Poco::Logger * log = &Poco::Logger::get("MergeTreeDataPartCloner"); + +namespace DistinctPartitionExpression +{ +std::unique_ptr updatePartitionFile( + const MergeTreeData & merge_tree_data, + const MergeTreePartition & partition, + const MergeTreeData::MutableDataPartPtr & dst_part, + IDataPartStorage & storage) +{ + storage.removeFile("partition.dat"); + // Leverage already implemented MergeTreePartition::store to create & store partition.dat. + // Checksum is re-calculated later. + return partition.store(merge_tree_data, storage, dst_part->checksums); +} + +IMergeTreeDataPart::MinMaxIndex::WrittenFiles updateMinMaxFiles( + const MergeTreeData & merge_tree_data, + const MergeTreeData::MutableDataPartPtr & dst_part, + IDataPartStorage & storage, + const StorageMetadataPtr & metadata_snapshot) +{ + for (const auto & column_name : MergeTreeData::getMinMaxColumnsNames(metadata_snapshot->partition_key)) + { + auto file = "minmax_" + escapeForFileName(column_name) + ".idx"; + storage.removeFile(file); + } + + return dst_part->minmax_idx->store(merge_tree_data, storage, dst_part->checksums); +} + +void finalizeNewFiles(const std::vector> & files, bool sync_new_files) +{ + for (const auto & file : files) + { + file->finalize(); + if (sync_new_files) + file->sync(); + } +} + +void updateNewPartFiles( + const MergeTreeData & merge_tree_data, + const MergeTreeData::MutableDataPartPtr & dst_part, + const MergeTreePartition & new_partition, + const IMergeTreeDataPart::MinMaxIndex & new_min_max_index, + const StorageMetadataPtr & src_metadata_snapshot, + bool sync_new_files) +{ + auto & storage = dst_part->getDataPartStorage(); + + *dst_part->minmax_idx = new_min_max_index; + + auto partition_file = updatePartitionFile(merge_tree_data, new_partition, dst_part, storage); + + auto min_max_files = updateMinMaxFiles(merge_tree_data, dst_part, storage, src_metadata_snapshot); + + IMergeTreeDataPart::MinMaxIndex::WrittenFiles written_files; + + if (partition_file) + written_files.emplace_back(std::move(partition_file)); + + written_files.insert(written_files.end(), std::make_move_iterator(min_max_files.begin()), std::make_move_iterator(min_max_files.end())); + + finalizeNewFiles(written_files, sync_new_files); + + // MergeTreeDataPartCloner::finalize_part calls IMergeTreeDataPart::loadColumnsChecksumsIndexes, which will re-create + // the checksum file if it doesn't exist. Relying on that is cumbersome, but this refactoring is simply a code extraction + // with small improvements. It can be further improved in the future. + storage.removeFile("checksums.txt"); +} +} + +namespace +{ +bool doesStoragePolicyAllowSameDisk(MergeTreeData * merge_tree_data, const MergeTreeData::DataPartPtr & src_part) +{ + for (const DiskPtr & disk : merge_tree_data->getStoragePolicy()->getDisks()) + if (disk->getName() == src_part->getDataPartStorage().getDiskName()) + return true; + return false; +} + +DataPartStoragePtr flushPartStorageToDiskIfInMemory( + MergeTreeData * merge_tree_data, + const MergeTreeData::DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + const String & tmp_part_prefix, + const String & tmp_dst_part_name, + scope_guard & src_flushed_tmp_dir_lock, + MergeTreeData::MutableDataPartPtr src_flushed_tmp_part) +{ + if (auto src_part_in_memory = asInMemoryPart(src_part)) + { + auto flushed_part_path = src_part_in_memory->getRelativePathForPrefix(tmp_part_prefix); + auto tmp_src_part_file_name = fs::path(tmp_dst_part_name).filename(); + + src_flushed_tmp_dir_lock = src_part->storage.getTemporaryPartDirectoryHolder(tmp_src_part_file_name); + + auto flushed_part_storage = src_part_in_memory->flushToDisk(*flushed_part_path, metadata_snapshot); + + src_flushed_tmp_part = MergeTreeDataPartBuilder(*merge_tree_data, src_part->name, flushed_part_storage) + .withPartInfo(src_part->info) + .withPartFormatFromDisk() + .build(); + + src_flushed_tmp_part->is_temp = true; + + return flushed_part_storage; + } + + return src_part->getDataPartStoragePtr(); +} + +std::shared_ptr hardlinkAllFiles( + MergeTreeData * merge_tree_data, + const DB::ReadSettings & read_settings, + const DB::WriteSettings & write_settings, + const DataPartStoragePtr & storage, + const String & path, + const DB::IDataPartStorage::ClonePartParams & params) +{ + return storage->freeze( + merge_tree_data->getRelativeDataPath(), + path, + read_settings, + write_settings, + /*save_metadata_callback=*/{}, + params); +} + +std::pair cloneSourcePart( + MergeTreeData * merge_tree_data, + const MergeTreeData::DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + const MergeTreePartInfo & dst_part_info, + const String & tmp_part_prefix, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const DB::IDataPartStorage::ClonePartParams & params) +{ + const auto dst_part_name = src_part->getNewName(dst_part_info); + + const auto tmp_dst_part_name = tmp_part_prefix + dst_part_name; + + auto temporary_directory_lock = merge_tree_data->getTemporaryPartDirectoryHolder(tmp_dst_part_name); + + src_part->getDataPartStorage().reserve(src_part->getBytesOnDisk()); + + scope_guard src_flushed_tmp_dir_lock; + MergeTreeData::MutableDataPartPtr src_flushed_tmp_part; + + auto src_part_storage = flushPartStorageToDiskIfInMemory( + merge_tree_data, src_part, metadata_snapshot, tmp_part_prefix, tmp_dst_part_name, src_flushed_tmp_dir_lock, src_flushed_tmp_part); + + auto dst_part_storage = hardlinkAllFiles(merge_tree_data, read_settings, write_settings, src_part_storage, tmp_dst_part_name, params); + + if (params.metadata_version_to_write.has_value()) + { + chassert(!params.keep_metadata_version); + auto out_metadata = dst_part_storage->writeFile( + IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, merge_tree_data->getContext()->getWriteSettings()); + writeText(metadata_snapshot->getMetadataVersion(), *out_metadata); + out_metadata->finalize(); + if (merge_tree_data->getSettings()->fsync_after_insert) + out_metadata->sync(); + } + + LOG_DEBUG( + log, + "Clone {} part {} to {}{}", + src_flushed_tmp_part ? "flushed" : "", + src_part_storage->getFullPath(), + std::string(fs::path(dst_part_storage->getFullRootPath()) / tmp_dst_part_name), + false); + + + auto part = MergeTreeDataPartBuilder(*merge_tree_data, dst_part_name, dst_part_storage).withPartFormatFromDisk().build(); + + return std::make_pair(part, std::move(temporary_directory_lock)); +} + +void handleHardLinkedParameterFiles(const MergeTreeData::DataPartPtr & src_part, const DB::IDataPartStorage::ClonePartParams & params) +{ + const auto & hardlinked_files = params.hardlinked_files; + + hardlinked_files->source_part_name = src_part->name; + hardlinked_files->source_table_shared_id = src_part->storage.getTableSharedID(); + + for (auto it = src_part->getDataPartStorage().iterate(); it->isValid(); it->next()) + { + if (!params.files_to_copy_instead_of_hardlinks.contains(it->name()) + && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED + && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) + { + hardlinked_files->hardlinks_from_source_part.insert(it->name()); + } + } +} + +void handleProjections(const MergeTreeData::DataPartPtr & src_part, const DB::IDataPartStorage::ClonePartParams & params) +{ + auto projections = src_part->getProjectionParts(); + for (const auto & [name, projection_part] : projections) + { + const auto & projection_storage = projection_part->getDataPartStorage(); + for (auto it = projection_storage.iterate(); it->isValid(); it->next()) + { + auto file_name_with_projection_prefix = fs::path(projection_storage.getPartDirectory()) / it->name(); + if (!params.files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix) + && it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED + && it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME) + { + params.hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix); + } + } + } +} + +MergeTreeData::MutableDataPartPtr finalizePart( + const MergeTreeData::MutableDataPartPtr & dst_part, const DB::IDataPartStorage::ClonePartParams & params, bool require_part_metadata) +{ + /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. + TransactionID tid = params.txn ? params.txn->tid : Tx::PrehistoricTID; + dst_part->version.setCreationTID(tid, nullptr); + dst_part->storeVersionMetadata(); + + dst_part->is_temp = true; + + dst_part->loadColumnsChecksumsIndexes(require_part_metadata, true); + + dst_part->modification_time = dst_part->getDataPartStorage().getLastModified().epochTime(); + + return dst_part; +} + +std::pair cloneAndHandleHardlinksAndProjections( + MergeTreeData * merge_tree_data, + const DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + const MergeTreePartInfo & dst_part_info, + const String & tmp_part_prefix, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const IDataPartStorage::ClonePartParams & params) +{ + if (!doesStoragePolicyAllowSameDisk(merge_tree_data, src_part)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Could not clone and load part {} because disk does not belong to storage policy", + quoteString(src_part->getDataPartStorage().getFullPath())); + + auto [destination_part, temporary_directory_lock] = cloneSourcePart( + merge_tree_data, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, read_settings, write_settings, params); + + if (!params.copy_instead_of_hardlink && params.hardlinked_files) + { + handleHardLinkedParameterFiles(src_part, params); + handleProjections(src_part, params); + } + + return std::make_pair(destination_part, std::move(temporary_directory_lock)); +} +} + +std::pair MergeTreeDataPartCloner::clone( + MergeTreeData * merge_tree_data, + const DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + const MergeTreePartInfo & dst_part_info, + const String & tmp_part_prefix, + bool require_part_metadata, + const IDataPartStorage::ClonePartParams & params, + const ReadSettings & read_settings, + const WriteSettings & write_settings) +{ + auto [destination_part, temporary_directory_lock] = cloneAndHandleHardlinksAndProjections( + merge_tree_data, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, read_settings, write_settings, params); + + return std::make_pair(finalizePart(destination_part, params, require_part_metadata), std::move(temporary_directory_lock)); +} + +std::pair MergeTreeDataPartCloner::cloneWithDistinctPartitionExpression( + MergeTreeData * merge_tree_data, + const DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + const MergeTreePartInfo & dst_part_info, + const String & tmp_part_prefix, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const MergeTreePartition & new_partition, + const IMergeTreeDataPart::MinMaxIndex & new_min_max_index, + bool sync_new_files, + const IDataPartStorage::ClonePartParams & params) +{ + auto [destination_part, temporary_directory_lock] = cloneAndHandleHardlinksAndProjections( + merge_tree_data, src_part, metadata_snapshot, dst_part_info, tmp_part_prefix, read_settings, write_settings, params); + + DistinctPartitionExpression::updateNewPartFiles( + *merge_tree_data, destination_part, new_partition, new_min_max_index, src_part->storage.getInMemoryMetadataPtr(), sync_new_files); + + return std::make_pair(finalizePart(destination_part, params, false), std::move(temporary_directory_lock)); +} + +} diff --git a/src/Storages/MergeTree/MergeTreeDataPartCloner.h b/src/Storages/MergeTree/MergeTreeDataPartCloner.h new file mode 100644 index 00000000000..53585f20b7f --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeDataPartCloner.h @@ -0,0 +1,43 @@ +#pragma once + +namespace DB +{ + +struct StorageInMemoryMetadata; +using StorageMetadataPtr = std::shared_ptr; +struct MergeTreePartition; +class IMergeTreeDataPart; + +class MergeTreeDataPartCloner +{ +public: + using DataPart = IMergeTreeDataPart; + using MutableDataPartPtr = std::shared_ptr; + using DataPartPtr = std::shared_ptr; + + static std::pair clone( + MergeTreeData * merge_tree_data, + const DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + const MergeTreePartInfo & dst_part_info, + const String & tmp_part_prefix, + bool require_part_metadata, + const IDataPartStorage::ClonePartParams & params, + const ReadSettings & read_settings, + const WriteSettings & write_settings); + + static std::pair cloneWithDistinctPartitionExpression( + MergeTreeData * merge_tree_data, + const DataPartPtr & src_part, + const StorageMetadataPtr & metadata_snapshot, + const MergeTreePartInfo & dst_part_info, + const String & tmp_part_prefix, + const ReadSettings & read_settings, + const WriteSettings & write_settings, + const MergeTreePartition & new_partition, + const IMergeTreeDataPart::MinMaxIndex & new_min_max_index, + bool sync_new_files, + const IDataPartStorage::ClonePartParams & params); +}; + +} diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index ddeaf69136a..76ef3be25b3 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -467,6 +467,45 @@ void MergeTreePartition::create(const StorageMetadataPtr & metadata_snapshot, Bl } } +void MergeTreePartition::createAndValidateMinMaxPartitionIds( + const StorageMetadataPtr & metadata_snapshot, Block block_with_min_max_partition_ids, ContextPtr context) +{ + if (!metadata_snapshot->hasPartitionKey()) + return; + + auto partition_key_names_and_types = executePartitionByExpression(metadata_snapshot, block_with_min_max_partition_ids, context); + value.resize(partition_key_names_and_types.size()); + + /// Executing partition_by expression adds new columns to passed block according to partition functions. + /// The block is passed by reference and is used afterwards. `moduloLegacy` needs to be substituted back + /// with just `modulo`, because it was a temporary substitution. + static constexpr std::string_view modulo_legacy_function_name = "moduloLegacy"; + + size_t i = 0; + for (const auto & element : partition_key_names_and_types) + { + auto & partition_column = block_with_min_max_partition_ids.getByName(element.name); + + if (element.name.starts_with(modulo_legacy_function_name)) + partition_column.name.replace(0, modulo_legacy_function_name.size(), "modulo"); + + Field extracted_min_partition_id_field; + Field extracted_max_partition_id_field; + + partition_column.column->get(0, extracted_min_partition_id_field); + partition_column.column->get(1, extracted_max_partition_id_field); + + if (extracted_min_partition_id_field != extracted_max_partition_id_field) + { + throw Exception( + ErrorCodes::INVALID_PARTITION_VALUE, + "Can not create the partition. A partition can not contain values that have different partition ids"); + } + + partition_column.column->get(0u, value[i++]); + } +} + NamesAndTypesList MergeTreePartition::executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context) { auto adjusted_partition_key = adjustPartitionKey(metadata_snapshot, context); diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index 78b141f26ec..fd7ae02cde4 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -1,11 +1,12 @@ #pragma once -#include +#include #include #include #include #include -#include +#include +#include namespace DB { @@ -51,6 +52,11 @@ public: void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row, ContextPtr context); + /// Copy of MergeTreePartition::create, but also validates if min max partition keys are equal. If they are different, + /// it means the partition can't be created because the data doesn't belong to the same partition. + void createAndValidateMinMaxPartitionIds( + const StorageMetadataPtr & metadata_snapshot, Block block_with_min_max_partition_ids, ContextPtr context); + static void appendFiles(const MergeTreeData & storage, Strings & files); /// Adjust partition key and execute its expression on block. Return sample block according to used expression. diff --git a/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.cpp b/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.cpp new file mode 100644 index 00000000000..21bcdb84a96 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.cpp @@ -0,0 +1,91 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} + +namespace +{ +bool isDestinationPartitionExpressionMonotonicallyIncreasing( + const std::vector & hyperrectangle, const MergeTreeData & destination_storage) +{ + auto destination_table_metadata = destination_storage.getInMemoryMetadataPtr(); + + auto key_description = destination_table_metadata->getPartitionKey(); + auto definition_ast = key_description.definition_ast->clone(); + + auto table_identifier = std::make_shared(destination_storage.getStorageID().getTableName()); + auto table_with_columns + = TableWithColumnNamesAndTypes{DatabaseAndTableWithAlias(table_identifier), destination_table_metadata->getColumns().getOrdinary()}; + + auto expression_list = extractKeyExpressionList(definition_ast); + + MonotonicityCheckVisitor::Data data{{table_with_columns}, destination_storage.getContext(), /*group_by_function_hashes*/ {}}; + + for (auto i = 0u; i < expression_list->children.size(); i++) + { + data.range = hyperrectangle[i]; + + MonotonicityCheckVisitor(data).visit(expression_list->children[i]); + + if (!data.monotonicity.is_monotonic || !data.monotonicity.is_positive) + return false; + } + + return true; +} + +bool isExpressionDirectSubsetOf(const ASTPtr source, const ASTPtr destination) +{ + auto source_expression_list = extractKeyExpressionList(source); + auto destination_expression_list = extractKeyExpressionList(destination); + + std::unordered_set source_columns; + + for (auto i = 0u; i < source_expression_list->children.size(); ++i) + source_columns.insert(source_expression_list->children[i]->getColumnName()); + + for (auto i = 0u; i < destination_expression_list->children.size(); ++i) + if (!source_columns.contains(destination_expression_list->children[i]->getColumnName())) + return false; + + return true; +} +} + +void MergeTreePartitionCompatibilityVerifier::verify( + const MergeTreeData & source_storage, const MergeTreeData & destination_storage, const DataPartsVector & source_parts) +{ + const auto source_metadata = source_storage.getInMemoryMetadataPtr(); + const auto destination_metadata = destination_storage.getInMemoryMetadataPtr(); + + const auto source_partition_key_ast = source_metadata->getPartitionKeyAST(); + const auto destination_partition_key_ast = destination_metadata->getPartitionKeyAST(); + + // If destination partition expression columns are a subset of source partition expression columns, + // there is no need to check for monotonicity. + if (isExpressionDirectSubsetOf(source_partition_key_ast, destination_partition_key_ast)) + return; + + const auto src_global_min_max_indexes = MergeTreePartitionGlobalMinMaxIdxCalculator::calculate(source_parts, destination_storage); + + assert(!src_global_min_max_indexes.hyperrectangle.empty()); + + if (!isDestinationPartitionExpressionMonotonicallyIncreasing(src_global_min_max_indexes.hyperrectangle, destination_storage)) + throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Destination table partition expression is not monotonically increasing"); + + MergeTreePartition().createAndValidateMinMaxPartitionIds( + destination_storage.getInMemoryMetadataPtr(), + src_global_min_max_indexes.getBlock(destination_storage), + destination_storage.getContext()); +} + +} diff --git a/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.h b/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.h new file mode 100644 index 00000000000..1682add3ebd --- /dev/null +++ b/src/Storages/MergeTree/MergeTreePartitionCompatibilityVerifier.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/* + * Verifies that source and destination partitions are compatible. + * To be compatible, one of the following criteria must be met: + * 1. Destination partition expression columns are a subset of source partition columns; or + * 2. Destination partition expression is monotonic on the source global min_max idx Range AND the computer partition id for + * the source global min_max idx range is the same. + * + * If not, an exception is thrown. + * */ + +class MergeTreePartitionCompatibilityVerifier +{ +public: + using DataPart = IMergeTreeDataPart; + using DataPartPtr = std::shared_ptr; + using DataPartsVector = std::vector; + + static void + verify(const MergeTreeData & source_storage, const MergeTreeData & destination_storage, const DataPartsVector & source_parts); +}; + +} diff --git a/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.cpp b/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.cpp new file mode 100644 index 00000000000..0871efadf0c --- /dev/null +++ b/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.cpp @@ -0,0 +1,25 @@ +#include + +namespace DB +{ + +IMergeTreeDataPart::MinMaxIndex +MergeTreePartitionGlobalMinMaxIdxCalculator::calculate(const DataPartsVector & parts, const MergeTreeData & storage) +{ + IMergeTreeDataPart::MinMaxIndex global_min_max_indexes; + + for (const auto & part : parts) + { + auto metadata_manager = std::make_shared(part.get()); + + auto local_min_max_index = MergeTreeData::DataPart::MinMaxIndex(); + + local_min_max_index.load(storage, metadata_manager); + + global_min_max_indexes.merge(local_min_max_index); + } + + return global_min_max_indexes; +} + +} diff --git a/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.h b/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.h new file mode 100644 index 00000000000..4f271177246 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreePartitionGlobalMinMaxIdxCalculator.h @@ -0,0 +1,24 @@ +#pragma once + +#include + +#include +#include + +namespace DB +{ + +/* + * Calculates global min max indexes for a given set of parts on given storage. + * */ +class MergeTreePartitionGlobalMinMaxIdxCalculator +{ + using DataPart = IMergeTreeDataPart; + using DataPartPtr = std::shared_ptr; + using DataPartsVector = std::vector; + +public: + static IMergeTreeDataPart::MinMaxIndex calculate(const DataPartsVector & parts, const MergeTreeData & storage); +}; + +} diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 4761ccd8b58..fd5354a00a9 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -5,9 +5,9 @@ #include #include -#include #include #include +#include #include "Common/Exception.h" #include #include @@ -20,25 +20,30 @@ #include #include #include +#include #include -#include #include #include #include #include #include -#include #include +#include +#include #include #include #include -#include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include +#include #include +#include +#include +#include #include #include #include @@ -2039,41 +2044,73 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con ProfileEventsScope profile_events_scope; MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, my_metadata_snapshot); - String partition_id = getPartitionIDFromQuery(partition, local_context); + String partition_id = src_data.getPartitionIDFromQuery(partition, local_context); DataPartsVector src_parts = src_data.getVisibleDataPartsVectorInPartition(local_context, partition_id); + + bool attach_empty_partition = !replace && src_parts.empty(); + if (attach_empty_partition) + return; + MutableDataPartsVector dst_parts; std::vector dst_parts_locks; static const String TMP_PREFIX = "tmp_replace_from_"; - for (const DataPartPtr & src_part : src_parts) + const auto my_partition_expression = my_metadata_snapshot->getPartitionKeyAST(); + const auto src_partition_expression = source_metadata_snapshot->getPartitionKeyAST(); + const auto is_partition_exp_different = queryToStringNullable(my_partition_expression) != queryToStringNullable(src_partition_expression); + + if (is_partition_exp_different && !src_parts.empty()) + MergeTreePartitionCompatibilityVerifier::verify(src_data, /* destination_storage */ *this, src_parts); + + for (DataPartPtr & src_part : src_parts) { if (!canReplacePartition(src_part)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot replace partition '{}' because part '{}' has inconsistent granularity with table", partition_id, src_part->name); - /// This will generate unique name in scope of current server process. - Int64 temp_index = insert_increment.get(); - MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level); - IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()}; - auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( - src_part, - TMP_PREFIX, - dst_part_info, - my_metadata_snapshot, - clone_params, - local_context->getReadSettings(), - local_context->getWriteSettings()); - dst_parts.emplace_back(std::move(dst_part)); - dst_parts_locks.emplace_back(std::move(part_lock)); - } + /// This will generate unique name in scope of current server process. + auto index = insert_increment.get(); - /// ATTACH empty part set - if (!replace && dst_parts.empty()) - return; + if (is_partition_exp_different) + { + auto [new_partition, new_min_max_index] = createPartitionAndMinMaxIndexFromSourcePart( + src_part, my_metadata_snapshot, local_context); + + auto [dst_part, part_lock] = cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( + src_part, + new_partition, + new_partition.getID(*this), + new_min_max_index, + TMP_PREFIX, + my_metadata_snapshot, + clone_params, + local_context, + index, + index); + + dst_parts.emplace_back(std::move(dst_part)); + dst_parts_locks.emplace_back(std::move(part_lock)); + } + else + { + MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); + + auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( + src_part, + TMP_PREFIX, + dst_part_info, + my_metadata_snapshot, + clone_params, + local_context->getReadSettings(), + local_context->getWriteSettings()); + dst_parts.emplace_back(std::move(dst_part)); + dst_parts_locks.emplace_back(std::move(part_lock)); + } + } MergeTreePartInfo drop_range; if (replace) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f7e6783dbc2..512811e39d7 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -26,22 +26,21 @@ #include -#include #include #include #include #include #include -#include #include #include -#include #include #include #include #include #include +#include #include +#include #include #include #include @@ -53,9 +52,11 @@ #include #include #include +#include #include #include #include +#include #include #include @@ -2713,16 +2714,48 @@ bool StorageReplicatedMergeTree::executeReplaceRange(LogEntry & entry) .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || ((our_zero_copy_enabled || source_zero_copy_enabled) && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport()), .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; - auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk( - part_desc->src_table_part, - TMP_PREFIX + "clone_", - part_desc->new_part_info, - metadata_snapshot, - clone_params, - getContext()->getReadSettings(), - getContext()->getWriteSettings()); - part_desc->res_part = std::move(res_part); - part_desc->temporary_part_lock = std::move(temporary_part_lock); + + const auto my_partition_expression = metadata_snapshot->getPartitionKeyAST(); + const auto src_partition_expression = source_table->getInMemoryMetadataPtr()->getPartitionKeyAST(); + + const auto is_partition_exp_different = queryToStringNullable(my_partition_expression) != queryToStringNullable(src_partition_expression); + + if (is_partition_exp_different) + { + auto [new_partition, new_min_max_index] = createPartitionAndMinMaxIndexFromSourcePart( + part_desc->src_table_part, metadata_snapshot, getContext()); + + auto partition_id = new_partition.getID(*this); + + auto [res_part, temporary_part_lock] = cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( + part_desc->src_table_part, + new_partition, + partition_id, + new_min_max_index, + TMP_PREFIX + "clone_", + metadata_snapshot, + clone_params, + getContext(), + part_desc->new_part_info.min_block, + part_desc->new_part_info.max_block); + + part_desc->res_part = std::move(res_part); + part_desc->temporary_part_lock = std::move(temporary_part_lock); + } + else + { + auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk( + part_desc->src_table_part, + TMP_PREFIX + "clone_", + part_desc->new_part_info, + metadata_snapshot, + clone_params, + getContext()->getReadSettings(), + getContext()->getWriteSettings()); + + part_desc->res_part = std::move(res_part); + part_desc->temporary_part_lock = std::move(temporary_part_lock); + } } else if (!part_desc->replica.empty()) { @@ -7852,11 +7885,22 @@ void StorageReplicatedMergeTree::replacePartitionFrom( ProfileEventsScope profile_events_scope; MergeTreeData & src_data = checkStructureAndGetMergeTreeData(source_table, source_metadata_snapshot, metadata_snapshot); - String partition_id = getPartitionIDFromQuery(partition, query_context); + String partition_id = src_data.getPartitionIDFromQuery(partition, query_context); /// NOTE: Some covered parts may be missing in src_all_parts if corresponding log entries are not executed yet. DataPartsVector src_all_parts = src_data.getVisibleDataPartsVectorInPartition(query_context, partition_id); + bool attach_empty_partition = !replace && src_all_parts.empty(); + if (attach_empty_partition) + return; + + const auto my_partition_expression = metadata_snapshot->getPartitionKeyAST(); + const auto src_partition_expression = source_metadata_snapshot->getPartitionKeyAST(); + const auto is_partition_exp_different = queryToStringNullable(my_partition_expression) != queryToStringNullable(src_partition_expression); + + if (is_partition_exp_different && !src_all_parts.empty()) + MergeTreePartitionCompatibilityVerifier::verify(src_data, /* destination_storage */ *this, src_all_parts); + LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size()); static const String TMP_PREFIX = "tmp_replace_from_"; @@ -7911,6 +7955,18 @@ void StorageReplicatedMergeTree::replacePartitionFrom( "Cannot replace partition '{}' because part '{}" "' has inconsistent granularity with table", partition_id, src_part->name); + IMergeTreeDataPart::MinMaxIndex min_max_index = *src_part->minmax_idx; + MergeTreePartition merge_tree_partition = src_part->partition; + + if (is_partition_exp_different) + { + auto [new_partition, new_min_max_index] = createPartitionAndMinMaxIndexFromSourcePart(src_part, metadata_snapshot, query_context); + + merge_tree_partition = new_partition; + min_max_index = new_min_max_index; + partition_id = merge_tree_partition.getID(*this); + } + String hash_hex = src_part->checksums.getTotalChecksumHex(); const bool is_duplicated_part = replaced_parts.contains(hash_hex); replaced_parts.insert(hash_hex); @@ -7929,27 +7985,52 @@ void StorageReplicatedMergeTree::replacePartitionFrom( continue; } - UInt64 index = lock->getNumber(); - MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication || dynamic_cast(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication; + + UInt64 index = lock->getNumber(); + IDataPartStorage::ClonePartParams clone_params { .copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()), .metadata_version_to_write = metadata_snapshot->getMetadataVersion() }; - auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( - src_part, - TMP_PREFIX, - dst_part_info, - metadata_snapshot, - clone_params, - query_context->getReadSettings(), - query_context->getWriteSettings()); + + if (is_partition_exp_different) + { + auto [dst_part, part_lock] = cloneAndLoadPartOnSameDiskWithDifferentPartitionKey( + src_part, + merge_tree_partition, + partition_id, + min_max_index, + TMP_PREFIX, + metadata_snapshot, + clone_params, + query_context, + index, + index); + + dst_parts.emplace_back(dst_part); + dst_parts_locks.emplace_back(std::move(part_lock)); + } + else + { + MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); + + auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk( + src_part, + TMP_PREFIX, + dst_part_info, + metadata_snapshot, + clone_params, + query_context->getReadSettings(), + query_context->getWriteSettings()); + + dst_parts.emplace_back(dst_part); + dst_parts_locks.emplace_back(std::move(part_lock)); + } + src_parts.emplace_back(src_part); - dst_parts.emplace_back(dst_part); - dst_parts_locks.emplace_back(std::move(part_lock)); ephemeral_locks.emplace_back(std::move(*lock)); block_id_paths.emplace_back(block_id_path); part_checksums.emplace_back(hash_hex); diff --git a/tests/integration/test_attach_partition_distinct_expression_replicated/__init__.py b/tests/integration/test_attach_partition_distinct_expression_replicated/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_attach_partition_distinct_expression_replicated/configs/remote_servers.xml b/tests/integration/test_attach_partition_distinct_expression_replicated/configs/remote_servers.xml new file mode 100644 index 00000000000..b40730e9f7d --- /dev/null +++ b/tests/integration/test_attach_partition_distinct_expression_replicated/configs/remote_servers.xml @@ -0,0 +1,17 @@ + + + + + true + + replica1 + 9000 + + + replica2 + 9000 + + + + + diff --git a/tests/integration/test_attach_partition_distinct_expression_replicated/test.py b/tests/integration/test_attach_partition_distinct_expression_replicated/test.py new file mode 100644 index 00000000000..1d8ac4e9e37 --- /dev/null +++ b/tests/integration/test_attach_partition_distinct_expression_replicated/test.py @@ -0,0 +1,214 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + +cluster = ClickHouseCluster(__file__) + +replica1 = cluster.add_instance( + "replica1", with_zookeeper=True, main_configs=["configs/remote_servers.xml"] +) +replica2 = cluster.add_instance( + "replica2", with_zookeeper=True, main_configs=["configs/remote_servers.xml"] +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + except Exception as ex: + print(ex) + finally: + cluster.shutdown() + + +def cleanup(nodes): + for node in nodes: + node.query("DROP TABLE IF EXISTS source SYNC") + node.query("DROP TABLE IF EXISTS destination SYNC") + + +def create_table(node, table_name, replicated): + replica = node.name + engine = ( + f"ReplicatedMergeTree('/clickhouse/tables/1/{table_name}', '{replica}')" + if replicated + else "MergeTree()" + ) + partition_expression = ( + "toYYYYMMDD(timestamp)" if table_name == "source" else "toYYYYMM(timestamp)" + ) + node.query_with_retry( + """ + CREATE TABLE {table_name}(timestamp DateTime) + ENGINE = {engine} + ORDER BY tuple() PARTITION BY {partition_expression} + SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; + """.format( + table_name=table_name, + engine=engine, + partition_expression=partition_expression, + ) + ) + + +def test_both_replicated(start_cluster): + for node in [replica1, replica2]: + create_table(node, "source", True) + create_table(node, "destination", True) + + replica1.query("INSERT INTO source VALUES ('2010-03-02 02:01:01')") + replica1.query("SYSTEM SYNC REPLICA source") + replica1.query("SYSTEM SYNC REPLICA destination") + replica1.query( + f"ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source" + ) + + assert_eq_with_retry( + replica1, f"SELECT * FROM destination", "2010-03-02 02:01:01\n" + ) + assert_eq_with_retry( + replica1, + f"SELECT * FROM destination", + replica2.query(f"SELECT * FROM destination"), + ) + + cleanup([replica1, replica2]) + + +def test_only_destination_replicated(start_cluster): + create_table(replica1, "source", False) + create_table(replica1, "destination", True) + create_table(replica2, "destination", True) + + replica1.query("INSERT INTO source VALUES ('2010-03-02 02:01:01')") + replica1.query("SYSTEM SYNC REPLICA destination") + replica1.query( + f"ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source" + ) + + assert_eq_with_retry( + replica1, f"SELECT * FROM destination", "2010-03-02 02:01:01\n" + ) + assert_eq_with_retry( + replica1, + f"SELECT * FROM destination", + replica2.query(f"SELECT * FROM destination"), + ) + + cleanup([replica1, replica2]) + + +def test_both_replicated_partitioned_to_unpartitioned(start_cluster): + def create_tables(nodes): + for node in nodes: + source_engine = ( + f"ReplicatedMergeTree('/clickhouse/tables/1/source', '{node.name}')" + ) + node.query( + """ + CREATE TABLE source(timestamp DateTime) + ENGINE = {engine} + ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp) + SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; + """.format( + engine=source_engine, + ) + ) + + destination_engine = f"ReplicatedMergeTree('/clickhouse/tables/1/destination', '{node.name}')" + node.query( + """ + CREATE TABLE destination(timestamp DateTime) + ENGINE = {engine} + ORDER BY tuple() PARTITION BY tuple() + SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; + """.format( + engine=destination_engine, + ) + ) + + create_tables([replica1, replica2]) + + replica1.query("INSERT INTO source VALUES ('2010-03-02 02:01:01')") + replica1.query("INSERT INTO source VALUES ('2010-03-03 02:01:01')") + replica1.query("SYSTEM SYNC REPLICA source") + replica1.query("SYSTEM SYNC REPLICA destination") + + replica1.query( + f"ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source" + ) + replica1.query( + f"ALTER TABLE destination ATTACH PARTITION ID '20100303' FROM source" + ) + + assert_eq_with_retry( + replica1, + f"SELECT * FROM destination ORDER BY timestamp", + "2010-03-02 02:01:01\n2010-03-03 02:01:01\n", + ) + assert_eq_with_retry( + replica1, + f"SELECT * FROM destination ORDER BY timestamp", + replica2.query(f"SELECT * FROM destination ORDER BY timestamp"), + ) + + cleanup([replica1, replica2]) + + +def test_both_replicated_different_exp_same_id(start_cluster): + def create_tables(nodes): + for node in nodes: + source_engine = ( + f"ReplicatedMergeTree('/clickhouse/tables/1/source', '{node.name}')" + ) + node.query( + """ + CREATE TABLE source(a UInt16,b UInt16,c UInt16,extra UInt64,Path String,Time DateTime,Value Float64,Timestamp Int64,sign Int8) + ENGINE = {engine} + ORDER BY tuple() PARTITION BY a % 3 + SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; + """.format( + engine=source_engine, + ) + ) + + destination_engine = f"ReplicatedMergeTree('/clickhouse/tables/1/destination', '{node.name}')" + node.query( + """ + CREATE TABLE destination(a UInt16,b UInt16,c UInt16,extra UInt64,Path String,Time DateTime,Value Float64,Timestamp Int64,sign Int8) + ENGINE = {engine} + ORDER BY tuple() PARTITION BY a + SETTINGS cleanup_delay_period=1, cleanup_delay_period_random_add=1, max_cleanup_delay_period=1; + """.format( + engine=destination_engine, + ) + ) + + create_tables([replica1, replica2]) + + replica1.query( + "INSERT INTO source (a, b, c, extra, sign) VALUES (1, 5, 9, 1000, 1)" + ) + replica1.query( + "INSERT INTO source (a, b, c, extra, sign) VALUES (2, 6, 10, 1000, 1)" + ) + replica1.query("SYSTEM SYNC REPLICA source") + replica1.query("SYSTEM SYNC REPLICA destination") + + replica1.query(f"ALTER TABLE destination ATTACH PARTITION 1 FROM source") + replica1.query(f"ALTER TABLE destination ATTACH PARTITION 2 FROM source") + + assert_eq_with_retry( + replica1, + f"SELECT * FROM destination ORDER BY a", + "1\t5\t9\t1000\t\t1970-01-01 00:00:00\t0\t0\t1\n2\t6\t10\t1000\t\t1970-01-01 00:00:00\t0\t0\t1\n", + ) + assert_eq_with_retry( + replica1, + f"SELECT * FROM destination ORDER BY a", + replica2.query(f"SELECT * FROM destination ORDER BY a"), + ) + + cleanup([replica1, replica2]) diff --git a/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.reference b/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.reference new file mode 100644 index 00000000000..f1d036b08bf --- /dev/null +++ b/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.reference @@ -0,0 +1,467 @@ +-- { echoOn } +-- Should be allowed since destination partition expr is monotonically increasing and compatible +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); +ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +201003 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION '20100302' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +201003 +-- Should be allowed since destination partition expr is monotonically increasing and compatible. Note that even though +-- the destination partition expression is more granular, the data would still fall in the same partition. Thus, it is valid +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); +ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +20100302 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION '201003' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +20100302 +-- Should be allowed since destination partition expr is monotonically increasing and compatible for those specific values +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY intDiv(A, 6); +CREATE TABLE destination (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY A; +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 1), ('2010-03-02 02:01:03', 1); +ALTER TABLE destination ATTACH PARTITION ID '0' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 1 +2010-03-02 02:01:03 1 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 1 +2010-03-02 02:01:03 1 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +1 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION 0 FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 1 +2010-03-02 02:01:03 1 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 1 +2010-03-02 02:01:03 1 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +1 +-- Should be allowed because dst partition exp is monot inc and data is not split +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY cityHash64(category); +CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); +INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); +INSERT INTO TABLE source VALUES ('rice', 'food'); +ALTER TABLE destination ATTACH PARTITION ID '17908065610379824077' from source; +SELECT * FROM source ORDER BY productName; +mop general +rice food +spaghetti food +SELECT * FROM destination ORDER BY productName; +rice food +spaghetti food +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +59532f3c39a412a413f0f014c7750a9d +59532f3c39a412a413f0f014c7750a9d +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION '17908065610379824077' from source; +SELECT * FROM source ORDER BY productName; +mop general +rice food +spaghetti food +SELECT * FROM destination ORDER BY productName; +rice food +spaghetti food +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +59532f3c39a412a413f0f014c7750a9d +59532f3c39a412a413f0f014c7750a9d +-- Should be allowed, extra test case to validate https://github.com/ClickHouse/ClickHouse/pull/39507#issuecomment-1747574133 + +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp Int64) engine=MergeTree ORDER BY (timestamp) PARTITION BY intDiv(timestamp, 86400000); +CREATE TABLE destination (timestamp Int64) engine=MergeTree ORDER BY (timestamp) PARTITION BY toYear(toDateTime(intDiv(timestamp, 1000))); +INSERT INTO TABLE source VALUES (1267495261123); +ALTER TABLE destination ATTACH PARTITION ID '14670' FROM source; +SELECT * FROM source ORDER BY timestamp; +1267495261123 +SELECT * FROM destination ORDER BY timestamp; +1267495261123 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +2010 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION '14670' from source; +SELECT * FROM source ORDER BY timestamp; +1267495261123 +SELECT * FROM destination ORDER BY timestamp; +1267495261123 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +2010 +-- Should be allowed, extra test case to validate https://github.com/ClickHouse/ClickHouse/pull/39507#issuecomment-1747511726 + +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime('UTC'), key Int64, f Float64) engine=MergeTree ORDER BY (key, timestamp) PARTITION BY toYear(timestamp); +CREATE TABLE destination (timestamp DateTime('UTC'), key Int64, f Float64) engine=MergeTree ORDER BY (key, timestamp) PARTITION BY (intDiv(toUInt32(timestamp),86400)); +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01',1,1),('2010-03-02 02:01:01',1,1),('2011-02-02 02:01:03',1,1); +ALTER TABLE destination ATTACH PARTITION ID '2010' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 1 1 +2010-03-02 02:01:01 1 1 +2011-02-02 02:01:03 1 1 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 1 1 +2010-03-02 02:01:01 1 1 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +14670 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION '2010' from source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 1 1 +2010-03-02 02:01:01 1 1 +2011-02-02 02:01:03 1 1 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 1 1 +2010-03-02 02:01:01 1 1 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +14670 +-- Should be allowed, partitioned table to unpartitioned. Since the destination is unpartitioned, parts would ultimately +-- fall into the same partition. +-- Destination partition by expression is omitted, which causes StorageMetadata::getPartitionKeyAST() to be nullptr. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple(); +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); +ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +all +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION '201003' from source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +all +-- Same as above, but destination partition by expression is explicitly defined. Test case required to validate that +-- partition by tuple() is accepted. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY tuple(); +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); +ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +all +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION '201003' from source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +all +-- Should be allowed because the destination partition expression columns are a subset of the source partition expression columns +-- Columns in this case refer to the expression elements, not to the actual table columns +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b, c); +CREATE TABLE destination (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b); +INSERT INTO TABLE source VALUES (1, 2, 3), (1, 2, 4); +ALTER TABLE destination ATTACH PARTITION ID '1-2-3' FROM source; +SELECT * FROM source ORDER BY (a, b, c); +1 2 3 +1 2 4 +SELECT * FROM destination ORDER BY (a, b, c); +1 2 3 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +1-2 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION (1, 2, 3) from source; +SELECT * FROM source ORDER BY (a, b, c); +1 2 3 +1 2 4 +SELECT * FROM destination ORDER BY (a, b, c); +1 2 3 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +1-2 +-- Should be allowed because the destination partition expression columns are a subset of the source partition expression columns +-- Columns in this case refer to the expression elements, not to the actual table columns +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b, c); +CREATE TABLE destination (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY a; +INSERT INTO TABLE source VALUES (1, 2, 3), (1, 2, 4); +ALTER TABLE destination ATTACH PARTITION ID '1-2-3' FROM source; +SELECT * FROM source ORDER BY (a, b, c); +1 2 3 +1 2 4 +SELECT * FROM destination ORDER BY (a, b, c); +1 2 3 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +1 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION (1, 2, 3) from source; +SELECT * FROM source ORDER BY (a, b, c); +1 2 3 +1 2 4 +SELECT * FROM destination ORDER BY (a, b, c); +1 2 3 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +1 +-- Should be allowed. Special test case, tricky to explain. First column of source partition expression is +-- timestamp, while first column of destination partition expression is `A`. One of the previous implementations +-- would not match the columns, which could lead to `timestamp` min max being used to calculate monotonicity of `A`. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (`timestamp` DateTime, `A` Int64) ENGINE = MergeTree PARTITION BY tuple(toYYYYMM(timestamp), intDiv(A, 6)) ORDER BY timestamp; +CREATE TABLE destination (`timestamp` DateTime, `A` Int64) ENGINE = MergeTree PARTITION BY A ORDER BY timestamp; +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 5); +ALTER TABLE destination ATTACH PARTITION ID '201003-0' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 5 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 5 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +5 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION (201003, 0) from source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 5 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 5 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +5 +-- Should be allowed. Destination partition expression contains multiple expressions, but all of them are monotonically +-- increasing in the source partition min max indexes. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(A, B) ORDER BY tuple(); +CREATE TABLE destination (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(intDiv(A, 2), intDiv(B, 2)) ORDER BY tuple(); +INSERT INTO TABLE source VALUES (6, 12); +ALTER TABLE destination ATTACH PARTITION ID '6-12' FROM source; +SELECT * FROM source ORDER BY A; +6 12 +SELECT * FROM destination ORDER BY A; +6 12 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +3-6 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION (6, 12) from source; +SELECT * FROM source ORDER BY A; +6 12 +SELECT * FROM destination ORDER BY A; +6 12 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +3-6 +-- Should be allowed. The same scenario as above, but partition expressions inverted. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(intDiv(A, 2), intDiv(B, 2)) ORDER BY tuple(); +CREATE TABLE destination (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(A, B) ORDER BY tuple(); +INSERT INTO TABLE source VALUES (6, 12); +ALTER TABLE destination ATTACH PARTITION ID '3-6' FROM source; +SELECT * FROM source ORDER BY A; +6 12 +SELECT * FROM destination ORDER BY A; +6 12 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +6-12 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION (3, 6) from source; +SELECT * FROM source ORDER BY A; +6 12 +SELECT * FROM destination ORDER BY A; +6 12 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +6-12 +-- Should be allowed, it is a local operation, no different than regular attach. Replicated to replicated. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE + source(timestamp DateTime) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/source_replicated_to_replicated_distinct_expression', '1') + PARTITION BY toYYYYMMDD(timestamp) + ORDER BY tuple(); +CREATE TABLE + destination(timestamp DateTime) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/destination_replicated_to_replicated_distinct_expression', '1') + PARTITION BY toYYYYMM(timestamp) + ORDER BY tuple(); +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); +ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +201003 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION '20100302' from source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +201003 +-- Should be allowed, it is a local operation, no different than regular attach. Non replicated to replicated +DROP TABLE IF EXISTS source SYNC; +DROP TABLE IF EXISTS destination SYNC; +CREATE TABLE source(timestamp DateTime) ENGINE = MergeTree() PARTITION BY toYYYYMMDD(timestamp) ORDER BY tuple(); +CREATE TABLE + destination(timestamp DateTime) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/destination_non_replicated_to_replicated_distinct_expression', '1') + PARTITION BY toYYYYMM(timestamp) + ORDER BY tuple(); +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); +ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +201003 +TRUNCATE TABLE destination; +ALTER TABLE destination ATTACH PARTITION '20100302' from source; +SELECT * FROM source ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT * FROM destination ORDER BY timestamp; +2010-03-02 02:01:01 +2010-03-02 02:01:03 +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +201003 +-- Should not be allowed because data would be split into two different partitions +DROP TABLE IF EXISTS source SYNC; +DROP TABLE IF EXISTS destination SYNC; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-03 02:01:03'); +ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; -- { serverError 248 } +ALTER TABLE destination ATTACH PARTITION '201003' from source; -- { serverError 248 } +-- Should not be allowed because data would be split into two different partitions +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY intDiv(A, 6); +CREATE TABLE destination (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY A; +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 1), ('2010-03-02 02:01:03', 2); +ALTER TABLE destination ATTACH PARTITION ID '0' FROM source; -- { serverError 248 } +ALTER TABLE destination ATTACH PARTITION 0 FROM source; -- { serverError 248 } +-- Should not be allowed because dst partition exp takes more than two arguments, so it's not considered monotonically inc +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); +CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY substring(category, 1, 2); +INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); +INSERT INTO TABLE source VALUES ('rice', 'food'); +ALTER TABLE destination ATTACH PARTITION ID '4590ba78048910b74a47d5bfb308abed' from source; -- { serverError 36 } +ALTER TABLE destination ATTACH PARTITION 'food' from source; -- { serverError 36 } +-- Should not be allowed because dst partition exp depends on a different set of columns +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); +CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(productName); +INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); +INSERT INTO TABLE source VALUES ('rice', 'food'); +ALTER TABLE destination ATTACH PARTITION ID '4590ba78048910b74a47d5bfb308abed' from source; -- { serverError 36 } +ALTER TABLE destination ATTACH PARTITION 'food' from source; -- { serverError 36 } +-- Should not be allowed because dst partition exp is not monotonically increasing +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (productName String) engine=MergeTree ORDER BY tuple() PARTITION BY left(productName, 2); +CREATE TABLE destination (productName String) engine=MergeTree ORDER BY tuple() PARTITION BY cityHash64(productName); +INSERT INTO TABLE source VALUES ('bread'), ('mop'); +INSERT INTO TABLE source VALUES ('broccoli'); +ALTER TABLE destination ATTACH PARTITION ID '4589453b7ee96ce9de1265bd57674496' from source; -- { serverError 36 } +ALTER TABLE destination ATTACH PARTITION 'br' from source; -- { serverError 36 } +-- Empty/ non-existent partition, same partition expression. Nothing should happen +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +ALTER TABLE destination ATTACH PARTITION ID '1' FROM source; +ALTER TABLE destination ATTACH PARTITION 1 FROM source; +SELECT * FROM destination; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +-- Empty/ non-existent partition, different partition expression. Nothing should happen +-- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +ALTER TABLE destination ATTACH PARTITION ID '1' FROM source; +ALTER TABLE destination ATTACH PARTITION 1 FROM source; +SELECT * FROM destination; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +-- Replace instead of attach. Empty/ non-existent partition, same partition expression. Nothing should happen +-- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +ALTER TABLE destination REPLACE PARTITION '1' FROM source; +SELECT * FROM destination; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; +-- Replace instead of attach. Empty/ non-existent partition to non-empty partition, same partition id. +-- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (A Int) engine=MergeTree ORDER BY tuple() PARTITION BY A; +CREATE TABLE destination (A Int) engine=MergeTree ORDER BY tuple() PARTITION BY A; +INSERT INTO TABLE destination VALUES (1); +ALTER TABLE destination REPLACE PARTITION '1' FROM source; +SELECT * FROM destination; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; diff --git a/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.sql b/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.sql new file mode 100644 index 00000000000..9547d6ae249 --- /dev/null +++ b/tests/queries/0_stateless/02456_test_attach_partition_different_partition_exp.sql @@ -0,0 +1,485 @@ +-- { echoOn } +-- Should be allowed since destination partition expr is monotonically increasing and compatible +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); + +ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION '20100302' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed since destination partition expr is monotonically increasing and compatible. Note that even though +-- the destination partition expression is more granular, the data would still fall in the same partition. Thus, it is valid +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); + +ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION '201003' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed since destination partition expr is monotonically increasing and compatible for those specific values +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY intDiv(A, 6); + +CREATE TABLE destination (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY A; + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 1), ('2010-03-02 02:01:03', 1); + +ALTER TABLE destination ATTACH PARTITION ID '0' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION 0 FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed because dst partition exp is monot inc and data is not split +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY cityHash64(category); +CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); + +INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); +INSERT INTO TABLE source VALUES ('rice', 'food'); + +ALTER TABLE destination ATTACH PARTITION ID '17908065610379824077' from source; + +SELECT * FROM source ORDER BY productName; +SELECT * FROM destination ORDER BY productName; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION '17908065610379824077' from source; + +SELECT * FROM source ORDER BY productName; +SELECT * FROM destination ORDER BY productName; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed, extra test case to validate https://github.com/ClickHouse/ClickHouse/pull/39507#issuecomment-1747574133 + +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (timestamp Int64) engine=MergeTree ORDER BY (timestamp) PARTITION BY intDiv(timestamp, 86400000); +CREATE TABLE destination (timestamp Int64) engine=MergeTree ORDER BY (timestamp) PARTITION BY toYear(toDateTime(intDiv(timestamp, 1000))); + +INSERT INTO TABLE source VALUES (1267495261123); + +ALTER TABLE destination ATTACH PARTITION ID '14670' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION '14670' from source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed, extra test case to validate https://github.com/ClickHouse/ClickHouse/pull/39507#issuecomment-1747511726 + +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (timestamp DateTime('UTC'), key Int64, f Float64) engine=MergeTree ORDER BY (key, timestamp) PARTITION BY toYear(timestamp); +CREATE TABLE destination (timestamp DateTime('UTC'), key Int64, f Float64) engine=MergeTree ORDER BY (key, timestamp) PARTITION BY (intDiv(toUInt32(timestamp),86400)); + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01',1,1),('2010-03-02 02:01:01',1,1),('2011-02-02 02:01:03',1,1); + +ALTER TABLE destination ATTACH PARTITION ID '2010' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION '2010' from source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed, partitioned table to unpartitioned. Since the destination is unpartitioned, parts would ultimately +-- fall into the same partition. +-- Destination partition by expression is omitted, which causes StorageMetadata::getPartitionKeyAST() to be nullptr. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple(); + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); + +ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION '201003' from source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Same as above, but destination partition by expression is explicitly defined. Test case required to validate that +-- partition by tuple() is accepted. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY tuple(); + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); + +ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION '201003' from source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed because the destination partition expression columns are a subset of the source partition expression columns +-- Columns in this case refer to the expression elements, not to the actual table columns +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b, c); +CREATE TABLE destination (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b); + +INSERT INTO TABLE source VALUES (1, 2, 3), (1, 2, 4); + +ALTER TABLE destination ATTACH PARTITION ID '1-2-3' FROM source; + +SELECT * FROM source ORDER BY (a, b, c); +SELECT * FROM destination ORDER BY (a, b, c); +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION (1, 2, 3) from source; + +SELECT * FROM source ORDER BY (a, b, c); +SELECT * FROM destination ORDER BY (a, b, c); +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed because the destination partition expression columns are a subset of the source partition expression columns +-- Columns in this case refer to the expression elements, not to the actual table columns +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE source (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY (a, b, c); +CREATE TABLE destination (a Int, b Int, c Int) engine=MergeTree ORDER BY tuple() PARTITION BY a; + +INSERT INTO TABLE source VALUES (1, 2, 3), (1, 2, 4); + +ALTER TABLE destination ATTACH PARTITION ID '1-2-3' FROM source; + +SELECT * FROM source ORDER BY (a, b, c); +SELECT * FROM destination ORDER BY (a, b, c); +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION (1, 2, 3) from source; + +SELECT * FROM source ORDER BY (a, b, c); +SELECT * FROM destination ORDER BY (a, b, c); +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed. Special test case, tricky to explain. First column of source partition expression is +-- timestamp, while first column of destination partition expression is `A`. One of the previous implementations +-- would not match the columns, which could lead to `timestamp` min max being used to calculate monotonicity of `A`. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (`timestamp` DateTime, `A` Int64) ENGINE = MergeTree PARTITION BY tuple(toYYYYMM(timestamp), intDiv(A, 6)) ORDER BY timestamp; +CREATE TABLE destination (`timestamp` DateTime, `A` Int64) ENGINE = MergeTree PARTITION BY A ORDER BY timestamp; + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 5); + +ALTER TABLE destination ATTACH PARTITION ID '201003-0' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION (201003, 0) from source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed. Destination partition expression contains multiple expressions, but all of them are monotonically +-- increasing in the source partition min max indexes. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(A, B) ORDER BY tuple(); +CREATE TABLE destination (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(intDiv(A, 2), intDiv(B, 2)) ORDER BY tuple(); + +INSERT INTO TABLE source VALUES (6, 12); + +ALTER TABLE destination ATTACH PARTITION ID '6-12' FROM source; + +SELECT * FROM source ORDER BY A; +SELECT * FROM destination ORDER BY A; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION (6, 12) from source; + +SELECT * FROM source ORDER BY A; +SELECT * FROM destination ORDER BY A; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed. The same scenario as above, but partition expressions inverted. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(intDiv(A, 2), intDiv(B, 2)) ORDER BY tuple(); +CREATE TABLE destination (A Int, B Int) ENGINE = MergeTree PARTITION BY tuple(A, B) ORDER BY tuple(); + +INSERT INTO TABLE source VALUES (6, 12); + +ALTER TABLE destination ATTACH PARTITION ID '3-6' FROM source; + +SELECT * FROM source ORDER BY A; +SELECT * FROM destination ORDER BY A; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION (3, 6) from source; + +SELECT * FROM source ORDER BY A; +SELECT * FROM destination ORDER BY A; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed, it is a local operation, no different than regular attach. Replicated to replicated. +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; +CREATE TABLE + source(timestamp DateTime) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/source_replicated_to_replicated_distinct_expression', '1') + PARTITION BY toYYYYMMDD(timestamp) + ORDER BY tuple(); + +CREATE TABLE + destination(timestamp DateTime) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/destination_replicated_to_replicated_distinct_expression', '1') + PARTITION BY toYYYYMM(timestamp) + ORDER BY tuple(); + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); + +ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION '20100302' from source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should be allowed, it is a local operation, no different than regular attach. Non replicated to replicated +DROP TABLE IF EXISTS source SYNC; +DROP TABLE IF EXISTS destination SYNC; +CREATE TABLE source(timestamp DateTime) ENGINE = MergeTree() PARTITION BY toYYYYMMDD(timestamp) ORDER BY tuple(); + +CREATE TABLE + destination(timestamp DateTime) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/destination_non_replicated_to_replicated_distinct_expression', '1') + PARTITION BY toYYYYMM(timestamp) + ORDER BY tuple(); + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-02 02:01:03'); + +ALTER TABLE destination ATTACH PARTITION ID '20100302' FROM source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +TRUNCATE TABLE destination; + +ALTER TABLE destination ATTACH PARTITION '20100302' from source; + +SELECT * FROM source ORDER BY timestamp; +SELECT * FROM destination ORDER BY timestamp; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Should not be allowed because data would be split into two different partitions +DROP TABLE IF EXISTS source SYNC; +DROP TABLE IF EXISTS destination SYNC; + +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01'), ('2010-03-03 02:01:03'); + +ALTER TABLE destination ATTACH PARTITION ID '201003' FROM source; -- { serverError 248 } +ALTER TABLE destination ATTACH PARTITION '201003' from source; -- { serverError 248 } + +-- Should not be allowed because data would be split into two different partitions +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY intDiv(A, 6); + +CREATE TABLE destination (timestamp DateTime, A Int64) engine=MergeTree ORDER BY timestamp PARTITION BY A; + +INSERT INTO TABLE source VALUES ('2010-03-02 02:01:01', 1), ('2010-03-02 02:01:03', 2); + +ALTER TABLE destination ATTACH PARTITION ID '0' FROM source; -- { serverError 248 } +ALTER TABLE destination ATTACH PARTITION 0 FROM source; -- { serverError 248 } + +-- Should not be allowed because dst partition exp takes more than two arguments, so it's not considered monotonically inc +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); +CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY substring(category, 1, 2); + +INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); +INSERT INTO TABLE source VALUES ('rice', 'food'); + +ALTER TABLE destination ATTACH PARTITION ID '4590ba78048910b74a47d5bfb308abed' from source; -- { serverError 36 } +ALTER TABLE destination ATTACH PARTITION 'food' from source; -- { serverError 36 } + +-- Should not be allowed because dst partition exp depends on a different set of columns +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(category); +CREATE TABLE destination (productName String, category String) engine=MergeTree ORDER BY tuple() PARTITION BY toString(productName); + +INSERT INTO TABLE source VALUES ('spaghetti', 'food'), ('mop', 'general'); +INSERT INTO TABLE source VALUES ('rice', 'food'); + +ALTER TABLE destination ATTACH PARTITION ID '4590ba78048910b74a47d5bfb308abed' from source; -- { serverError 36 } +ALTER TABLE destination ATTACH PARTITION 'food' from source; -- { serverError 36 } + +-- Should not be allowed because dst partition exp is not monotonically increasing +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (productName String) engine=MergeTree ORDER BY tuple() PARTITION BY left(productName, 2); +CREATE TABLE destination (productName String) engine=MergeTree ORDER BY tuple() PARTITION BY cityHash64(productName); + +INSERT INTO TABLE source VALUES ('bread'), ('mop'); +INSERT INTO TABLE source VALUES ('broccoli'); + +ALTER TABLE destination ATTACH PARTITION ID '4589453b7ee96ce9de1265bd57674496' from source; -- { serverError 36 } +ALTER TABLE destination ATTACH PARTITION 'br' from source; -- { serverError 36 } + +-- Empty/ non-existent partition, same partition expression. Nothing should happen +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); + +ALTER TABLE destination ATTACH PARTITION ID '1' FROM source; +ALTER TABLE destination ATTACH PARTITION 1 FROM source; + +SELECT * FROM destination; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Empty/ non-existent partition, different partition expression. Nothing should happen +-- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMMDD(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); + +ALTER TABLE destination ATTACH PARTITION ID '1' FROM source; +ALTER TABLE destination ATTACH PARTITION 1 FROM source; + +SELECT * FROM destination; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Replace instead of attach. Empty/ non-existent partition, same partition expression. Nothing should happen +-- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); +CREATE TABLE destination (timestamp DateTime) engine=MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(timestamp); + +ALTER TABLE destination REPLACE PARTITION '1' FROM source; + +SELECT * FROM destination; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; + +-- Replace instead of attach. Empty/ non-existent partition to non-empty partition, same partition id. +-- https://github.com/ClickHouse/ClickHouse/pull/39507#discussion_r1399839045 +DROP TABLE IF EXISTS source; +DROP TABLE IF EXISTS destination; + +CREATE TABLE source (A Int) engine=MergeTree ORDER BY tuple() PARTITION BY A; +CREATE TABLE destination (A Int) engine=MergeTree ORDER BY tuple() PARTITION BY A; + +INSERT INTO TABLE destination VALUES (1); + +ALTER TABLE destination REPLACE PARTITION '1' FROM source; + +SELECT * FROM destination; +SELECT partition_id FROM system.parts where table='destination' AND database = currentDatabase() AND active = 1; From 5179891aef9792366d948efd9f1a2454dfe8da69 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 23 Jan 2024 11:43:08 -0300 Subject: [PATCH 049/325] remove static log --- src/Storages/MergeTree/MergeTreeDataPartCloner.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp b/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp index 78cb9aa0624..e384e1b7066 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCloner.cpp @@ -13,8 +13,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -static Poco::Logger * log = &Poco::Logger::get("MergeTreeDataPartCloner"); - namespace DistinctPartitionExpression { std::unique_ptr updatePartitionFile( @@ -182,7 +180,7 @@ std::pair cloneSourcePart( } LOG_DEBUG( - log, + &Poco::Logger::get("MergeTreeDataPartCloner"), "Clone {} part {} to {}{}", src_flushed_tmp_part ? "flushed" : "", src_part_storage->getFullPath(), From 7e86c0e9280bb6e46183c2c358474bfd283e2554 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 23 Jan 2024 23:03:15 +0800 Subject: [PATCH 050/325] Compress state of dashboard --- programs/server/dashboard.html | 6 +++--- programs/server/js/lz-string.js | 1 + src/Server/WebUIRequestHandler.cpp | 9 +++++++++ 3 files changed, 13 insertions(+), 3 deletions(-) create mode 100644 programs/server/js/lz-string.js diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 04fdfb2d3ca..1f32048da79 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -5,6 +5,7 @@ ClickHouse Dashboard +