From 6097651ffe0bdac03f16f64f2082bcef95655a11 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 7 Oct 2025 13:32:48 -0300 Subject: [PATCH 01/58] tmp --- src/Access/Common/AccessType.h | 1 + src/Interpreters/InterpreterAlterQuery.cpp | 6 + src/Parsers/ASTAlterQuery.cpp | 11 + src/Parsers/ASTAlterQuery.h | 1 + src/Parsers/CommonParsers.h | 1 + src/Parsers/ParserAlterQuery.cpp | 17 + src/Storages/IStorage.h | 9 + src/Storages/MergeTree/MergeTreeData.cpp | 36 +- src/Storages/MergeTree/MergeTreeData.h | 10 + .../MergeTree/MergeTreeExportManifest.h | 13 +- .../ReplicatedMergeTreeRestartingThread.cpp | 1 + .../ObjectStorageFilePathGenerator.h | 2 +- .../ObjectStorage/StorageObjectStorage.cpp | 20 + .../ObjectStorage/StorageObjectStorage.h | 6 + .../StorageObjectStorageCluster.cpp | 11 + .../StorageObjectStorageCluster.h | 6 + src/Storages/PartitionCommands.cpp | 11 + src/Storages/PartitionCommands.h | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 350 ++++++++++++++++++ src/Storages/StorageReplicatedMergeTree.h | 7 +- 20 files changed, 511 insertions(+), 9 deletions(-) diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 843d8ee7a72f..3ebf43b0e6ae 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -211,6 +211,7 @@ enum class AccessType : uint8_t M(ALTER_SETTINGS, "ALTER SETTING, ALTER MODIFY SETTING, MODIFY SETTING, RESET SETTING", TABLE, ALTER_TABLE) /* allows to execute ALTER MODIFY SETTING */\ M(ALTER_MOVE_PARTITION, "ALTER MOVE PART, MOVE PARTITION, MOVE PART", TABLE, ALTER_TABLE) \ M(ALTER_EXPORT_PART, "ALTER EXPORT PART, EXPORT PART", TABLE, ALTER_TABLE) \ + M(ALTER_EXPORT_PARTITION, "ALTER EXPORT PARTITION, EXPORT PARTITION", TABLE, ALTER_TABLE) \ M(ALTER_FETCH_PARTITION, "ALTER FETCH PART, FETCH PARTITION", TABLE, ALTER_TABLE) \ M(ALTER_FREEZE_PARTITION, "FREEZE PARTITION, UNFREEZE", TABLE, ALTER_TABLE) \ M(ALTER_UNLOCK_SNAPSHOT, "UNLOCK SNAPSHOT", TABLE, ALTER_TABLE) \ diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 988df44e7049..aac89a671861 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -545,6 +545,12 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS required_access.emplace_back(AccessType::INSERT, command.to_database, command.to_table); break; } + case ASTAlterCommand::EXPORT_PARTITION: + { + required_access.emplace_back(AccessType::ALTER_EXPORT_PARTITION, command.to_database, command.to_table); + required_access.emplace_back(AccessType::INSERT, command.to_database, command.to_table); + break; + } case ASTAlterCommand::FETCH_PARTITION: { required_access.emplace_back(AccessType::ALTER_FETCH_PARTITION, database, table); diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 30b5f9dca156..a777280bea69 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -378,6 +378,17 @@ void ASTAlterCommand::formatImpl(WriteBuffer & ostr, const FormatSettings & sett } } + else if (type == ASTAlterCommand::EXPORT_PARTITION) + { + ostr << (settings.hilite ? hilite_keyword : "") << "EXPORT PARTITION " << (settings.hilite ? hilite_none : ""); + partition->format(ostr, settings, state, frame); + ostr << " TO TABLE "; + if (!to_database.empty()) + { + ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(to_database) << (settings.hilite ? hilite_none : "") << "."; + } + ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(to_table) << (settings.hilite ? hilite_none : ""); + } else if (type == ASTAlterCommand::REPLACE_PARTITION) { ostr << (replace ? "REPLACE" : "ATTACH") << " PARTITION " diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index d8d502cb87c6..7683b2e11c3d 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -72,6 +72,7 @@ class ASTAlterCommand : public IAST UNFREEZE_PARTITION, UNFREEZE_ALL, EXPORT_PART, + EXPORT_PARTITION, DELETE, UPDATE, diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 58694bde8984..c846a12eab02 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -333,6 +333,7 @@ namespace DB MR_MACROS(MOVE_PART, "MOVE PART") \ MR_MACROS(MOVE_PARTITION, "MOVE PARTITION") \ MR_MACROS(EXPORT_PART, "EXPORT PART") \ + MR_MACROS(EXPORT_PARTITION, "EXPORT PARTITION") \ MR_MACROS(MOVE, "MOVE") \ MR_MACROS(MS, "MS") \ MR_MACROS(MUTATION, "MUTATION") \ diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 775d495492cf..eff14253b97f 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -83,6 +83,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_move_partition(Keyword::MOVE_PARTITION); ParserKeyword s_move_part(Keyword::MOVE_PART); ParserKeyword s_export_part(Keyword::EXPORT_PART); + ParserKeyword s_export_partition(Keyword::EXPORT_PARTITION); ParserKeyword s_drop_detached_partition(Keyword::DROP_DETACHED_PARTITION); ParserKeyword s_drop_detached_part(Keyword::DROP_DETACHED_PART); ParserKeyword s_fetch_partition(Keyword::FETCH_PARTITION); @@ -553,6 +554,22 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected return false; command->move_destination_type = DataDestinationType::TABLE; } + else if (s_export_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command_partition, expected)) + return false; + + command->type = ASTAlterCommand::EXPORT_PARTITION; + + if (!s_to_table.ignore(pos, expected)) + { + return false; + } + + if (!parseDatabaseAndTableName(pos, expected, command->to_database, command->to_table)) + return false; + command->move_destination_type = DataDestinationType::TABLE; + } else if (s_move_partition.ignore(pos, expected)) { if (!parser_partition.parse(pos, command_partition, expected)) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index cb2bc3afb85a..2e2202fe02b6 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -472,6 +472,15 @@ It is currently only implemented in StorageObjectStorage. { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Import is not implemented for storage {}", getName()); } + + virtual void commitExportPartitionTransaction( + const String & /* transaction_id */, + const String & /* partition_id */, + const Strings & /* exported_paths */, + ContextPtr /* local_context */) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "commitExportPartitionTransaction is not implemented for storage type {}", getName()); + } /** Writes the data to a table in distributed manner. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e9cf056b1135..804e529bd261 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6206,10 +6206,20 @@ void MergeTreeData::exportPartToTable(const PartitionCommand & command, ContextP "Exporting merge tree part is experimental. Set `allow_experimental_export_merge_tree_part` to enable it"); } - String dest_database = query_context->resolveDatabase(command.to_database); - auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, command.to_table}, query_context); + auto part_name = command.partition->as().value.safeGet(); - if (dest_storage->getStorageID() == this->getStorageID()) + exportPartToTable(part_name, StorageID{command.to_database, command.to_table}, query_context); +} + +void MergeTreeData::exportPartToTable( + const std::string & part_name, + const StorageID & destination_storage_id, + ContextPtr query_context, + std::function completion_callback) +{ + auto dest_storage = DatabaseCatalog::instance().getTable(destination_storage_id, query_context); + + if (destination_storage_id == this->getStorageID()) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Exporting to the same table is not allowed"); } @@ -6231,8 +6241,6 @@ void MergeTreeData::exportPartToTable(const PartitionCommand & command, ContextP if (query_to_string(src_snapshot->getPartitionKeyAST()) != query_to_string(destination_snapshot->getPartitionKeyAST())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key"); - auto part_name = command.partition->as().value.safeGet(); - auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); if (!part) @@ -6244,7 +6252,8 @@ void MergeTreeData::exportPartToTable(const PartitionCommand & command, ContextP dest_storage->getStorageID(), part, query_context->getSettingsRef()[Setting::export_merge_tree_part_overwrite_file_if_exists], - query_context->getSettingsRef()[Setting::output_format_parallel_formatting]); + query_context->getSettingsRef()[Setting::output_format_parallel_formatting], + completion_callback); std::lock_guard lock(export_manifests_mutex); @@ -6311,6 +6320,9 @@ void MergeTreeData::exportPartToTableImpl( std::lock_guard inner_lock(export_manifests_mutex); export_manifests.erase(manifest); + + if (manifest.completion_callback) + manifest.completion_callback({}); return; } @@ -6400,6 +6412,9 @@ void MergeTreeData::exportPartToTableImpl( ProfileEvents::increment(ProfileEvents::PartsExports); ProfileEvents::increment(ProfileEvents::PartsExportTotalMilliseconds, static_cast((*exports_list_entry)->elapsed * 1000)); + + if (manifest.completion_callback) + manifest.completion_callback({true, destination_file_path}); } catch (...) { @@ -6421,6 +6436,9 @@ void MergeTreeData::exportPartToTableImpl( export_manifests.erase(manifest); + if (manifest.completion_callback) + manifest.completion_callback({}); + throw; } } @@ -6482,6 +6500,12 @@ Pipe MergeTreeData::alterPartition( break; } + case PartitionCommand::EXPORT_PARTITION: + { + exportPartitionToTable(command, query_context); + break; + } + case PartitionCommand::DROP_DETACHED_PARTITION: dropDetached(command.partition, command.part, query_context); break; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 7072b8d52e82..30b13ae600a4 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -984,6 +984,16 @@ class MergeTreeData : public IStorage, public WithMutableContext void exportPartToTable(const PartitionCommand & command, ContextPtr query_context); + void exportPartToTable( + const std::string & part_name, + const StorageID & destination_storage_id, ContextPtr query_context, + std::function completion_callback = {}); + + virtual void exportPartitionToTable(const PartitionCommand &, ContextPtr) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "EXPORT PARTITION is not implemented"); + } + void exportPartToTableImpl( const MergeTreeExportManifest & manifest, ContextPtr local_context); diff --git a/src/Storages/MergeTree/MergeTreeExportManifest.h b/src/Storages/MergeTree/MergeTreeExportManifest.h index 36831fd132ba..0ad93b41798b 100644 --- a/src/Storages/MergeTree/MergeTreeExportManifest.h +++ b/src/Storages/MergeTree/MergeTreeExportManifest.h @@ -8,16 +8,24 @@ struct MergeTreeExportManifest { using DataPartPtr = std::shared_ptr; + struct CompletionCallbackResult + { + bool success = false; + String relative_path_in_destination_storage; + }; + MergeTreeExportManifest( const StorageID & destination_storage_id_, const DataPartPtr & data_part_, bool overwrite_file_if_exists_, - bool parallel_formatting_) + bool parallel_formatting_, + std::function completion_callback_ = {}) : destination_storage_id(destination_storage_id_), data_part(data_part_), overwrite_file_if_exists(overwrite_file_if_exists_), parallel_formatting(parallel_formatting_), + completion_callback(completion_callback_), create_time(time(nullptr)) {} StorageID destination_storage_id; @@ -25,6 +33,9 @@ struct MergeTreeExportManifest bool overwrite_file_if_exists; bool parallel_formatting; + + std::function completion_callback; + time_t create_time; mutable bool in_progress = false; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 8420cd5738c2..0009a6a7199e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -174,6 +174,7 @@ bool ReplicatedMergeTreeRestartingThread::runImpl() storage.cleanup_thread.start(); storage.async_block_ids_cache.start(); storage.part_check_thread.start(); + storage.export_merge_tree_partition_select_task->activateAndSchedule(); LOG_DEBUG(log, "Table started successfully"); return true; diff --git a/src/Storages/ObjectStorage/ObjectStorageFilePathGenerator.h b/src/Storages/ObjectStorage/ObjectStorageFilePathGenerator.h index a7e3b102e3e7..a1f21dc502d5 100644 --- a/src/Storages/ObjectStorage/ObjectStorageFilePathGenerator.h +++ b/src/Storages/ObjectStorage/ObjectStorageFilePathGenerator.h @@ -55,7 +55,7 @@ namespace DB result += raw_path; - if (raw_path.back() != '/') + if (!result.empty() && result.back() != '/') { result += "/"; } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 377a1f5e9beb..226d5efcc286 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -512,6 +512,26 @@ SinkToStoragePtr StorageObjectStorage::import( local_context); } +void StorageObjectStorage::commitExportPartitionTransaction(const String & transaction_id, const String & partition_id, const Strings & exported_paths, ContextPtr local_context) +{ + const String commit_object = configuration->getRawPath().path + "/commit_" + partition_id + "_" + transaction_id; + + /// if file already exists, nothing to be done + if (object_storage->exists(StoredObject(commit_object))) + { + LOG_DEBUG(getLogger("StorageObjectStorage"), "Commit file already exists, nothing to be done: {}", commit_object); + return; + } + + auto out = object_storage->writeObject(StoredObject(commit_object), WriteMode::Rewrite, /* attributes= */ {}, DBMS_DEFAULT_BUFFER_SIZE, local_context->getWriteSettings()); + for (const auto & p : exported_paths) + { + out->write(p.data(), p.size()); + out->write("\n", 1); + } + out->finalize(); +} + void StorageObjectStorage::truncate( const ASTPtr & /* query */, const StorageMetadataPtr & /* metadata_snapshot */, diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 9c118913ef46..922ff470f034 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -88,6 +88,12 @@ class StorageObjectStorage : public IStorage bool /* overwrite_if_exists */, ContextPtr /* context */) override; + void commitExportPartitionTransaction( + const String & transaction_id, + const String & partition_id, + const Strings & exported_paths, + ContextPtr local_context) override; + void truncate( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index efae7b129d7a..8c412b78a547 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -905,4 +905,15 @@ bool StorageObjectStorageCluster::prefersLargeBlocks() const return IStorageCluster::prefersLargeBlocks(); } +void StorageObjectStorageCluster::commitExportPartitionTransaction( + const String & transaction_id, + const String & partition_id, + const Strings & exported_paths, + ContextPtr local_context) +{ + if (pure_storage) + return pure_storage->commitExportPartitionTransaction(transaction_id, partition_id, exported_paths, local_context); + return IStorageCluster::commitExportPartitionTransaction(transaction_id, partition_id, exported_paths, local_context); +} + } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index f8b05846f09c..23cd836b5fad 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -132,6 +132,12 @@ class StorageObjectStorageCluster : public IStorageCluster ContextPtr /* context */) override; bool prefersLargeBlocks() const override; + void commitExportPartitionTransaction( + const String & transaction_id, + const String & partition_id, + const Strings & exported_paths, + ContextPtr local_context) override; + private: void updateQueryToSendIfNeeded( ASTPtr & query, diff --git a/src/Storages/PartitionCommands.cpp b/src/Storages/PartitionCommands.cpp index 96f49a60e511..b8ef557604bc 100644 --- a/src/Storages/PartitionCommands.cpp +++ b/src/Storages/PartitionCommands.cpp @@ -140,6 +140,15 @@ std::optional PartitionCommand::parse(const ASTAlterCommand * res.to_table = command_ast->to_table; return res; } + if (command_ast->type == ASTAlterCommand::EXPORT_PARTITION) + { + PartitionCommand res; + res.type = EXPORT_PARTITION; + res.partition = command_ast->partition->clone(); + res.to_database = command_ast->to_database; + res.to_table = command_ast->to_table; + return res; + } return {}; } @@ -183,6 +192,8 @@ std::string PartitionCommand::typeToString() const return "REPLACE PARTITION"; case PartitionCommand::Type::EXPORT_PART: return "EXPORT PART"; + case PartitionCommand::Type::EXPORT_PARTITION: + return "EXPORT PARTITION"; default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Uninitialized partition command"); } diff --git a/src/Storages/PartitionCommands.h b/src/Storages/PartitionCommands.h index 15d2a7fb869f..e3f36d0e7c1f 100644 --- a/src/Storages/PartitionCommands.h +++ b/src/Storages/PartitionCommands.h @@ -34,6 +34,7 @@ struct PartitionCommand UNFREEZE_PARTITION, REPLACE_PARTITION, EXPORT_PART, + EXPORT_PARTITION, }; Type type = UNKNOWN; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index de377150f21e..cc45d4f8234e 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -116,6 +116,9 @@ #include #include +#include +#include +#include #include #include @@ -183,6 +186,7 @@ namespace Setting extern const SettingsInt64 replication_wait_for_inactive_replica_timeout; extern const SettingsUInt64 select_sequential_consistency; extern const SettingsBool update_sequential_consistency; + extern const SettingsBool allow_experimental_export_merge_tree_part; } namespace MergeTreeSetting @@ -455,6 +459,11 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( mutations_finalizing_task = getContext()->getSchedulePool().createTask( getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::mutationsFinalizingTask)", [this] { mutationsFinalizingTask(); }); + export_merge_tree_partition_select_task = getContext()->getSchedulePool().createTask( + getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_select_task)", [this] { selectPartsToExport(); }); + + export_merge_tree_partition_select_task->deactivate(); + /// This task can be scheduled by different parts of code even when storage is readonly. /// This can lead to redundant exceptions during startup. /// Will be activated by restarting thread. @@ -4354,6 +4363,197 @@ void StorageReplicatedMergeTree::mutationsFinalizingTask() } } +// static void exportPartitionCallback( +// MergeTreeExportManifest::CompletionCallbackResult result, +// zkutil::ZooKeeperPtr & zookeeper, +// const String & partition_export, +// const String & part_name) +// { +// if (!result.success) +// { +// LOG_INFO(log, "Failed to export partition {} part {}, releasing lock", partition_export, part_name); +// zookeeper->remove(owner_path); +// return; +// } + +// zookeeper->create(part_path / "path", result.relative_path_in_destination_storage, zkutil::CreateMode::Persistent); +// LOG_INFO(log, "Exported partition {} part {} to {}", partition_export, part_name, result.relative_path_in_destination_storage); + +// /// loop over all parts again and check if they all have the path set, if so, mark the export as done +// bool completed = true; +// Strings exported_paths; +// for (const auto & possibly_commited_part : part_names) +// { +// /// todo arthur is this a case of "try to write as opposed to try to read because it might be stale?" +// const auto possibly_commited_part_path = parts_path / possibly_commited_part; +// if (!zookeeper->exists(possibly_commited_part_path / "path")) +// { +// completed = false; +// break; +// } +// exported_paths.push_back(zookeeper->get(possibly_commited_part_path / "path")); +// } + +// if (completed) +// { + +// const auto destination_storage = DatabaseCatalog::instance().tryGetTable( +// StorageID(QualifiedTableName::parseFromString(destination_storage_id)), +// getContext()); +// const auto transaction_id = zookeeper->get(partition_export_path / "transaction_id"); +// /// possibly need to add a check here to see if the table still exists +// destination_storage->commitExportPartitionTransaction(transaction_id, partition_export, exported_paths, getContext()); +// zookeeper->set(partition_export_path / "status", "COMPLETED"); +// } +// } + +void StorageReplicatedMergeTree::selectPartsToExport() +{ + auto zookeeper = getZooKeeper(); + auto exports_path = fs::path(zookeeper_path) / "exports"; + + /// grab all exports/ + Strings partition_exports; + zookeeper->tryGetChildren(exports_path, partition_exports); + + for (const auto & partition_export : partition_exports) + { + const auto partition_export_path = fs::path(exports_path) / partition_export; + const auto destination_storage_id = zookeeper->get(partition_export_path / "destination_storage_id"); + + if (destination_storage_id.empty()) + { + LOG_WARNING(log, "Failed to grab destination storage id for partition {} export", partition_export); + continue; + } + + const auto time_to_live_seconds = std::stoi(zookeeper->get(partition_export_path / "time_to_live_seconds")); + const auto create_time = std::stoi(zookeeper->get(partition_export_path / "create_time")); + + if (time(nullptr) - create_time > time_to_live_seconds) + { + LOG_INFO(log, "Partition {} export has expired, removing zk entry", partition_export); + zookeeper->tryRemoveRecursive(partition_export_path); + continue; + } + + if (zookeeper->get(partition_export_path / "status") != "IN_PROGRESS") + { + LOG_INFO(log, "Partition {} export is not in progress, skipping", partition_export); + continue; + } + + /// grab all part names, if it already has an owner, skip + /// otherwise, try to atomically claim it + const auto parts_path = partition_export_path / "parts"; + auto part_names = zookeeper->getChildren(parts_path); + + for (const auto & part_name : part_names) + { + /// The check if this replica contains the part is also performed in `exportPartToTable`. + /// Doing it here so that we don't even bother claiming the part export if we don't have it. + auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + if (!part) + { + LOG_INFO(log, "Part {} not found in the replica, skipping export", part_name); + continue; + } + + auto part_path = parts_path / part_name; + auto owner_path = part_path / "owner"; + + /// Ephemeral because it acts like a lock. AFAIK, if this node crashes, the lock will be released and another replica can claim it. + if (Coordination::Error::ZOK != zookeeper->tryCreate(owner_path, replica_name, zkutil::CreateMode::Ephemeral)) + { + LOG_INFO(log, "Export partition {} part {} already has an owner or failed to claim it, skipping", partition_export, part_name); + continue; + } + + LOG_INFO(log, "Claimed export partition {} part {}", partition_export, part_name); + + /// Two failure paths: catch block and the lambda function. The try-catch is for the case where it fails to schedule the export. + /// The lambda is for the case the export itself fails. + try + { + std::function callback = [this, partition_export, part_name, owner_path, part_path, parts_path, part_names, destination_storage_id, partition_export_path] + (MergeTreeExportManifest::CompletionCallbackResult result) + { + auto zk = getZooKeeper(); + if (!result.success) + { + LOG_INFO(log, "Failed to export part {}, releasing lock", part_name); + + const auto retry_count = std::stoi(zk->get(part_path / "retry_count")) + 1; + const auto max_retry_count = std::stoi(zk->get(part_path / "max_retry_count")); + + if (retry_count >= max_retry_count) + { + LOG_INFO(log, "Failed to export part {} after {} retries, setting status to FAILED", part_name, retry_count); + zk->set(partition_export_path / "status", "FAILED"); + } + else + { + zk->set(part_path / "retry_count", std::to_string(retry_count)); + } + + zk->remove(owner_path); + return; + } + + zk->create(part_path / "path", result.relative_path_in_destination_storage, zkutil::CreateMode::Persistent); + LOG_INFO(log, "Exported partition {} part {} to {}", partition_export, part_name, result.relative_path_in_destination_storage); + + /// loop over all parts again and check if they all have the path set, if so, mark the export as done + Strings exported_paths; + for (const auto & possibly_commited_part : part_names) + { + /// todo arthur is this a case of "try to write as opposed to try to read because it might be stale?" + const auto possibly_commited_part_path = parts_path / possibly_commited_part; + if (!zk->exists(possibly_commited_part_path / "path")) + { + break; + } + exported_paths.push_back(zk->get(possibly_commited_part_path / "path")); + } + + bool exported_all_parts = exported_paths.size() == part_names.size(); + if (!exported_all_parts) + { + return; + } + + const auto destination_storage = DatabaseCatalog::instance().tryGetTable( + StorageID(QualifiedTableName::parseFromString(destination_storage_id)), + getContext()); + const auto transaction_id = zk->get(partition_export_path / "transaction_id"); + /// possibly need to add a check here to see if the table still exists + destination_storage->commitExportPartitionTransaction(transaction_id, partition_export, exported_paths, getContext()); + zk->tryCreate(partition_export_path / "status", "COMPLETED", zkutil::CreateMode::Persistent); + }; + + exportPartToTable( + part_name, + StorageID(QualifiedTableName::parseFromString(destination_storage_id)), + getContext(), + callback); + + /// for now, export only one part at a time + break; + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + + zookeeper->remove(owner_path); + /// for now, export only one part at a time + break; + } + } + } + + export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); +} + StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::createLogEntryToMergeParts( zkutil::ZooKeeperPtr & zookeeper, @@ -7877,6 +8077,156 @@ void StorageReplicatedMergeTree::fetchPartition( LOG_TRACE(log, "Fetch took {} sec. ({} tries)", watch.elapsedSeconds(), try_no); } +void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & command, ContextPtr query_context) +{ + if (!query_context->getSettingsRef()[Setting::allow_experimental_export_merge_tree_part]) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Exporting merge tree part is experimental. Set `allow_experimental_export_merge_tree_part` to enable it"); + } + + String dest_database = query_context->resolveDatabase(command.to_database); + auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, command.to_table}, query_context); + + if (dest_storage->getStorageID() == this->getStorageID()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Exporting to the same table is not allowed"); + } + + if (!dest_storage->supportsImport()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Destination storage {} does not support MergeTree parts or uses unsupported partitioning", dest_storage->getName()); + + auto query_to_string = [] (const ASTPtr & ast) + { + return ast ? ast->formatWithSecretsOneLine() : ""; + }; + + auto src_snapshot = getInMemoryMetadataPtr(); + auto destination_snapshot = dest_storage->getInMemoryMetadataPtr(); + + if (destination_snapshot->getColumns().getAllPhysical().sizeOfDifference(src_snapshot->getColumns().getAllPhysical())) + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Tables have different structure"); + + if (query_to_string(src_snapshot->getPartitionKeyAST()) != query_to_string(destination_snapshot->getPartitionKeyAST())) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key"); + + zkutil::ZooKeeperPtr zookeeper = getZooKeeperAndAssertNotReadonly(); + + const String partition_id = getPartitionIDFromQuery(command.partition, query_context); + + const auto exports_path = fs::path(zookeeper_path) / "exports"; + Coordination::Requests ops; + + /// maybe this should go in initialization somewhere else + if (!zookeeper->exists(exports_path)) + { + ops.emplace_back(zkutil::makeCreateRequest(exports_path, "", zkutil::CreateMode::Persistent)); + } + + const auto partition_exports_path = exports_path / partition_id; + + /// check if entry already exists + if (zookeeper->exists(partition_exports_path)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition {} already exported or it is being exported", partition_id); + } + + ops.emplace_back(zkutil::makeCreateRequest(partition_exports_path, "", zkutil::CreateMode::Persistent)); + + const Strings parts = zookeeper->getChildren(fs::path(replica_path) / "parts"); + const ActiveDataPartSet active_parts_set(format_version, parts); + const auto part_infos = active_parts_set.getPartInfos(); + std::vector parts_to_export; + for (const auto & part_info : part_infos) + { + if (part_info.getPartitionId() == partition_id) + { + parts_to_export.push_back(part_info.getPartNameV1()); + } + } + + if (parts_to_export.empty()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition {} doesn't exist", partition_id); + } + + /// somehow check if the list of parts is updated "enough" + + /* + table_path/exports/partition_id + table_path/exports/partition_id/transaction_id + table_path/exports/partition_id/destination_storage_id + table_path/exports/partition_id/status + table_path/exports/partition_id/create_time + table_path/exports/partition_id/time_to_live_seconds + table_path/exports/partition_id/part_name_1 + table_path/exports/partition_id/part_name_1/status + table_path/exports/partition_id/part_name_1/owner + table_path/exports/partition_id/part_name_1/retry_count + table_path/exports/partition_id/part_name_1/max_retry_count + table_path/exports/partition_id/part_name_1/path + table_path/exports/partition_id/part_name_n + ... + */ + + const auto transaction_id = std::to_string(generateSnowflakeID()); + + const auto parts_path = fs::path(partition_exports_path) / "parts"; + ops.emplace_back(zkutil::makeCreateRequest( + parts_path, "", + zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "transaction_id", + transaction_id, + zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "destination_storage_id", + dest_storage->getStorageID().getFullTableName(), + zkutil::CreateMode::Persistent)); + + /// status: IN_PROGRESS, COMPLETED, FAILED + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "status", + "IN_PROGRESS", + zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "create_time", + std::to_string(time(nullptr)), + zkutil::CreateMode::Persistent)); + + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "time_to_live_seconds", + std::to_string(3600), + zkutil::CreateMode::Persistent)); + + for (const String & part_name : parts_to_export) + { + const String part_path = fs::path(parts_path) / part_name; + + ops.emplace_back(zkutil::makeCreateRequest(part_path, "", zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(part_path) / "retry_count", + "0", + zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(part_path) / "max_retry_count", + "3", + zkutil::CreateMode::Persistent)); + } + + Coordination::Responses responses; + Coordination::Error code = zookeeper->tryMulti(ops, responses); + + if (code != Coordination::Error::ZOK) + throw zkutil::KeeperException::fromPath(code, partition_exports_path); +} + void StorageReplicatedMergeTree::forgetPartition(const ASTPtr & partition, ContextPtr query_context) { diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 5abf14c1400d..223240ac504c 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -510,6 +510,8 @@ class StorageReplicatedMergeTree final : public MergeTreeData /// A task that marks finished mutations as done. BackgroundSchedulePoolTaskHolder mutations_finalizing_task; + BackgroundSchedulePoolTaskHolder export_merge_tree_partition_select_task; + /// A thread that removes old parts, log entries, and blocks. ReplicatedMergeTreeCleanupThread cleanup_thread; @@ -737,6 +739,8 @@ class StorageReplicatedMergeTree final : public MergeTreeData /// Checks if some mutations are done and marks them as done. void mutationsFinalizingTask(); + void selectPartsToExport(); + /** Write the selected parts to merge into the log, * Call when merge_selecting_mutex is locked. * Returns false if any part is not in ZK. @@ -923,7 +927,8 @@ class StorageReplicatedMergeTree final : public MergeTreeData bool fetch_part, ContextPtr query_context) override; void forgetPartition(const ASTPtr & partition, ContextPtr query_context) override; - + + void exportPartitionToTable(const PartitionCommand &, ContextPtr) override; /// NOTE: there are no guarantees for concurrent merges. Dropping part can /// be concurrently merged into some covering part and dropPart will do From 43e945995831cd0720a377500978775ff2f5622d Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 9 Oct 2025 10:43:50 -0300 Subject: [PATCH 02/58] tmp2 - just in case of disaster recovery --- ...portReplicatedMergeTreePartitionManifest.h | 68 ++ ...ortReplicatedMergeTreePartitionTaskEntry.h | 13 + .../ReplicatedMergeTreeRestartingThread.cpp | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 601 +++++++++++++----- src/Storages/StorageReplicatedMergeTree.h | 12 + 5 files changed, 541 insertions(+), 154 deletions(-) create mode 100644 src/Storages/ExportReplicatedMergeTreePartitionManifest.h create mode 100644 src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h diff --git a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h new file mode 100644 index 000000000000..11bdb45e5bc3 --- /dev/null +++ b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h @@ -0,0 +1,68 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +struct ExportReplicatedMergeTreePartitionManifest +{ + String transaction_id; + String partition_id; + String destination_database; + String destination_table; + String source_replica; + size_t number_of_parts; + std::vector parts; + time_t create_time; + + std::string toJsonString() const + { + Poco::JSON::Object json; + json.set("transaction_id", transaction_id); + json.set("partition_id", partition_id); + json.set("destination_database", destination_database); + json.set("destination_table", destination_table); + json.set("source_replica", source_replica); + json.set("number_of_parts", number_of_parts); + + Poco::JSON::Array::Ptr parts_array = new Poco::JSON::Array(); + for (const auto & part : parts) + parts_array->add(part); + json.set("parts", parts_array); + + json.set("create_time", create_time); + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + return oss.str(); + } + + static ExportReplicatedMergeTreePartitionManifest fromJsonString(const std::string & json_string) + { + Poco::JSON::Parser parser; + auto json = parser.parse(json_string).extract(); + chassert(json); + + ExportReplicatedMergeTreePartitionManifest manifest; + manifest.transaction_id = json->getValue("transaction_id"); + manifest.partition_id = json->getValue("partition_id"); + manifest.destination_database = json->getValue("destination_database"); + manifest.destination_table = json->getValue("destination_table"); + manifest.source_replica = json->getValue("source_replica"); + manifest.number_of_parts = json->getValue("number_of_parts"); + + auto parts_array = json->getArray("parts"); + for (size_t i = 0; i < parts_array->size(); ++i) + manifest.parts.push_back(parts_array->getElement(static_cast(i))); + + manifest.create_time = json->getValue("create_time"); + return manifest; + } +}; + +} diff --git a/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h new file mode 100644 index 000000000000..122c29b1b244 --- /dev/null +++ b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h @@ -0,0 +1,13 @@ +#pragma once + +#include + +namespace DB +{ +struct ExportReplicatedMergeTreePartitionTaskEntry +{ + ExportReplicatedMergeTreePartitionManifest manifest; + + std::size_t parts_to_do; +}; +} diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 0009a6a7199e..c7b183ea0895 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -174,6 +174,7 @@ bool ReplicatedMergeTreeRestartingThread::runImpl() storage.cleanup_thread.start(); storage.async_block_ids_cache.start(); storage.part_check_thread.start(); + storage.export_merge_tree_partition_updating_task->activateAndSchedule(); storage.export_merge_tree_partition_select_task->activateAndSchedule(); LOG_DEBUG(log, "Table started successfully"); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index cc45d4f8234e..d2e03114585d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7,6 +7,7 @@ #include #include +#include "Common/ZooKeeper/IKeeper.h" #include #include #include @@ -116,6 +117,9 @@ #include #include +#include "Interpreters/StorageID.h" +#include "Storages/ExportReplicatedMergeTreePartitionManifest.h" +#include "Storages/ExportReplicatedMergeTreePartitionTaskEntry.h" #include #include #include @@ -459,16 +463,23 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( mutations_finalizing_task = getContext()->getSchedulePool().createTask( getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::mutationsFinalizingTask)", [this] { mutationsFinalizingTask(); }); - export_merge_tree_partition_select_task = getContext()->getSchedulePool().createTask( - getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_select_task)", [this] { selectPartsToExport(); }); - - export_merge_tree_partition_select_task->deactivate(); - /// This task can be scheduled by different parts of code even when storage is readonly. /// This can lead to redundant exceptions during startup. /// Will be activated by restarting thread. mutations_finalizing_task->deactivate(); + export_merge_tree_partition_updating_task = getContext()->getSchedulePool().createTask( + getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_updating_task)", [this] { exportMergeTreePartitionUpdatingTask(); }); + + export_merge_tree_partition_updating_task->deactivate(); + + export_merge_tree_partition_watch_callback = std::make_shared(export_merge_tree_partition_updating_task->getWatchCallback()); + + export_merge_tree_partition_select_task = getContext()->getSchedulePool().createTask( + getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_select_task)", [this] { selectPartsToExport(); }); + + export_merge_tree_partition_select_task->deactivate(); + bool has_zookeeper = getContext()->hasZooKeeper() || getContext()->hasAuxiliaryZooKeeper(zookeeper_info.zookeeper_name); if (has_zookeeper) { @@ -1055,6 +1066,8 @@ bool StorageReplicatedMergeTree::createTableIfNotExistsAttempt(const StorageMeta zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/mutations", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/exports", "", + zkutil::CreateMode::Persistent)); /// And create first replica atomically. See also "createReplica" method that is used to create not the first replicas. @@ -4407,147 +4420,415 @@ void StorageReplicatedMergeTree::mutationsFinalizingTask() // } // } -void StorageReplicatedMergeTree::selectPartsToExport() +void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() { - auto zookeeper = getZooKeeper(); - auto exports_path = fs::path(zookeeper_path) / "exports"; + std::lock_guard lock(export_merge_tree_partition_mutex); + + auto zk = getZooKeeper(); + + const auto exports_path = fs::path(zookeeper_path) / "exports"; + /// why do I need the stat? + Coordination::Stat stat; + const auto partition_exports_in_zk = zk->getChildrenWatch(exports_path, &stat, export_merge_tree_partition_watch_callback); + const auto partition_exports_in_zk_set = std::unordered_set(partition_exports_in_zk.begin(), partition_exports_in_zk.end()); - /// grab all exports/ - Strings partition_exports; - zookeeper->tryGetChildren(exports_path, partition_exports); + std::unordered_set local_export_partition_entries; + for (const auto & entry : export_merge_tree_partition_task_entries) + local_export_partition_entries.insert(entry.first); - for (const auto & partition_export : partition_exports) + std::unordered_set new_entries; + std::unordered_set removed_entries; + + for (const auto & partition_export_task_entry : local_export_partition_entries) + { + if (!partition_exports_in_zk_set.contains(partition_export_task_entry)) + removed_entries.insert(partition_export_task_entry); + } + + for (const auto & partition_export : partition_exports_in_zk_set) { - const auto partition_export_path = fs::path(exports_path) / partition_export; - const auto destination_storage_id = zookeeper->get(partition_export_path / "destination_storage_id"); + if (!local_export_partition_entries.contains(partition_export)) + new_entries.insert(partition_export); + } + + /// remove the removed entries from the local set + for (const auto & entry : removed_entries) + export_merge_tree_partition_task_entries.erase(entry); - if (destination_storage_id.empty()) + // add the new entries to the local set + for (const auto & entry_key : new_entries) + { + // get entry from zk + const auto entry_path = fs::path(exports_path) / entry_key; + const auto metadata_entry_path = fs::path(entry_path) / "metadata.json"; + const auto parts_to_do_path = fs::path(entry_path) / "parts_to_do"; + + std::string parts_to_do_string; + if (!zk->tryGet(parts_to_do_path, parts_to_do_string)) { - LOG_WARNING(log, "Failed to grab destination storage id for partition {} export", partition_export); + LOG_INFO(log, "Skipping..."); continue; } - const auto time_to_live_seconds = std::stoi(zookeeper->get(partition_export_path / "time_to_live_seconds")); - const auto create_time = std::stoi(zookeeper->get(partition_export_path / "create_time")); + const auto parts_to_do = std::stoull(parts_to_do_string.c_str()); - if (time(nullptr) - create_time > time_to_live_seconds) + if (parts_to_do == 0) { - LOG_INFO(log, "Partition {} export has expired, removing zk entry", partition_export); - zookeeper->tryRemoveRecursive(partition_export_path); + LOG_INFO(log, "Skipping... Parts to do is 0, will not load it"); continue; } - if (zookeeper->get(partition_export_path / "status") != "IN_PROGRESS") + std::string metadata_json; + if (!zk->tryGet(metadata_entry_path, metadata_json)) { - LOG_INFO(log, "Partition {} export is not in progress, skipping", partition_export); + LOG_INFO(log, "Skipping..."); continue; } - /// grab all part names, if it already has an owner, skip - /// otherwise, try to atomically claim it - const auto parts_path = partition_export_path / "parts"; - auto part_names = zookeeper->getChildren(parts_path); + const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); - for (const auto & part_name : part_names) + export_merge_tree_partition_task_entries[entry_key] = ExportReplicatedMergeTreePartitionTaskEntry {metadata, parts_to_do}; + } +} + +void StorageReplicatedMergeTree::selectPartsToExport() +{ + const auto exports_path = fs::path(zookeeper_path) / "exports"; + + auto complete_part_export = [&]( + const std::string & export_partition_path, + const std::string & part_status_path, + const std::string & parts_to_do_path, + const std::string & lock_path, + const std::string & next_idx_path, + const std::size_t next_idx_local, + const ZooKeeperPtr & zk) -> bool + { + /// todo arthur is it possible to grab stats using a multi-op? + Coordination::Stat parts_to_do_stat; + Coordination::Stat lock_stat; + Coordination::Stat next_idx_stat; + std::string parts_to_do_string; + + int retries = 0; + const int max_retries = 3; + while (retries < max_retries) { - /// The check if this replica contains the part is also performed in `exportPartToTable`. - /// Doing it here so that we don't even bother claiming the part export if we don't have it. - auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + if (!zk->tryGet(parts_to_do_path, parts_to_do_string, &parts_to_do_stat)) + { + LOG_INFO(log, "Failed to get parts_to_do, skipping"); + return false; + } + + std::string locked_by; + + if (!zk->tryGet(lock_path, locked_by, &lock_stat)) + { + LOG_INFO(log, "Failed to get locked_by, skipping"); + return false; + } + + if (locked_by != replica_name) + { + LOG_INFO(log, "Skipping... Locked by {}, not by {}", locked_by, replica_name); + return false; + } + + std::string next_idx_string; + if (!zk->tryGet(next_idx_path, next_idx_string, &next_idx_stat)) + { + LOG_INFO(log, "Failed to get next_idx, skipping"); + return false; + } + + const std::size_t next_idx_zk = std::stoull(next_idx_string.c_str()); + + std::size_t parts_to_do = std::stoull(parts_to_do_string.c_str()); + + if (parts_to_do == 0) + { + LOG_INFO(log, "Skipping... Parts to do is 0, maybe someone else already completed it? that sounds weird"); + return false; + } + + parts_to_do--; + + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(lock_path, lock_stat.version)); + ops.emplace_back(zkutil::makeCheckRequest(parts_to_do_path, parts_to_do_stat.version)); + ops.emplace_back(zkutil::makeCheckRequest(next_idx_path, next_idx_stat.version)); + ops.emplace_back(zkutil::makeSetRequest(next_idx_path, std::to_string(std::max(next_idx_zk, next_idx_local + 1)), next_idx_stat.version)); + ops.emplace_back(zkutil::makeSetRequest(part_status_path, "COMPLETED", -1)); + ops.emplace_back(zkutil::makeRemoveRequest(lock_path, lock_stat.version)); + ops.emplace_back(zkutil::makeSetRequest(parts_to_do_path, std::to_string(parts_to_do), parts_to_do_stat.version)); + + if (parts_to_do == 0) + { + ops.emplace_back(zkutil::makeSetRequest(fs::path(export_partition_path) / "status", "COMPLETED", -1)); + } + + Coordination::Responses responses; + if (zk->tryMulti(ops, responses) == Coordination::Error::ZOK) + { + return true; + } + + retries++; + } + + return false; + }; + + auto lock_part = [&]( + const std::string & export_partition_path, + const std::string & part_path, + const std::string & status_path, + const std::string & lock_path, + const std::string & node_name, + const ZooKeeperPtr & zk) -> bool + { + Coordination::Requests ops; + + /// if the part path exists, it can be one of the following: + /// 1. PENDING and Locked - Some replica is working on it - we should skip + /// 2. PENDING and unlocked - Whoever was working on it died - we should acquire the lock + /// 3. COMPLETED and unlocked - We should skip + if (zk->exists(part_path)) + { + Coordination::Stat stat; + std::string status; + + if (zk->tryGet(status_path, status, &stat)) + { + if (status != "PENDING") + { + LOG_INFO(log, "Skipping... Status is not PENDING"); + return false; + } + + std::string parts_to_do_zk; + if (!zk->tryGet(fs::path(export_partition_path) / "parts_to_do", parts_to_do_zk)) + { + LOG_INFO(log, "Failed to get parts_to_do, skipping"); + return false; + } + + const auto parts_to_do = std::stoull(parts_to_do_zk.c_str()); + + if (parts_to_do == 0) + { + LOG_INFO(log, "Skipping... Parts to do is 0, maybe someone else already completed it?"); + return false; + } + + /// only try to lock it if the status is still PENDING + /// if we did not check for status = pending, chances are some other replica completed and released the lock in the meantime + ops.emplace_back(zkutil::makeCheckRequest(status_path, stat.version)); + /// todo do I need to "re-set" the status to PENDING? I don't think so. + ops.emplace_back(zkutil::makeCreateRequest(lock_path, node_name, zkutil::CreateMode::Ephemeral)); + /// no need to update retry count here. + } + else + { + LOG_INFO(log, "Skipping... Failed to get status, probably killed"); + return false; + } + } + else + { + /// if the node does not exist, just create everything from scratch. + ops.emplace_back(zkutil::makeCreateRequest(part_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(status_path, "PENDING", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(lock_path, node_name, zkutil::CreateMode::Ephemeral)); + ops.emplace_back(zkutil::makeCreateRequest(fs::path(part_path) / "retry_count", "0", zkutil::CreateMode::Persistent)); + } + + Coordination::Responses responses; + + return zk->tryMulti(ops, responses) == Coordination::Error::ZOK; + }; + + auto try_to_acquire_a_part = [&]( + const ExportReplicatedMergeTreePartitionManifest & manifest, + const std::string & partition_export_path, + const std::size_t next_idx, + const ZooKeeperPtr & zk) -> std::optional + { + for (auto i = next_idx; i < manifest.number_of_parts; i++) + { + const auto part_path = fs::path(partition_export_path) / "parts" / manifest.parts[i]; + const auto lock_path = fs::path(part_path) / "lock"; + const auto status_path = fs::path(part_path) / "status"; + + const auto part = getPartIfExists(manifest.parts[i], {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); if (!part) { - LOG_INFO(log, "Part {} not found in the replica, skipping export", part_name); + LOG_INFO(log, "Skipping... Part {} not found locally", manifest.parts[i]); continue; } - auto part_path = parts_path / part_name; - auto owner_path = part_path / "owner"; + if (lock_part(partition_export_path, part_path, status_path, lock_path, replica_name, zk)) + { + return manifest.parts[i]; + } + } + + /// failed to lock a part in the range of `next_idx...number_of_parts` + /// now try the full scan `0...next_idx` + for (auto i = 0u; i < next_idx; i++) + { + const auto part_path = fs::path(partition_export_path) / "parts" / manifest.parts[i]; + const auto lock_path = fs::path(part_path) / "lock"; + const auto status_path = fs::path(part_path) / "status"; - /// Ephemeral because it acts like a lock. AFAIK, if this node crashes, the lock will be released and another replica can claim it. - if (Coordination::Error::ZOK != zookeeper->tryCreate(owner_path, replica_name, zkutil::CreateMode::Ephemeral)) + const auto part = getPartIfExists(manifest.parts[i], {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + if (!part) { - LOG_INFO(log, "Export partition {} part {} already has an owner or failed to claim it, skipping", partition_export, part_name); + LOG_INFO(log, "Skipping... Part {} not found locally", manifest.parts[i]); continue; } - LOG_INFO(log, "Claimed export partition {} part {}", partition_export, part_name); + if (lock_part(partition_export_path, part_path, status_path, lock_path, replica_name, zk)) + { + return manifest.parts[i]; + } + } + + return std::nullopt; + }; + + const auto zk = getZooKeeper(); - /// Two failure paths: catch block and the lambda function. The try-catch is for the case where it fails to schedule the export. - /// The lambda is for the case the export itself fails. + std::lock_guard lock(export_merge_tree_partition_mutex); + + for (const auto & [key, task_entry] : export_merge_tree_partition_task_entries) + { + /// this sounds impossible, but just in case + if (task_entry.parts_to_do == 0) + { + LOG_INFO(log, "Already completed, skipping"); + continue; + } + + const auto & manifest = task_entry.manifest; + const auto & database = getContext()->resolveDatabase(manifest.destination_database); + const auto & table = manifest.destination_table; + + const auto destination_storage_id = StorageID(QualifiedTableName {database, table}); + + const auto destination_storage = DatabaseCatalog::instance().tryGetTable(destination_storage_id, getContext()); + + if (!destination_storage) + { + LOG_INFO(log, "Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); + continue; + } + + const auto partition_path = fs::path(exports_path) / key; + const auto next_idx_path = fs::path(partition_path) / "next_idx"; + std::string next_idx_string; + if (!zk->tryGet(next_idx_path, next_idx_string)) + { + LOG_INFO(log, "Failed to get next_idx, skipping"); + continue; + } + + const auto next_idx = std::stoull(next_idx_string.c_str()); + + const auto part_to_export = try_to_acquire_a_part(manifest, partition_path, next_idx, zk); + + if (part_to_export.has_value()) + { try { - std::function callback = [this, partition_export, part_name, owner_path, part_path, parts_path, part_names, destination_storage_id, partition_export_path] - (MergeTreeExportManifest::CompletionCallbackResult result) + exportPartToTable( + part_to_export.value(), + destination_storage_id, + getContext(), + [this, exports_path, key, part_to_export, complete_part_export, partition_path, next_idx_path, next_idx] + (MergeTreeExportManifest::CompletionCallbackResult result) { - auto zk = getZooKeeper(); - if (!result.success) + + const auto zk_client = getZooKeeper(); + if (result.success) + { + complete_part_export( + partition_path, + fs::path(exports_path) / key / "parts" / part_to_export.value() / "status", + fs::path(exports_path) / key / "parts_to_do", + fs::path(exports_path) / key / "parts" / part_to_export.value() / "lock", + next_idx_path, + next_idx, + zk_client); + + /// maybe get up to date from complete_parts_export? + std::lock_guard inner_lock(export_merge_tree_partition_mutex); + export_merge_tree_partition_task_entries[key].parts_to_do--; + + if (export_merge_tree_partition_task_entries[key].parts_to_do == 0) + { + export_merge_tree_partition_task_entries.erase(key); + } + } + else { - LOG_INFO(log, "Failed to export part {}, releasing lock", part_name); + /// increment retry_count + /// if above threshhold, fail the entire export - hopefully it is safe to do so :D + /// I could also leave this for the cleanup thread, but will do it here for now. - const auto retry_count = std::stoi(zk->get(part_path / "retry_count")) + 1; - const auto max_retry_count = std::stoi(zk->get(part_path / "max_retry_count")); + bool erase_entry = false; + Coordination::Requests ops; - if (retry_count >= max_retry_count) + std::string retry_count_string; + if (zk_client->tryGet(fs::path(exports_path) / key / "parts" / part_to_export.value() / "retry_count", retry_count_string)) { - LOG_INFO(log, "Failed to export part {} after {} retries, setting status to FAILED", part_name, retry_count); - zk->set(partition_export_path / "status", "FAILED"); + std::size_t retry_count = std::stoull(retry_count_string.c_str()) + 1; + + //// todo arthur unhardcode this + if (retry_count > 3) + { + ops.emplace_back(zkutil::makeRemoveRequest(partition_path, -1)); + erase_entry = true; + } + else + { + ops.emplace_back(zkutil::makeSetRequest(fs::path(exports_path) / key / "parts" / part_to_export.value() / "retry_count", std::to_string(retry_count), -1)); + /// unlock the part + ops.emplace_back(zkutil::makeRemoveRequest(fs::path(exports_path) / key / "parts" / part_to_export.value() / "lock", -1)); + } } else { - zk->set(part_path / "retry_count", std::to_string(retry_count)); + LOG_INFO(log, "Failed to get retry_count, will not try to update it"); + ops.emplace_back(zkutil::makeRemoveRequest(fs::path(partition_path) / "parts" / part_to_export.value() / "lock", -1)); } - zk->remove(owner_path); - return; - } - - zk->create(part_path / "path", result.relative_path_in_destination_storage, zkutil::CreateMode::Persistent); - LOG_INFO(log, "Exported partition {} part {} to {}", partition_export, part_name, result.relative_path_in_destination_storage); - - /// loop over all parts again and check if they all have the path set, if so, mark the export as done - Strings exported_paths; - for (const auto & possibly_commited_part : part_names) - { - /// todo arthur is this a case of "try to write as opposed to try to read because it might be stale?" - const auto possibly_commited_part_path = parts_path / possibly_commited_part; - if (!zk->exists(possibly_commited_part_path / "path")) + Coordination::Responses responses; + if (zk_client->tryMulti(ops, responses) != Coordination::Error::ZOK) { - break; + LOG_INFO(log, "All failure mechanism failed, will not try to update it"); + return; } - exported_paths.push_back(zk->get(possibly_commited_part_path / "path")); - } - - bool exported_all_parts = exported_paths.size() == part_names.size(); - if (!exported_all_parts) - { - return; - } - const auto destination_storage = DatabaseCatalog::instance().tryGetTable( - StorageID(QualifiedTableName::parseFromString(destination_storage_id)), - getContext()); - const auto transaction_id = zk->get(partition_export_path / "transaction_id"); - /// possibly need to add a check here to see if the table still exists - destination_storage->commitExportPartitionTransaction(transaction_id, partition_export, exported_paths, getContext()); - zk->tryCreate(partition_export_path / "status", "COMPLETED", zkutil::CreateMode::Persistent); - }; - - exportPartToTable( - part_name, - StorageID(QualifiedTableName::parseFromString(destination_storage_id)), - getContext(), - callback); + if (erase_entry) + { + std::lock_guard inner_lock(export_merge_tree_partition_mutex); + export_merge_tree_partition_task_entries.erase(key); + return; + } - /// for now, export only one part at a time - break; + } + }); } catch (...) { + /// failed to schedule the part export tryLogCurrentException(log, __PRETTY_FUNCTION__); - zookeeper->remove(owner_path); - /// for now, export only one part at a time - break; + /// best-effort to remove the lock (actually, we should make sure the lock is released..) + zk->tryRemove(fs::path(partition_path) / "parts" / part_to_export.value() / "lock"); } + } } @@ -5932,6 +6213,8 @@ void StorageReplicatedMergeTree::partialShutdown() queue_updating_task->deactivate(); mutations_updating_task->deactivate(); mutations_finalizing_task->deactivate(); + export_merge_tree_partition_updating_task->deactivate(); + export_merge_tree_partition_select_task->deactivate(); cleanup_thread.stop(); async_block_ids_cache.stop(); @@ -8117,6 +8400,32 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & const auto exports_path = fs::path(zookeeper_path) / "exports"; Coordination::Requests ops; + /* + exports + partition_id + target storage id: (or may be hash is safer?) + metadata (znode) <- immutable, every replica read it once to get full meta, znode mtime - is a timestamp + parition id + destination id + source replica + number of parts: 100 + list of parts: <- processed in strict order + 2020_0_0_1 + 2020_1_1_1 + ... + parts_to_do: 100 (znode) + exceptions_per_replica (znode) + replica1: + num_exceptions: 1 + last_exception (znode, znode mtime - is a timestamp of last exception) + part: + exception + parts/ + part_name/ <-- the value of that znode is pending initially, and finished later. + lock = ephemeral, when processing + replica: r1 + start_time: 123445 + */ + /// maybe this should go in initialization somewhere else if (!zookeeper->exists(exports_path)) { @@ -8133,93 +8442,77 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & ops.emplace_back(zkutil::makeCreateRequest(partition_exports_path, "", zkutil::CreateMode::Persistent)); - const Strings parts = zookeeper->getChildren(fs::path(replica_path) / "parts"); - const ActiveDataPartSet active_parts_set(format_version, parts); - const auto part_infos = active_parts_set.getPartInfos(); - std::vector parts_to_export; - for (const auto & part_info : part_infos) + auto data_parts_lock = lockParts(); + + const auto parts = getDataPartsVectorInPartitionForInternalUsage(MergeTreeDataPartState::Active, partition_id, &data_parts_lock); + + // const Strings parts = zookeeper->getChildren(fs::path(replica_path) / "parts"); + // const ActiveDataPartSet active_parts_set(format_version, parts); + // const auto part_infos = active_parts_set.getPartInfos(); + // std::vector parts_to_export; + // for (const auto & part : parts) + // { + // if (part_info.getPartitionId() == partition_id) + // { + // parts_to_export.push_back(part_info.getPartNameV1()); + // } + // } + + if (parts.empty()) { - if (part_info.getPartitionId() == partition_id) - { - parts_to_export.push_back(part_info.getPartNameV1()); - } + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition {} doesn't exist", partition_id); } - if (parts_to_export.empty()) + std::vector part_names; + for (const auto & part : parts) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition {} doesn't exist", partition_id); + part_names.push_back(part->name); } - /// somehow check if the list of parts is updated "enough" + /// TODO arthur somehow check if the list of parts is updated "enough" - /* - table_path/exports/partition_id - table_path/exports/partition_id/transaction_id - table_path/exports/partition_id/destination_storage_id - table_path/exports/partition_id/status - table_path/exports/partition_id/create_time - table_path/exports/partition_id/time_to_live_seconds - table_path/exports/partition_id/part_name_1 - table_path/exports/partition_id/part_name_1/status - table_path/exports/partition_id/part_name_1/owner - table_path/exports/partition_id/part_name_1/retry_count - table_path/exports/partition_id/part_name_1/max_retry_count - table_path/exports/partition_id/part_name_1/path - table_path/exports/partition_id/part_name_n - ... - */ + ExportReplicatedMergeTreePartitionManifest manifest; - const auto transaction_id = std::to_string(generateSnowflakeID()); + manifest.transaction_id = std::to_string(generateSnowflakeID()); + manifest.partition_id = partition_id; + manifest.destination_database = command.to_database; + manifest.destination_table = command.to_table; + manifest.source_replica = replica_name; + manifest.number_of_parts = part_names.size(); + manifest.parts = part_names; + manifest.create_time = time(nullptr); - const auto parts_path = fs::path(partition_exports_path) / "parts"; ops.emplace_back(zkutil::makeCreateRequest( - parts_path, "", + fs::path(partition_exports_path) / "metadata.json", + manifest.toJsonString(), zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest( - fs::path(partition_exports_path) / "transaction_id", - transaction_id, + fs::path(partition_exports_path) / "parts_to_do", + std::to_string(part_names.size()), zkutil::CreateMode::Persistent)); - + ops.emplace_back(zkutil::makeCreateRequest( - fs::path(partition_exports_path) / "destination_storage_id", - dest_storage->getStorageID().getFullTableName(), + fs::path(partition_exports_path) / "next_idx", + "0", zkutil::CreateMode::Persistent)); - /// status: IN_PROGRESS, COMPLETED, FAILED ops.emplace_back(zkutil::makeCreateRequest( - fs::path(partition_exports_path) / "status", - "IN_PROGRESS", + fs::path(partition_exports_path) / "exceptions_per_replica", + "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest( - fs::path(partition_exports_path) / "create_time", - std::to_string(time(nullptr)), + fs::path(partition_exports_path) / "parts", + "", zkutil::CreateMode::Persistent)); - + /// status: IN_PROGRESS, COMPLETED, FAILED ops.emplace_back(zkutil::makeCreateRequest( - fs::path(partition_exports_path) / "time_to_live_seconds", - std::to_string(3600), + fs::path(partition_exports_path) / "status", + "PENDING", zkutil::CreateMode::Persistent)); - for (const String & part_name : parts_to_export) - { - const String part_path = fs::path(parts_path) / part_name; - - ops.emplace_back(zkutil::makeCreateRequest(part_path, "", zkutil::CreateMode::Persistent)); - - ops.emplace_back(zkutil::makeCreateRequest( - fs::path(part_path) / "retry_count", - "0", - zkutil::CreateMode::Persistent)); - - ops.emplace_back(zkutil::makeCreateRequest( - fs::path(part_path) / "max_retry_count", - "3", - zkutil::CreateMode::Persistent)); - } - Coordination::Responses responses; Coordination::Error code = zookeeper->tryMulti(ops, responses); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 223240ac504c..05263ecd909c 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -510,8 +511,16 @@ class StorageReplicatedMergeTree final : public MergeTreeData /// A task that marks finished mutations as done. BackgroundSchedulePoolTaskHolder mutations_finalizing_task; + BackgroundSchedulePoolTaskHolder export_merge_tree_partition_updating_task; + + Coordination::WatchCallbackPtr export_merge_tree_partition_watch_callback; + + std::mutex export_merge_tree_partition_mutex; + BackgroundSchedulePoolTaskHolder export_merge_tree_partition_select_task; + + std::unordered_map export_merge_tree_partition_task_entries; /// A thread that removes old parts, log entries, and blocks. ReplicatedMergeTreeCleanupThread cleanup_thread; @@ -741,6 +750,9 @@ class StorageReplicatedMergeTree final : public MergeTreeData void selectPartsToExport(); + /// update in-memory list of partition exports + void exportMergeTreePartitionUpdatingTask(); + /** Write the selected parts to merge into the log, * Call when merge_selecting_mutex is locked. * Returns false if any part is not in ZK. From 659b30958466cef8ba4fa8dae3bfbcf6c7d49751 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 10 Oct 2025 07:19:24 -0300 Subject: [PATCH 03/58] able to export partition using two different replicas and upload commit file --- src/Storages/StorageReplicatedMergeTree.cpp | 79 +++++++++++++++++---- 1 file changed, 65 insertions(+), 14 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d2e03114585d..75f84c9dc5d6 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -118,6 +118,7 @@ #include #include "Interpreters/StorageID.h" +#include "QueryPipeline/QueryPlanResourceHolder.h" #include "Storages/ExportReplicatedMergeTreePartitionManifest.h" #include "Storages/ExportReplicatedMergeTreePartitionTaskEntry.h" #include @@ -4497,11 +4498,16 @@ void StorageReplicatedMergeTree::selectPartsToExport() auto complete_part_export = [&]( const std::string & export_partition_path, + const std::string & part_path, const std::string & part_status_path, const std::string & parts_to_do_path, const std::string & lock_path, const std::string & next_idx_path, const std::size_t next_idx_local, + const std::string & path_in_destination_storage_path, + const StoragePtr & destination_storage, + const std::string & transaction_id, + const std::string & partition_id, const ZooKeeperPtr & zk) -> bool { /// todo arthur is it possible to grab stats using a multi-op? @@ -4561,9 +4567,38 @@ void StorageReplicatedMergeTree::selectPartsToExport() ops.emplace_back(zkutil::makeSetRequest(part_status_path, "COMPLETED", -1)); ops.emplace_back(zkutil::makeRemoveRequest(lock_path, lock_stat.version)); ops.emplace_back(zkutil::makeSetRequest(parts_to_do_path, std::to_string(parts_to_do), parts_to_do_stat.version)); + ops.emplace_back(zkutil::makeCreateRequest(fs::path(part_path) / "finished_by", replica_name, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(fs::path(part_path) / "path_in_destination_storage", path_in_destination_storage_path, zkutil::CreateMode::Persistent)); if (parts_to_do == 0) { + /// loop over all parts under `/parts` and grab all paths in destination storage + Strings exported_paths; + const auto parts_path = fs::path(export_partition_path) / "parts"; + Strings parts = zk->getChildren(parts_path); + + for (const auto & part : parts) + { + std::string path_in_destination_storage; + const auto path_in_destination_storage_zk_path = fs::path(parts_path) / part / "path_in_destination_storage"; + + if (zk->tryGet(path_in_destination_storage_zk_path, path_in_destination_storage)) + { + exported_paths.push_back(path_in_destination_storage); + } + else + { + /// todo arthur what should I do here? + LOG_WARNING(log, "Failed to get path_in_destination_storage for part {} in export", part); + } + } + + LOG_INFO(log, "Collected {} exported paths for export", exported_paths.size()); + + /// manually add the export we just finished because it is not zk yet + exported_paths.push_back(path_in_destination_storage_path); + + destination_storage->commitExportPartitionTransaction(transaction_id, partition_id, exported_paths, getContext()); ops.emplace_back(zkutil::makeSetRequest(fs::path(export_partition_path) / "status", "COMPLETED", -1)); } @@ -4701,14 +4736,30 @@ void StorageReplicatedMergeTree::selectPartsToExport() std::lock_guard lock(export_merge_tree_partition_mutex); - for (const auto & [key, task_entry] : export_merge_tree_partition_task_entries) + for (auto & [key, task_entry] : export_merge_tree_partition_task_entries) { /// this sounds impossible, but just in case if (task_entry.parts_to_do == 0) { LOG_INFO(log, "Already completed, skipping"); continue; - } + } + + std::string parts_to_do_string; + if (!zk->tryGet(fs::path(exports_path) / key / "parts_to_do", parts_to_do_string)) + { + LOG_INFO(log, "Failed to get parts_to_do, skipping"); + continue; + } + + const auto parts_to_do = std::stoull(parts_to_do_string.c_str()); + task_entry.parts_to_do = parts_to_do; + + if (task_entry.parts_to_do == 0) + { + LOG_INFO(log, "Already completed, skipping"); + continue; + } const auto & manifest = task_entry.manifest; const auto & database = getContext()->resolveDatabase(manifest.destination_database); @@ -4737,6 +4788,9 @@ void StorageReplicatedMergeTree::selectPartsToExport() const auto part_to_export = try_to_acquire_a_part(manifest, partition_path, next_idx, zk); + const auto partition_id = manifest.partition_id; + const auto transaction_id = manifest.transaction_id; + if (part_to_export.has_value()) { try @@ -4745,20 +4799,26 @@ void StorageReplicatedMergeTree::selectPartsToExport() part_to_export.value(), destination_storage_id, getContext(), - [this, exports_path, key, part_to_export, complete_part_export, partition_path, next_idx_path, next_idx] + [this, partition_id, transaction_id, exports_path, key, part_to_export, complete_part_export, partition_path, next_idx_path, next_idx, destination_storage] (MergeTreeExportManifest::CompletionCallbackResult result) { const auto zk_client = getZooKeeper(); if (result.success) { + complete_part_export( partition_path, + fs::path(exports_path) / key / "parts" / part_to_export.value(), fs::path(exports_path) / key / "parts" / part_to_export.value() / "status", fs::path(exports_path) / key / "parts_to_do", fs::path(exports_path) / key / "parts" / part_to_export.value() / "lock", next_idx_path, next_idx, + result.relative_path_in_destination_storage, + destination_storage, + transaction_id, + partition_id, zk_client); /// maybe get up to date from complete_parts_export? @@ -4766,7 +4826,7 @@ void StorageReplicatedMergeTree::selectPartsToExport() export_merge_tree_partition_task_entries[key].parts_to_do--; if (export_merge_tree_partition_task_entries[key].parts_to_do == 0) - { + { export_merge_tree_partition_task_entries.erase(key); } } @@ -4776,7 +4836,6 @@ void StorageReplicatedMergeTree::selectPartsToExport() /// if above threshhold, fail the entire export - hopefully it is safe to do so :D /// I could also leave this for the cleanup thread, but will do it here for now. - bool erase_entry = false; Coordination::Requests ops; std::string retry_count_string; @@ -4787,8 +4846,7 @@ void StorageReplicatedMergeTree::selectPartsToExport() //// todo arthur unhardcode this if (retry_count > 3) { - ops.emplace_back(zkutil::makeRemoveRequest(partition_path, -1)); - erase_entry = true; + ops.emplace_back(zkutil::makeRemoveRecursiveRequest(partition_path, 1000)); } else { @@ -4810,13 +4868,6 @@ void StorageReplicatedMergeTree::selectPartsToExport() return; } - if (erase_entry) - { - std::lock_guard inner_lock(export_merge_tree_partition_mutex); - export_merge_tree_partition_task_entries.erase(key); - return; - } - } }); } From d3bb8202ba33e65ab02bb0e2fd0f10b09a74527c Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 13 Oct 2025 08:03:53 -0300 Subject: [PATCH 04/58] checkpoint --- src/Storages/StorageReplicatedMergeTree.cpp | 166 ++++++++++---------- 1 file changed, 83 insertions(+), 83 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 75f84c9dc5d6..ca528e8305dd 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4377,119 +4377,119 @@ void StorageReplicatedMergeTree::mutationsFinalizingTask() } } -// static void exportPartitionCallback( -// MergeTreeExportManifest::CompletionCallbackResult result, -// zkutil::ZooKeeperPtr & zookeeper, -// const String & partition_export, -// const String & part_name) -// { -// if (!result.success) -// { -// LOG_INFO(log, "Failed to export partition {} part {}, releasing lock", partition_export, part_name); -// zookeeper->remove(owner_path); -// return; -// } - -// zookeeper->create(part_path / "path", result.relative_path_in_destination_storage, zkutil::CreateMode::Persistent); -// LOG_INFO(log, "Exported partition {} part {} to {}", partition_export, part_name, result.relative_path_in_destination_storage); - -// /// loop over all parts again and check if they all have the path set, if so, mark the export as done -// bool completed = true; -// Strings exported_paths; -// for (const auto & possibly_commited_part : part_names) -// { -// /// todo arthur is this a case of "try to write as opposed to try to read because it might be stale?" -// const auto possibly_commited_part_path = parts_path / possibly_commited_part; -// if (!zookeeper->exists(possibly_commited_part_path / "path")) -// { -// completed = false; -// break; -// } -// exported_paths.push_back(zookeeper->get(possibly_commited_part_path / "path")); -// } - -// if (completed) -// { - -// const auto destination_storage = DatabaseCatalog::instance().tryGetTable( -// StorageID(QualifiedTableName::parseFromString(destination_storage_id)), -// getContext()); -// const auto transaction_id = zookeeper->get(partition_export_path / "transaction_id"); -// /// possibly need to add a check here to see if the table still exists -// destination_storage->commitExportPartitionTransaction(transaction_id, partition_export, exported_paths, getContext()); -// zookeeper->set(partition_export_path / "status", "COMPLETED"); -// } -// } - void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() { std::lock_guard lock(export_merge_tree_partition_mutex); auto zk = getZooKeeper(); - const auto exports_path = fs::path(zookeeper_path) / "exports"; - /// why do I need the stat? - Coordination::Stat stat; - const auto partition_exports_in_zk = zk->getChildrenWatch(exports_path, &stat, export_merge_tree_partition_watch_callback); - const auto partition_exports_in_zk_set = std::unordered_set(partition_exports_in_zk.begin(), partition_exports_in_zk.end()); + const std::string exports_path = fs::path(zookeeper_path) / "exports"; + const std::string cleanup_lock_path = fs::path(zookeeper_path) / "exports_cleanup_lock"; - std::unordered_set local_export_partition_entries; - for (const auto & entry : export_merge_tree_partition_task_entries) - local_export_partition_entries.insert(entry.first); + bool cleanup_lock_acquired = zk->tryCreate(cleanup_lock_path, "", zkutil::CreateMode::Ephemeral) == Coordination::Error::ZOK; - std::unordered_set new_entries; - std::unordered_set removed_entries; - - for (const auto & partition_export_task_entry : local_export_partition_entries) + if (cleanup_lock_acquired) { - if (!partition_exports_in_zk_set.contains(partition_export_task_entry)) - removed_entries.insert(partition_export_task_entry); + LOG_INFO(log, "Cleanup lock acquired, will remove stale entries"); } - for (const auto & partition_export : partition_exports_in_zk_set) - { - if (!local_export_partition_entries.contains(partition_export)) - new_entries.insert(partition_export); - } + Coordination::Stat stat; + const auto children = zk->getChildrenWatch(exports_path, &stat, export_merge_tree_partition_watch_callback); + const std::unordered_set zk_children(children.begin(), children.end()); - /// remove the removed entries from the local set - for (const auto & entry : removed_entries) - export_merge_tree_partition_task_entries.erase(entry); + const auto now = time(nullptr); - // add the new entries to the local set - for (const auto & entry_key : new_entries) + /// Load new entries + /// If we have the cleanup lock, also remove stale entries from zk and local + for (const auto & key : zk_children) { - // get entry from zk - const auto entry_path = fs::path(exports_path) / entry_key; - const auto metadata_entry_path = fs::path(entry_path) / "metadata.json"; - const auto parts_to_do_path = fs::path(entry_path) / "parts_to_do"; + if (!cleanup_lock_acquired && export_merge_tree_partition_task_entries.contains(key)) + continue; - std::string parts_to_do_string; - if (!zk->tryGet(parts_to_do_path, parts_to_do_string)) + const std::string entry_path = fs::path(exports_path) / key; + + std::string metadata_json; + if (!zk->tryGet(fs::path(entry_path) / "metadata.json", metadata_json)) { - LOG_INFO(log, "Skipping..."); + LOG_INFO(log, "Skipping {}: missing metadata.json", key); continue; } - const auto parts_to_do = std::stoull(parts_to_do_string.c_str()); + std::string status; + if (!zk->tryGet(fs::path(entry_path) / "status", status)) + { + LOG_INFO(log, "Skipping {}: missing status", key); + continue; + } - if (parts_to_do == 0) + bool is_not_pending = status != "PENDING"; + + const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + + if (cleanup_lock_acquired) + { + bool has_expired = metadata.create_time < now - 45; + + if (has_expired && is_not_pending) + { + zk->tryRemoveRecursive(fs::path(entry_path)); + export_merge_tree_partition_task_entries.erase(key); + LOG_INFO(log, "Removed {}: expired", key); + continue; + } + } + + if (is_not_pending) { - LOG_INFO(log, "Skipping... Parts to do is 0, will not load it"); + LOG_INFO(log, "Skipping {}: status is not PENDING", key); continue; } - std::string metadata_json; - if (!zk->tryGet(metadata_entry_path, metadata_json)) + if (export_merge_tree_partition_task_entries.contains(key)) { - LOG_INFO(log, "Skipping..."); + LOG_INFO(log, "Skipping {}: already exists", key); continue; } - const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + std::string parts_to_do_str; + if (!zk->tryGet(fs::path(entry_path) / "parts_to_do", parts_to_do_str)) + { + LOG_INFO(log, "Skipping {}: no parts_to_do", key); + continue; + } - export_merge_tree_partition_task_entries[entry_key] = ExportReplicatedMergeTreePartitionTaskEntry {metadata, parts_to_do}; + uint64_t parts_to_do = 0; + try + { + parts_to_do = std::stoull(parts_to_do_str); + } + catch (...) + { + LOG_WARNING(log, "Skipping {}: invalid parts_to_do='{}'", key, parts_to_do_str); + continue; + } + + if (parts_to_do == 0) + { + LOG_INFO(log, "Skipping {}: parts_to_do is 0", key); + continue; + } + + export_merge_tree_partition_task_entries.emplace( + key, + ExportReplicatedMergeTreePartitionTaskEntry{std::move(metadata), parts_to_do}); + } + + /// Remove entries that were deleted by someone else + std::erase_if(export_merge_tree_partition_task_entries, + [&](auto const & kv) { return !zk_children.contains(kv.first); }); + + if (cleanup_lock_acquired) + { + zk->tryRemove(cleanup_lock_path); } + + export_merge_tree_partition_updating_task->scheduleAfter(30 * 1000); } void StorageReplicatedMergeTree::selectPartsToExport() From 444e0eeb2f0affac2ebb166d40503c7f0ae62a64 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 13 Oct 2025 09:16:34 -0300 Subject: [PATCH 05/58] some changes --- src/Storages/StorageReplicatedMergeTree.cpp | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ca528e8305dd..c4abd8b1c3bf 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4870,6 +4870,9 @@ void StorageReplicatedMergeTree::selectPartsToExport() } }); + + /// managed to schedule a task, re-run immediately to pick up more tasks if possible + export_merge_tree_partition_select_task->schedule(); } catch (...) { @@ -4878,12 +4881,15 @@ void StorageReplicatedMergeTree::selectPartsToExport() /// best-effort to remove the lock (actually, we should make sure the lock is released..) zk->tryRemove(fs::path(partition_path) / "parts" / part_to_export.value() / "lock"); + + /// re-run after some time + export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); } } } - export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); + // export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); } @@ -8419,8 +8425,9 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & "Exporting merge tree part is experimental. Set `allow_experimental_export_merge_tree_part` to enable it"); } - String dest_database = query_context->resolveDatabase(command.to_database); - auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, command.to_table}, query_context); + const auto dest_database = query_context->resolveDatabase(command.to_database); + const auto dest_table = command.to_table; + auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, dest_table}, query_context); if (dest_storage->getStorageID() == this->getStorageID()) { @@ -8526,8 +8533,8 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & manifest.transaction_id = std::to_string(generateSnowflakeID()); manifest.partition_id = partition_id; - manifest.destination_database = command.to_database; - manifest.destination_table = command.to_table; + manifest.destination_database = dest_database; + manifest.destination_table = dest_table; manifest.source_replica = replica_name; manifest.number_of_parts = part_names.size(); manifest.parts = part_names; From 35c6cca0ad219a30de4424d57ed1dd54fb8a587e Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 13 Oct 2025 10:27:05 -0300 Subject: [PATCH 06/58] add a silly test --- src/Storages/StorageReplicatedMergeTree.cpp | 4 +- ...3604_export_merge_tree_partition.reference | 31 +++++++++++ .../03604_export_merge_tree_partition.sh | 54 +++++++++++++++++++ 3 files changed, 86 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03604_export_merge_tree_partition.reference create mode 100755 tests/queries/0_stateless/03604_export_merge_tree_partition.sh diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c4abd8b1c3bf..bca31deaf353 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4490,6 +4490,7 @@ void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() } export_merge_tree_partition_updating_task->scheduleAfter(30 * 1000); + export_merge_tree_partition_select_task->schedule(); } void StorageReplicatedMergeTree::selectPartsToExport() @@ -4870,9 +4871,6 @@ void StorageReplicatedMergeTree::selectPartsToExport() } }); - - /// managed to schedule a task, re-run immediately to pick up more tasks if possible - export_merge_tree_partition_select_task->schedule(); } catch (...) { diff --git a/tests/queries/0_stateless/03604_export_merge_tree_partition.reference b/tests/queries/0_stateless/03604_export_merge_tree_partition.reference new file mode 100644 index 000000000000..97e17ae5aeec --- /dev/null +++ b/tests/queries/0_stateless/03604_export_merge_tree_partition.reference @@ -0,0 +1,31 @@ + +Select from source table +1 2020 +2 2020 +3 2020 +4 2021 +5 2021 +6 2022 +7 2022 +Select from destination table +1 2020 +2 2020 +3 2020 +4 2021 +Export partition 2022 +Select from destination table again +1 2020 +2 2020 +3 2020 +4 2021 +5 2021 +6 2022 +7 2022 +---- Data in roundtrip ReplicatedMergeTree table (should match s3_table) +1 2020 +2 2020 +3 2020 +4 2021 +5 2021 +6 2022 +7 2022 diff --git a/tests/queries/0_stateless/03604_export_merge_tree_partition.sh b/tests/queries/0_stateless/03604_export_merge_tree_partition.sh new file mode 100755 index 000000000000..1198281afc72 --- /dev/null +++ b/tests/queries/0_stateless/03604_export_merge_tree_partition.sh @@ -0,0 +1,54 @@ +#!/usr/bin/env bash +# Tags: replica, no-parallel, no-replicated-database + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +rmt_table="rmt_table_${RANDOM}" +s3_table="s3_table_${RANDOM}" +rmt_table_roundtrip="rmt_table_roundtrip_${RANDOM}" + +query() { + $CLICKHOUSE_CLIENT --query "$1" +} + +query "DROP TABLE IF EXISTS $rmt_table, $s3_table, $rmt_table_roundtrip" + +query "CREATE TABLE $rmt_table (id UInt64, year UInt16) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/$rmt_table', 'replica1') PARTITION BY year ORDER BY tuple()" +query "CREATE TABLE $s3_table (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$s3_table', format=Parquet, partition_strategy='hive') PARTITION BY year" + +query "INSERT INTO $rmt_table VALUES (1, 2020), (2, 2020), (4, 2021)" + +query "INSERT INTO $rmt_table VALUES (3, 2020), (5, 2021)" + +query "INSERT INTO $rmt_table VALUES (6, 2022), (7, 2022)" + +query "ALTER TABLE $rmt_table EXPORT PARTITION ID '2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +query "ALTER TABLE $rmt_table EXPORT PARTITION ID '2021' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +# todo poll some kind of status +sleep 15 + +echo "Select from source table" +query "SELECT * FROM $rmt_table ORDER BY id" + +echo "Select from destination table" +query "SELECT * FROM $s3_table ORDER BY id" + +echo "Export partition 2022" +query "ALTER TABLE $rmt_table EXPORT PARTITION ID '2022' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +# todo poll some kind of status +sleep 5 + +echo "Select from destination table again" +query "SELECT * FROM $s3_table ORDER BY id" + +query "CREATE TABLE $rmt_table_roundtrip ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/$rmt_table_roundtrip', 'replica1') PARTITION BY year ORDER BY tuple() AS SELECT * FROM $s3_table" + +echo "---- Data in roundtrip ReplicatedMergeTree table (should match s3_table)" +query "SELECT * FROM $rmt_table_roundtrip ORDER BY id" + +query "DROP TABLE IF EXISTS $rmt_table, $s3_table, $rmt_table_roundtrip" \ No newline at end of file From b884fd37dc3ef6f68086e51e6afd1bff67e6c7c0 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 13 Oct 2025 11:20:32 -0300 Subject: [PATCH 07/58] hold parts references to prevent deletion --- .../ExportReplicatedMergeTreePartitionTaskEntry.h | 8 ++++++++ src/Storages/StorageReplicatedMergeTree.cpp | 12 +++++++++++- 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h index 122c29b1b244..bdaef2ec3fa7 100644 --- a/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h +++ b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h @@ -1,13 +1,21 @@ #pragma once #include +#include namespace DB { struct ExportReplicatedMergeTreePartitionTaskEntry { + using DataPartPtr = std::shared_ptr; ExportReplicatedMergeTreePartitionManifest manifest; std::size_t parts_to_do; + /// References to the parts that should be exported + /// This is used to prevent the parts from being deleted before finishing the export operation + /// It does not mean this replica will export all the parts + /// There is also a chance this replica does not contain a given part and it is totally ok. + std::vector part_references; }; + } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index bca31deaf353..45260de224fa 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4475,9 +4475,19 @@ void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() continue; } + std::vector part_references; + + for (const auto & part_name : metadata.parts) + { + if (const auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated})) + { + part_references.push_back(part); + } + } + export_merge_tree_partition_task_entries.emplace( key, - ExportReplicatedMergeTreePartitionTaskEntry{std::move(metadata), parts_to_do}); + ExportReplicatedMergeTreePartitionTaskEntry {metadata, parts_to_do, std::move(part_references)}); } /// Remove entries that were deleted by someone else From c7493cb5860e7bc68154c6c8d47cff0f63e2754f Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 13 Oct 2025 13:51:49 -0300 Subject: [PATCH 08/58] fix a few tests --- src/Storages/MergeTree/MergeTreeData.cpp | 6 ++++-- tests/queries/0_stateless/01271_show_privileges.reference | 1 + .../02221_system_zookeeper_unrestricted.reference | 2 ++ .../02221_system_zookeeper_unrestricted_like.reference | 1 + 4 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 804e529bd261..724949d5e1b6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6206,9 +6206,11 @@ void MergeTreeData::exportPartToTable(const PartitionCommand & command, ContextP "Exporting merge tree part is experimental. Set `allow_experimental_export_merge_tree_part` to enable it"); } - auto part_name = command.partition->as().value.safeGet(); + const auto part_name = command.partition->as().value.safeGet(); - exportPartToTable(part_name, StorageID{command.to_database, command.to_table}, query_context); + const auto database_name = query_context->resolveDatabase(command.to_database); + + exportPartToTable(part_name, StorageID{database_name, command.to_table}, query_context); } void MergeTreeData::exportPartToTable( diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 1bdf1cd250fd..a4b9d2dafd9d 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -44,6 +44,7 @@ ALTER MATERIALIZE TTL ['MATERIALIZE TTL'] TABLE ALTER TABLE ALTER SETTINGS ['ALTER SETTING','ALTER MODIFY SETTING','MODIFY SETTING','RESET SETTING'] TABLE ALTER TABLE ALTER MOVE PARTITION ['ALTER MOVE PART','MOVE PARTITION','MOVE PART'] TABLE ALTER TABLE ALTER EXPORT PART ['ALTER EXPORT PART','EXPORT PART'] TABLE ALTER TABLE +ALTER EXPORT PARTITION ['ALTER EXPORT PARTITION','EXPORT PARTITION'] TABLE ALTER TABLE ALTER FETCH PARTITION ['ALTER FETCH PART','FETCH PARTITION'] TABLE ALTER TABLE ALTER FREEZE PARTITION ['FREEZE PARTITION','UNFREEZE'] TABLE ALTER TABLE ALTER UNLOCK SNAPSHOT ['UNLOCK SNAPSHOT'] TABLE ALTER TABLE diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference index eea09bd06035..1b4fcf850453 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference @@ -18,6 +18,8 @@ columns columns creator_info creator_info +exports +exports failed_parts failed_parts flags diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference index 0d6c21be132f..8e69bdac72d7 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference @@ -49,6 +49,7 @@ blocks columns columns creator_info +exports failed_parts flags host From f4f9d5227a5f5e9400160d617f2c0279660dfc32 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 14 Oct 2025 14:59:12 -0300 Subject: [PATCH 09/58] try to fix integ test failure and fix failure handling --- src/Storages/MergeTree/MergeTreeData.cpp | 15 +- .../MergeTree/MergeTreeExportManifest.h | 17 +- src/Storages/StorageReplicatedMergeTree.cpp | 569 ++++++++++-------- 3 files changed, 339 insertions(+), 262 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 724949d5e1b6..4668157c882d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6324,7 +6324,7 @@ void MergeTreeData::exportPartToTableImpl( export_manifests.erase(manifest); if (manifest.completion_callback) - manifest.completion_callback({}); + manifest.completion_callback(MergeTreeExportManifest::CompletionCallbackResult::createFailure(e.message())); return; } @@ -6398,6 +6398,13 @@ void MergeTreeData::exportPartToTableImpl( CompletedPipelineExecutor exec(pipeline); exec.execute(); + volatile bool x = true; + + if (x) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Some issue"); + } + std::lock_guard inner_lock(export_manifests_mutex); writePartLog( PartLogElement::Type::EXPORT_PART, @@ -6416,9 +6423,9 @@ void MergeTreeData::exportPartToTableImpl( ProfileEvents::increment(ProfileEvents::PartsExportTotalMilliseconds, static_cast((*exports_list_entry)->elapsed * 1000)); if (manifest.completion_callback) - manifest.completion_callback({true, destination_file_path}); + manifest.completion_callback(MergeTreeExportManifest::CompletionCallbackResult::createSuccess(destination_file_path)); } - catch (...) + catch (const Exception & e) { tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("while exporting the part {}. User should retry.", manifest.data_part->name)); @@ -6439,7 +6446,7 @@ void MergeTreeData::exportPartToTableImpl( export_manifests.erase(manifest); if (manifest.completion_callback) - manifest.completion_callback({}); + manifest.completion_callback(MergeTreeExportManifest::CompletionCallbackResult::createFailure(e.message())); throw; } diff --git a/src/Storages/MergeTree/MergeTreeExportManifest.h b/src/Storages/MergeTree/MergeTreeExportManifest.h index 0ad93b41798b..bcc480c8507e 100644 --- a/src/Storages/MergeTree/MergeTreeExportManifest.h +++ b/src/Storages/MergeTree/MergeTreeExportManifest.h @@ -10,11 +10,26 @@ struct MergeTreeExportManifest struct CompletionCallbackResult { + private: + CompletionCallbackResult(bool success_, const String & relative_path_in_destination_storage_, const String & exception_) + : success(success_), relative_path_in_destination_storage(relative_path_in_destination_storage_), exception(exception_) {} + public: + + static CompletionCallbackResult createSuccess(const String & relative_path_in_destination_storage_) + { + return CompletionCallbackResult(true, relative_path_in_destination_storage_, ""); + } + + static CompletionCallbackResult createFailure(const String & exception_) + { + return CompletionCallbackResult(false, "", exception_); + } + bool success = false; String relative_path_in_destination_storage; + String exception; }; - MergeTreeExportManifest( const StorageID & destination_storage_id_, const DataPartPtr & data_part_, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 45260de224fa..995dccf93042 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1586,7 +1586,7 @@ bool StorageReplicatedMergeTree::removeTableNodesFromZooKeeper(zkutil::ZooKeeper /// NOTE /block_numbers/ actually is not flat, because /block_numbers// may have ephemeral children, /// but we assume that all ephemeral block locks are already removed when table is being dropped. - static constexpr std::array flat_nodes = {"block_numbers", "blocks", "async_blocks", "leader_election", "log", "mutations", "pinned_part_uuids"}; + static constexpr std::array flat_nodes = {"block_numbers", "blocks", "async_blocks", "leader_election", "log", "mutations", "pinned_part_uuids", "exports_cleanup_lock"}; /// First try to remove paths that are known to be flat for (const auto * node : flat_nodes) @@ -4379,128 +4379,137 @@ void StorageReplicatedMergeTree::mutationsFinalizingTask() void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() { - std::lock_guard lock(export_merge_tree_partition_mutex); - - auto zk = getZooKeeper(); - - const std::string exports_path = fs::path(zookeeper_path) / "exports"; - const std::string cleanup_lock_path = fs::path(zookeeper_path) / "exports_cleanup_lock"; - - bool cleanup_lock_acquired = zk->tryCreate(cleanup_lock_path, "", zkutil::CreateMode::Ephemeral) == Coordination::Error::ZOK; - - if (cleanup_lock_acquired) - { - LOG_INFO(log, "Cleanup lock acquired, will remove stale entries"); - } - - Coordination::Stat stat; - const auto children = zk->getChildrenWatch(exports_path, &stat, export_merge_tree_partition_watch_callback); - const std::unordered_set zk_children(children.begin(), children.end()); - - const auto now = time(nullptr); - - /// Load new entries - /// If we have the cleanup lock, also remove stale entries from zk and local - for (const auto & key : zk_children) + try { - if (!cleanup_lock_acquired && export_merge_tree_partition_task_entries.contains(key)) - continue; - - const std::string entry_path = fs::path(exports_path) / key; - - std::string metadata_json; - if (!zk->tryGet(fs::path(entry_path) / "metadata.json", metadata_json)) - { - LOG_INFO(log, "Skipping {}: missing metadata.json", key); - continue; - } + std::lock_guard lock(export_merge_tree_partition_mutex); - std::string status; - if (!zk->tryGet(fs::path(entry_path) / "status", status)) + auto zk = getZooKeeper(); + + const std::string exports_path = fs::path(zookeeper_path) / "exports"; + const std::string cleanup_lock_path = fs::path(zookeeper_path) / "exports_cleanup_lock"; + + bool cleanup_lock_acquired = zk->tryCreate(cleanup_lock_path, "", zkutil::CreateMode::Ephemeral) == Coordination::Error::ZOK; + + if (cleanup_lock_acquired) { - LOG_INFO(log, "Skipping {}: missing status", key); - continue; + LOG_INFO(log, "Cleanup lock acquired, will remove stale entries"); } - - bool is_not_pending = status != "PENDING"; - - const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); - - if (cleanup_lock_acquired) + + Coordination::Stat stat; + const auto children = zk->getChildrenWatch(exports_path, &stat, export_merge_tree_partition_watch_callback); + const std::unordered_set zk_children(children.begin(), children.end()); + + const auto now = time(nullptr); + + /// Load new entries + /// If we have the cleanup lock, also remove stale entries from zk and local + for (const auto & key : zk_children) { - bool has_expired = metadata.create_time < now - 45; - - if (has_expired && is_not_pending) + if (!cleanup_lock_acquired && export_merge_tree_partition_task_entries.contains(key)) + continue; + + const std::string entry_path = fs::path(exports_path) / key; + + std::string metadata_json; + if (!zk->tryGet(fs::path(entry_path) / "metadata.json", metadata_json)) { - zk->tryRemoveRecursive(fs::path(entry_path)); - export_merge_tree_partition_task_entries.erase(key); - LOG_INFO(log, "Removed {}: expired", key); + LOG_INFO(log, "Skipping {}: missing metadata.json", key); continue; } - } - - if (is_not_pending) - { - LOG_INFO(log, "Skipping {}: status is not PENDING", key); - continue; - } - - if (export_merge_tree_partition_task_entries.contains(key)) - { - LOG_INFO(log, "Skipping {}: already exists", key); - continue; - } - - std::string parts_to_do_str; - if (!zk->tryGet(fs::path(entry_path) / "parts_to_do", parts_to_do_str)) - { - LOG_INFO(log, "Skipping {}: no parts_to_do", key); - continue; - } - - uint64_t parts_to_do = 0; - try - { - parts_to_do = std::stoull(parts_to_do_str); - } - catch (...) - { - LOG_WARNING(log, "Skipping {}: invalid parts_to_do='{}'", key, parts_to_do_str); - continue; - } - - if (parts_to_do == 0) - { - LOG_INFO(log, "Skipping {}: parts_to_do is 0", key); - continue; - } - - std::vector part_references; - - for (const auto & part_name : metadata.parts) - { - if (const auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated})) + + std::string status; + if (!zk->tryGet(fs::path(entry_path) / "status", status)) + { + LOG_INFO(log, "Skipping {}: missing status", key); + continue; + } + + bool is_not_pending = status != "PENDING"; + + const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + + if (cleanup_lock_acquired) + { + bool has_expired = metadata.create_time < now - 45; + + if (has_expired && is_not_pending) + { + zk->tryRemoveRecursive(fs::path(entry_path)); + export_merge_tree_partition_task_entries.erase(key); + LOG_INFO(log, "Removed {}: expired", key); + continue; + } + } + + if (is_not_pending) + { + LOG_INFO(log, "Skipping {}: status is not PENDING", key); + continue; + } + + if (export_merge_tree_partition_task_entries.contains(key)) + { + LOG_INFO(log, "Skipping {}: already exists", key); + continue; + } + + std::string parts_to_do_str; + if (!zk->tryGet(fs::path(entry_path) / "parts_to_do", parts_to_do_str)) + { + LOG_INFO(log, "Skipping {}: no parts_to_do", key); + continue; + } + + uint64_t parts_to_do = 0; + try + { + parts_to_do = std::stoull(parts_to_do_str); + } + catch (...) { - part_references.push_back(part); + LOG_WARNING(log, "Skipping {}: invalid parts_to_do='{}'", key, parts_to_do_str); + continue; } + + if (parts_to_do == 0) + { + LOG_INFO(log, "Skipping {}: parts_to_do is 0", key); + continue; + } + + std::vector part_references; + + for (const auto & part_name : metadata.parts) + { + if (const auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated})) + { + part_references.push_back(part); + } + } + + export_merge_tree_partition_task_entries.emplace( + key, + ExportReplicatedMergeTreePartitionTaskEntry {metadata, parts_to_do, std::move(part_references)}); + } + + /// Remove entries that were deleted by someone else + std::erase_if(export_merge_tree_partition_task_entries, + [&](auto const & kv) { return !zk_children.contains(kv.first); }); + + if (cleanup_lock_acquired) + { + zk->tryRemove(cleanup_lock_path); } - export_merge_tree_partition_task_entries.emplace( - key, - ExportReplicatedMergeTreePartitionTaskEntry {metadata, parts_to_do, std::move(part_references)}); + export_merge_tree_partition_select_task->schedule(); } - - /// Remove entries that were deleted by someone else - std::erase_if(export_merge_tree_partition_task_entries, - [&](auto const & kv) { return !zk_children.contains(kv.first); }); - - if (cleanup_lock_acquired) + catch (...) { - zk->tryRemove(cleanup_lock_path); + tryLogCurrentException(log, __PRETTY_FUNCTION__); } + export_merge_tree_partition_updating_task->scheduleAfter(30 * 1000); - export_merge_tree_partition_select_task->schedule(); } void StorageReplicatedMergeTree::selectPartsToExport() @@ -4626,7 +4635,6 @@ void StorageReplicatedMergeTree::selectPartsToExport() }; auto lock_part = [&]( - const std::string & export_partition_path, const std::string & part_path, const std::string & status_path, const std::string & lock_path, @@ -4652,20 +4660,20 @@ void StorageReplicatedMergeTree::selectPartsToExport() return false; } - std::string parts_to_do_zk; - if (!zk->tryGet(fs::path(export_partition_path) / "parts_to_do", parts_to_do_zk)) - { - LOG_INFO(log, "Failed to get parts_to_do, skipping"); - return false; - } + // std::string parts_to_do_zk; + // if (!zk->tryGet(fs::path(export_partition_path) / "parts_to_do", parts_to_do_zk)) + // { + // LOG_INFO(log, "Failed to get parts_to_do, skipping"); + // return false; + // } - const auto parts_to_do = std::stoull(parts_to_do_zk.c_str()); + // const auto parts_to_do = std::stoull(parts_to_do_zk.c_str()); - if (parts_to_do == 0) - { - LOG_INFO(log, "Skipping... Parts to do is 0, maybe someone else already completed it?"); - return false; - } + // if (parts_to_do == 0) + // { + // LOG_INFO(log, "Skipping... Parts to do is 0, maybe someone else already completed it?"); + // return false; + // } /// only try to lock it if the status is still PENDING /// if we did not check for status = pending, chances are some other replica completed and released the lock in the meantime @@ -4713,7 +4721,7 @@ void StorageReplicatedMergeTree::selectPartsToExport() continue; } - if (lock_part(partition_export_path, part_path, status_path, lock_path, replica_name, zk)) + if (lock_part(part_path, status_path, lock_path, replica_name, zk)) { return manifest.parts[i]; } @@ -4734,7 +4742,7 @@ void StorageReplicatedMergeTree::selectPartsToExport() continue; } - if (lock_part(partition_export_path, part_path, status_path, lock_path, replica_name, zk)) + if (lock_part(part_path, status_path, lock_path, replica_name, zk)) { return manifest.parts[i]; } @@ -4743,160 +4751,207 @@ void StorageReplicatedMergeTree::selectPartsToExport() return std::nullopt; }; - const auto zk = getZooKeeper(); - - std::lock_guard lock(export_merge_tree_partition_mutex); - - for (auto & [key, task_entry] : export_merge_tree_partition_task_entries) - { - /// this sounds impossible, but just in case - if (task_entry.parts_to_do == 0) - { - LOG_INFO(log, "Already completed, skipping"); - continue; - } + try { + const auto zk = getZooKeeper(); - std::string parts_to_do_string; - if (!zk->tryGet(fs::path(exports_path) / key / "parts_to_do", parts_to_do_string)) + std::lock_guard lock(export_merge_tree_partition_mutex); + + for (auto & [key, task_entry] : export_merge_tree_partition_task_entries) { - LOG_INFO(log, "Failed to get parts_to_do, skipping"); - continue; - } + /// this sounds impossible, but just in case + if (task_entry.parts_to_do == 0) + { + LOG_INFO(log, "Already completed, skipping"); + continue; + } + + std::string parts_to_do_string; + if (!zk->tryGet(fs::path(exports_path) / key / "parts_to_do", parts_to_do_string)) + { + LOG_INFO(log, "Failed to get parts_to_do, skipping"); + continue; + } + + const auto parts_to_do = std::stoull(parts_to_do_string.c_str()); + task_entry.parts_to_do = parts_to_do; + + if (task_entry.parts_to_do == 0) + { + LOG_INFO(log, "Already completed, skipping"); + continue; + } + + std::string status; + if (!zk->tryGet(fs::path(exports_path) / key / "status", status)) + { + LOG_INFO(log, "Failed to get status, skipping"); + continue; + } + + if (status != "PENDING") + { + LOG_INFO(log, "Skipping... Status is not PENDING"); + continue; + } + + const auto & manifest = task_entry.manifest; + const auto & database = getContext()->resolveDatabase(manifest.destination_database); + const auto & table = manifest.destination_table; + + const auto destination_storage_id = StorageID(QualifiedTableName {database, table}); + + const auto destination_storage = DatabaseCatalog::instance().tryGetTable(destination_storage_id, getContext()); - const auto parts_to_do = std::stoull(parts_to_do_string.c_str()); - task_entry.parts_to_do = parts_to_do; - - if (task_entry.parts_to_do == 0) - { - LOG_INFO(log, "Already completed, skipping"); - continue; - } - - const auto & manifest = task_entry.manifest; - const auto & database = getContext()->resolveDatabase(manifest.destination_database); - const auto & table = manifest.destination_table; - - const auto destination_storage_id = StorageID(QualifiedTableName {database, table}); - - const auto destination_storage = DatabaseCatalog::instance().tryGetTable(destination_storage_id, getContext()); + if (!destination_storage) + { + LOG_INFO(log, "Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); + continue; + } - if (!destination_storage) - { - LOG_INFO(log, "Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); - continue; - } - - const auto partition_path = fs::path(exports_path) / key; - const auto next_idx_path = fs::path(partition_path) / "next_idx"; - std::string next_idx_string; - if (!zk->tryGet(next_idx_path, next_idx_string)) - { - LOG_INFO(log, "Failed to get next_idx, skipping"); - continue; - } - - const auto next_idx = std::stoull(next_idx_string.c_str()); - - const auto part_to_export = try_to_acquire_a_part(manifest, partition_path, next_idx, zk); - - const auto partition_id = manifest.partition_id; - const auto transaction_id = manifest.transaction_id; - - if (part_to_export.has_value()) - { - try + const auto partition_path = fs::path(exports_path) / key; + const auto next_idx_path = fs::path(partition_path) / "next_idx"; + std::string next_idx_string; + if (!zk->tryGet(next_idx_path, next_idx_string)) + { + LOG_INFO(log, "Failed to get next_idx, skipping"); + continue; + } + + const auto next_idx = std::stoull(next_idx_string.c_str()); + + const auto part_to_export = try_to_acquire_a_part(manifest, partition_path, next_idx, zk); + + const auto partition_id = manifest.partition_id; + const auto transaction_id = manifest.transaction_id; + + if (part_to_export.has_value()) { - exportPartToTable( - part_to_export.value(), - destination_storage_id, - getContext(), - [this, partition_id, transaction_id, exports_path, key, part_to_export, complete_part_export, partition_path, next_idx_path, next_idx, destination_storage] - (MergeTreeExportManifest::CompletionCallbackResult result) + try { - - const auto zk_client = getZooKeeper(); - if (result.success) + exportPartToTable( + part_to_export.value(), + destination_storage_id, + getContext(), + [this, partition_id, transaction_id, exports_path, key, part_to_export, complete_part_export, partition_path, next_idx_path, next_idx, destination_storage] + (MergeTreeExportManifest::CompletionCallbackResult result) { - - complete_part_export( - partition_path, - fs::path(exports_path) / key / "parts" / part_to_export.value(), - fs::path(exports_path) / key / "parts" / part_to_export.value() / "status", - fs::path(exports_path) / key / "parts_to_do", - fs::path(exports_path) / key / "parts" / part_to_export.value() / "lock", - next_idx_path, - next_idx, - result.relative_path_in_destination_storage, - destination_storage, - transaction_id, - partition_id, - zk_client); - - /// maybe get up to date from complete_parts_export? - std::lock_guard inner_lock(export_merge_tree_partition_mutex); - export_merge_tree_partition_task_entries[key].parts_to_do--; - - if (export_merge_tree_partition_task_entries[key].parts_to_do == 0) - { - export_merge_tree_partition_task_entries.erase(key); + const auto zk_client = getZooKeeper(); + if (result.success) + { + complete_part_export( + partition_path, + fs::path(partition_path) / "parts" / part_to_export.value(), + fs::path(partition_path) / "parts" / part_to_export.value() / "status", + fs::path(partition_path) / "parts_to_do", + fs::path(partition_path) / "parts" / part_to_export.value() / "lock", + next_idx_path, + next_idx, + result.relative_path_in_destination_storage, + destination_storage, + transaction_id, + partition_id, + zk_client); + + /// maybe get up to date from complete_parts_export? + std::lock_guard inner_lock(export_merge_tree_partition_mutex); + export_merge_tree_partition_task_entries[key].parts_to_do--; + + if (export_merge_tree_partition_task_entries[key].parts_to_do == 0) + { + export_merge_tree_partition_task_entries.erase(key); + } } - } - else - { - /// increment retry_count - /// if above threshhold, fail the entire export - hopefully it is safe to do so :D - /// I could also leave this for the cleanup thread, but will do it here for now. - - Coordination::Requests ops; - - std::string retry_count_string; - if (zk_client->tryGet(fs::path(exports_path) / key / "parts" / part_to_export.value() / "retry_count", retry_count_string)) + else { - std::size_t retry_count = std::stoull(retry_count_string.c_str()) + 1; - - //// todo arthur unhardcode this - if (retry_count > 3) + /// increment retry_count + /// if above threshhold, fail the entire export - hopefully it is safe to do so :D + /// I could also leave this for the cleanup thread, but will do it here for now. + + Coordination::Requests ops; + + std::string retry_count_string; + if (zk_client->tryGet(fs::path(partition_path) / "parts" / part_to_export.value() / "retry_count", retry_count_string)) { - ops.emplace_back(zkutil::makeRemoveRecursiveRequest(partition_path, 1000)); + std::size_t retry_count = std::stoull(retry_count_string.c_str()) + 1; + + //// todo arthur unhardcode this + if (retry_count >= 3) + { + /// instead of removing the entire partition, just mark the status as failed + ops.emplace_back(zkutil::makeSetRequest(fs::path(partition_path) / "status", "FAILED", -1)); + /// mark the part as failed as well + ops.emplace_back(zkutil::makeSetRequest(fs::path(partition_path) / "parts" / part_to_export.value() / "status", "FAILED", -1)); + } + else + { + ops.emplace_back(zkutil::makeSetRequest(fs::path(partition_path) / "parts" / part_to_export.value() / "retry_count", std::to_string(retry_count), -1)); + } + + ops.emplace_back(zkutil::makeRemoveRequest(fs::path(partition_path) / "parts" / part_to_export.value() / "lock", -1)); } else { - ops.emplace_back(zkutil::makeSetRequest(fs::path(exports_path) / key / "parts" / part_to_export.value() / "retry_count", std::to_string(retry_count), -1)); - /// unlock the part - ops.emplace_back(zkutil::makeRemoveRequest(fs::path(exports_path) / key / "parts" / part_to_export.value() / "lock", -1)); + LOG_INFO(log, "Failed to get retry_count, will not try to update it"); + ops.emplace_back(zkutil::makeRemoveRequest(fs::path(partition_path) / "parts" / part_to_export.value() / "lock", -1)); } + + std::size_t num_exceptions = 0; + + const auto exceptions_per_replica_path = fs::path(partition_path) / "exceptions_per_replica" / replica_name; + const auto count_path = exceptions_per_replica_path / "count"; + const auto last_exception_path = exceptions_per_replica_path / "last_exception"; + + if (zk_client->exists(exceptions_per_replica_path)) + { + std::string num_exceptions_string; + zk_client->tryGet(count_path, num_exceptions_string); + num_exceptions = std::stoull(num_exceptions_string.c_str()); + + ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "part", part_to_export.value(), -1)); + ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "exception", result.exception, -1)); + } + else + { + ops.emplace_back(zkutil::makeCreateRequest(exceptions_per_replica_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(count_path, "0", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "part", part_to_export.value(), zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "exception", result.exception, zkutil::CreateMode::Persistent)); + } + + num_exceptions++; + ops.emplace_back(zkutil::makeSetRequest(count_path, std::to_string(num_exceptions), -1)); + + Coordination::Responses responses; + if (zk_client->tryMulti(ops, responses) != Coordination::Error::ZOK) + { + LOG_INFO(log, "All failure mechanism failed, will not try to update it"); + return; + } + } - else - { - LOG_INFO(log, "Failed to get retry_count, will not try to update it"); - ops.emplace_back(zkutil::makeRemoveRequest(fs::path(partition_path) / "parts" / part_to_export.value() / "lock", -1)); - } - - Coordination::Responses responses; - if (zk_client->tryMulti(ops, responses) != Coordination::Error::ZOK) - { - LOG_INFO(log, "All failure mechanism failed, will not try to update it"); - return; - } - - } - }); - } - catch (...) - { - /// failed to schedule the part export - tryLogCurrentException(log, __PRETTY_FUNCTION__); - - /// best-effort to remove the lock (actually, we should make sure the lock is released..) - zk->tryRemove(fs::path(partition_path) / "parts" / part_to_export.value() / "lock"); - - /// re-run after some time - export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); + }); + } + catch (...) + { + /// failed to schedule the part export + tryLogCurrentException(log, __PRETTY_FUNCTION__); + + /// best-effort to remove the lock (actually, we should make sure the lock is released..) + zk->tryRemove(fs::path(partition_path) / "parts" / part_to_export.value() / "lock"); + + /// re-run after some time + export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); + } + } - } } - + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); + } // export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); } From 91c7ec2c1b0c28f465f6557fbea3234aa9f52502 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 14 Oct 2025 17:52:07 -0300 Subject: [PATCH 10/58] a few fixes --- src/Storages/MergeTree/MergeTreeData.cpp | 7 ------- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- .../02221_system_zookeeper_unrestricted_like.reference | 1 + .../0_stateless/03604_export_merge_tree_partition.sh | 3 +++ 4 files changed, 5 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 4668157c882d..6b2a1f4a48d2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6398,13 +6398,6 @@ void MergeTreeData::exportPartToTableImpl( CompletedPipelineExecutor exec(pipeline); exec.execute(); - volatile bool x = true; - - if (x) - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Some issue"); - } - std::lock_guard inner_lock(export_manifests_mutex); writePartLog( PartLogElement::Type::EXPORT_PART, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 995dccf93042..48e5974c42b1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1586,7 +1586,7 @@ bool StorageReplicatedMergeTree::removeTableNodesFromZooKeeper(zkutil::ZooKeeper /// NOTE /block_numbers/ actually is not flat, because /block_numbers// may have ephemeral children, /// but we assume that all ephemeral block locks are already removed when table is being dropped. - static constexpr std::array flat_nodes = {"block_numbers", "blocks", "async_blocks", "leader_election", "log", "mutations", "pinned_part_uuids", "exports_cleanup_lock"}; + static constexpr std::array flat_nodes = {"block_numbers", "blocks", "async_blocks", "leader_election", "log", "mutations", "pinned_part_uuids"}; /// First try to remove paths that are known to be flat for (const auto * node : flat_nodes) diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference index 8e69bdac72d7..9a250fa65580 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference @@ -8,6 +8,7 @@ blocks columns columns creator_info +exports failed_parts flags host diff --git a/tests/queries/0_stateless/03604_export_merge_tree_partition.sh b/tests/queries/0_stateless/03604_export_merge_tree_partition.sh index 1198281afc72..ea77a2e3f6ff 100755 --- a/tests/queries/0_stateless/03604_export_merge_tree_partition.sh +++ b/tests/queries/0_stateless/03604_export_merge_tree_partition.sh @@ -24,6 +24,9 @@ query "INSERT INTO $rmt_table VALUES (3, 2020), (5, 2021)" query "INSERT INTO $rmt_table VALUES (6, 2022), (7, 2022)" +# sync replicas +query "SYSTEM SYNC REPLICA $rmt_table"s + query "ALTER TABLE $rmt_table EXPORT PARTITION ID '2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" query "ALTER TABLE $rmt_table EXPORT PARTITION ID '2021' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" From 69cd83fb9e08dc8bd7ed1dfab8e3e6839858b705 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 15 Oct 2025 11:31:24 -0300 Subject: [PATCH 11/58] make dest storage id part of the key --- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++++--- .../0_stateless/03604_export_merge_tree_partition.sh | 2 +- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 48e5974c42b1..89599319b423 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4430,7 +4430,7 @@ void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() if (cleanup_lock_acquired) { - bool has_expired = metadata.create_time < now - 45; + bool has_expired = metadata.create_time < now - 90; if (has_expired && is_not_pending) { @@ -4751,7 +4751,8 @@ void StorageReplicatedMergeTree::selectPartsToExport() return std::nullopt; }; - try { + try + { const auto zk = getZooKeeper(); std::lock_guard lock(export_merge_tree_partition_mutex); @@ -8490,6 +8491,7 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & const auto dest_database = query_context->resolveDatabase(command.to_database); const auto dest_table = command.to_table; + const auto dest_storage_id = StorageID(dest_database, dest_table); auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, dest_table}, query_context); if (dest_storage->getStorageID() == this->getStorageID()) @@ -8553,7 +8555,9 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & ops.emplace_back(zkutil::makeCreateRequest(exports_path, "", zkutil::CreateMode::Persistent)); } - const auto partition_exports_path = exports_path / partition_id; + const auto export_key = partition_id + "_" + dest_storage_id.getNameForLogs(); + + const auto partition_exports_path = fs::path(exports_path) / export_key; /// check if entry already exists if (zookeeper->exists(partition_exports_path)) diff --git a/tests/queries/0_stateless/03604_export_merge_tree_partition.sh b/tests/queries/0_stateless/03604_export_merge_tree_partition.sh index ea77a2e3f6ff..d27126469603 100755 --- a/tests/queries/0_stateless/03604_export_merge_tree_partition.sh +++ b/tests/queries/0_stateless/03604_export_merge_tree_partition.sh @@ -25,7 +25,7 @@ query "INSERT INTO $rmt_table VALUES (3, 2020), (5, 2021)" query "INSERT INTO $rmt_table VALUES (6, 2022), (7, 2022)" # sync replicas -query "SYSTEM SYNC REPLICA $rmt_table"s +query "SYSTEM SYNC REPLICA $rmt_table" query "ALTER TABLE $rmt_table EXPORT PARTITION ID '2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" From 62cb51f48aaac280be5a6b59e4ad8c1e1559025e Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 16 Oct 2025 11:48:59 -0300 Subject: [PATCH 12/58] add system.replicated_partition_exports --- src/Storages/StorageReplicatedMergeTree.cpp | 55 +++++++ src/Storages/StorageReplicatedMergeTree.h | 4 + ...torageSystemReplicatedPartitionExports.cpp | 135 ++++++++++++++++++ .../StorageSystemReplicatedPartitionExports.h | 38 +++++ src/Storages/System/attachSystemTables.cpp | 2 + 5 files changed, 234 insertions(+) create mode 100644 src/Storages/System/StorageSystemReplicatedPartitionExports.cpp create mode 100644 src/Storages/System/StorageSystemReplicatedPartitionExports.h diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 89599319b423..b589c6b72f9a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -68,6 +68,7 @@ #include #include #include +#include #include #include #include @@ -4956,6 +4957,60 @@ void StorageReplicatedMergeTree::selectPartsToExport() // export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); } +std::vector StorageReplicatedMergeTree::getPartitionExportsInfo() const +{ + std::vector infos; + + const auto zk = getZooKeeper(); + const auto exports_path = fs::path(zookeeper_path) / "exports"; + const auto children = zk->getChildren(exports_path); + + for (const auto & child : children) + { + ReplicatedPartitionExportInfo info; + + const auto export_partition_path = fs::path(exports_path) / child; + std::string metadata_json; + if (!zk->tryGet(export_partition_path / "metadata.json", metadata_json)) + { + LOG_INFO(log, "Skipping {}: missing metadata.json", child); + continue; + } + + std::string parts_to_do_string; + if (!zk->tryGet(export_partition_path / "parts_to_do", parts_to_do_string)) + { + LOG_INFO(log, "Skipping {}: missing parts_to_do", child); + continue; + } + + std::string status; + if (!zk->tryGet(export_partition_path / "status", status)) + { + LOG_INFO(log, "Skipping {}: missing status", child); + continue; + } + + const auto parts_to_do = std::stoull(parts_to_do_string.c_str()); + const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + + info.destination_database = metadata.destination_database; + info.destination_table = metadata.destination_table; + info.partition_id = metadata.partition_id; + info.transaction_id = metadata.transaction_id; + info.create_time = metadata.create_time; + info.source_replica = metadata.source_replica; + info.parts_count = metadata.number_of_parts; + info.parts_to_do = parts_to_do; + info.parts = metadata.parts; + info.status = status; + + infos.emplace_back(std::move(info)); + } + + return infos; +} + StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::createLogEntryToMergeParts( zkutil::ZooKeeperPtr & zookeeper, diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 05263ecd909c..9cdbf9c9dd10 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -98,6 +98,8 @@ namespace DB class ZooKeeperWithFaultInjection; using ZooKeeperWithFaultInjectionPtr = std::shared_ptr; +struct ReplicatedPartitionExportInfo; + class StorageReplicatedMergeTree final : public MergeTreeData { public: @@ -372,6 +374,8 @@ class StorageReplicatedMergeTree final : public MergeTreeData using ShutdownDeadline = std::chrono::time_point; void waitForUniquePartsToBeFetchedByOtherReplicas(ShutdownDeadline shutdown_deadline); + std::vector getPartitionExportsInfo() const; + private: std::atomic_bool are_restoring_replica {false}; diff --git a/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp b/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp new file mode 100644 index 000000000000..e207755f7fff --- /dev/null +++ b/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp @@ -0,0 +1,135 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "Columns/ColumnString.h" +#include "Storages/VirtualColumnUtils.h" + + +namespace DB +{ + +ColumnsDescription StorageSystemReplicatedPartitionExports::getColumnsDescription() +{ + return ColumnsDescription + { + {"source_database", std::make_shared(), "Name of the source database."}, + {"source_table", std::make_shared(), "Name of the source table."}, + {"destination_database", std::make_shared(), "Name of the destination database."}, + {"destination_table", std::make_shared(), "Name of the destination table."}, + {"create_time", std::make_shared(), "Date and time when the export command was submitted"}, + {"partition_id", std::make_shared(), "ID of the partition"}, + {"transaction_id", std::make_shared(), "ID of the transaction."}, + {"source_replica", std::make_shared(), "Name of the source replica."}, + {"parts", std::make_shared(std::make_shared()), "List of part names to be exported."}, + {"parts_count", std::make_shared(), "Number of parts in the export."}, + {"parts_to_do", std::make_shared(), "Number of parts pending to be exported."}, + {"status", std::make_shared(), "Status of the export."}, + }; +} + +void StorageSystemReplicatedPartitionExports::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const +{ + const auto access = context->getAccess(); + const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); + + std::map> replicated_merge_tree_tables; + for (const auto & db : DatabaseCatalog::instance().getDatabases()) + { + /// Check if database can contain MergeTree tables + if (!db.second->canContainMergeTreeTables()) + continue; + + const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first); + + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) + { + const auto & table = iterator->table(); + if (!table) + continue; + + StorageReplicatedMergeTree * table_replicated = dynamic_cast(table.get()); + if (!table_replicated) + continue; + + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name())) + continue; + + replicated_merge_tree_tables[db.first][iterator->name()] = table; + } + } + + MutableColumnPtr col_database_mut = ColumnString::create(); + MutableColumnPtr col_table_mut = ColumnString::create(); + + for (auto & db : replicated_merge_tree_tables) + { + for (auto & table : db.second) + { + col_database_mut->insert(db.first); + col_table_mut->insert(table.first); + } + } + + ColumnPtr col_database = std::move(col_database_mut); + ColumnPtr col_table = std::move(col_table_mut); + + /// Determine what tables are needed by the conditions in the query. + { + Block filtered_block + { + { col_database, std::make_shared(), "database" }, + { col_table, std::make_shared(), "table" }, + }; + + VirtualColumnUtils::filterBlockWithPredicate(predicate, filtered_block, context); + + if (!filtered_block.rows()) + return; + + col_database = filtered_block.getByName("database").column; + col_table = filtered_block.getByName("table").column; + } + + for (size_t i_storage = 0; i_storage < col_database->size(); ++i_storage) + { + const auto database = (*col_database)[i_storage].safeGet(); + const auto table = (*col_table)[i_storage].safeGet(); + + std::vector partition_exports_info; + { + const IStorage * storage = replicated_merge_tree_tables[database][table].get(); + if (const auto * replicated_merge_tree = dynamic_cast(storage)) + partition_exports_info = replicated_merge_tree->getPartitionExportsInfo(); + } + + for (const ReplicatedPartitionExportInfo & info : partition_exports_info) + { + std::size_t i = 0; + res_columns[i++]->insert(database); + res_columns[i++]->insert(table); + res_columns[i++]->insert(info.destination_database); + res_columns[i++]->insert(info.destination_table); + res_columns[i++]->insert(info.create_time); + res_columns[i++]->insert(info.partition_id); + res_columns[i++]->insert(info.transaction_id); + res_columns[i++]->insert(info.source_replica); + Array parts_array; + parts_array.reserve(info.parts.size()); + for (const auto & part : info.parts) + parts_array.push_back(part); + res_columns[i++]->insert(parts_array); + res_columns[i++]->insert(info.parts_count); + res_columns[i++]->insert(info.parts_to_do); + res_columns[i++]->insert(info.status); + } + } +} + +} diff --git a/src/Storages/System/StorageSystemReplicatedPartitionExports.h b/src/Storages/System/StorageSystemReplicatedPartitionExports.h new file mode 100644 index 000000000000..23e2471a92ff --- /dev/null +++ b/src/Storages/System/StorageSystemReplicatedPartitionExports.h @@ -0,0 +1,38 @@ +#pragma once + +#include + +namespace DB +{ + +class Context; + +struct ReplicatedPartitionExportInfo +{ + String destination_database; + String destination_table; + String partition_id; + String transaction_id; + time_t create_time; + String source_replica; + size_t parts_count; + size_t parts_to_do; + std::vector parts; + String status; +}; + +class StorageSystemReplicatedPartitionExports final : public IStorageSystemOneBlock +{ +public: + + std::string getName() const override { return "SystemReplicatedPartitionExports"; } + + static ColumnsDescription getColumnsDescription(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 5a3a4d30599d..358ac3d78438 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -1,4 +1,5 @@ #include +#include #include "config.h" #include @@ -211,6 +212,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "merges", "Contains a list of merges currently executing merges of MergeTree tables and their progress. Each merge operation is represented by a single row."); attach(context, system_database, "moves", "Contains information about in-progress data part moves of MergeTree tables. Each data part movement is represented by a single row."); attach(context, system_database, "exports", "Contains a list of exports currently executing exports of MergeTree tables and their progress. Each export operation is represented by a single row."); + attach(context, system_database, "replicated_partition_exports", "Contains a list of partition exports of ReplicatedMergeTree tables and their progress. Each export operation is represented by a single row."); attach(context, system_database, "mutations", "Contains a list of mutations and their progress. Each mutation command is represented by a single row."); attachNoDescription(context, system_database, "replicas", "Contains information and status of all table replicas on current server. Each replica is represented by a single row."); attach(context, system_database, "replication_queue", "Contains information about tasks from replication queues stored in ClickHouse Keeper, or ZooKeeper, for each table replica."); From 54c2dfb2e3cc2d96a279917fe5cdb2b75bc7aaf6 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 17 Oct 2025 08:53:18 -0300 Subject: [PATCH 13/58] add exception to replicated systems table --- src/Storages/StorageReplicatedMergeTree.cpp | 35 +++++++++++++++++++ ...torageSystemReplicatedPartitionExports.cpp | 6 ++++ .../StorageSystemReplicatedPartitionExports.h | 3 ++ ...3604_export_merge_tree_partition.reference | 1 - 4 files changed, 44 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b589c6b72f9a..91016f9180d2 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4992,6 +4992,38 @@ std::vector StorageReplicatedMergeTree::getPartit } const auto parts_to_do = std::stoull(parts_to_do_string.c_str()); + + std::string last_exception; + std::string exception_part; + std::size_t exception_count = 0; + + const auto exceptions_per_replica_path = export_partition_path / "exceptions_per_replica"; + + const auto exception_children = zk->getChildren(exceptions_per_replica_path); + for (const auto & exception_child : exception_children) + { + std::string exception_count_string; + if (!zk->tryGet(exceptions_per_replica_path / exception_child / "count", exception_count_string)) + { + LOG_INFO(log, "Skipping {}: missing count", exception_child); + continue; + } + exception_count += std::stoull(exception_count_string.c_str()); + + if (last_exception.empty()) + { + const auto last_exception_path = exceptions_per_replica_path / exception_child / "last_exception"; + std::string last_exception_string; + if (!zk->tryGet(last_exception_path / "exception", last_exception_string)) + { + LOG_INFO(log, "Skipping {}: missing last_exception/exception", last_exception_path); + continue; + } + last_exception = last_exception_string; + exception_part = last_exception_path / "part"; + } + } + const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); info.destination_database = metadata.destination_database; @@ -5004,6 +5036,9 @@ std::vector StorageReplicatedMergeTree::getPartit info.parts_to_do = parts_to_do; info.parts = metadata.parts; info.status = status; + info.last_exception = last_exception; + info.exception_part = exception_part; + info.exception_count = exception_count; infos.emplace_back(std::move(info)); } diff --git a/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp b/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp index e207755f7fff..94dabe8bf36a 100644 --- a/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp +++ b/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp @@ -31,6 +31,9 @@ ColumnsDescription StorageSystemReplicatedPartitionExports::getColumnsDescriptio {"parts_count", std::make_shared(), "Number of parts in the export."}, {"parts_to_do", std::make_shared(), "Number of parts pending to be exported."}, {"status", std::make_shared(), "Status of the export."}, + {"last_exception", std::make_shared(), "Last exception message of any part (not necessarily the last global exception)"}, + {"exception_part", std::make_shared(), "Part that caused the last exception"}, + {"exception_count", std::make_shared(), "Number of global exceptions"}, }; } @@ -128,6 +131,9 @@ void StorageSystemReplicatedPartitionExports::fillData(MutableColumns & res_colu res_columns[i++]->insert(info.parts_count); res_columns[i++]->insert(info.parts_to_do); res_columns[i++]->insert(info.status); + res_columns[i++]->insert(info.last_exception); + res_columns[i++]->insert(info.exception_part); + res_columns[i++]->insert(info.exception_count); } } } diff --git a/src/Storages/System/StorageSystemReplicatedPartitionExports.h b/src/Storages/System/StorageSystemReplicatedPartitionExports.h index 23e2471a92ff..de9b6bc1d7ff 100644 --- a/src/Storages/System/StorageSystemReplicatedPartitionExports.h +++ b/src/Storages/System/StorageSystemReplicatedPartitionExports.h @@ -19,6 +19,9 @@ struct ReplicatedPartitionExportInfo size_t parts_to_do; std::vector parts; String status; + std::string last_exception; + std::string exception_part; + size_t exception_count; }; class StorageSystemReplicatedPartitionExports final : public IStorageSystemOneBlock diff --git a/tests/queries/0_stateless/03604_export_merge_tree_partition.reference b/tests/queries/0_stateless/03604_export_merge_tree_partition.reference index 97e17ae5aeec..2bda7b6165a4 100644 --- a/tests/queries/0_stateless/03604_export_merge_tree_partition.reference +++ b/tests/queries/0_stateless/03604_export_merge_tree_partition.reference @@ -1,4 +1,3 @@ - Select from source table 1 2020 2 2020 From 7b3a7c9aa25b0953a0d72bff8f9b43c6ac491c4a Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 17 Oct 2025 09:12:35 -0300 Subject: [PATCH 14/58] add the replica that caused the exception --- src/Storages/StorageReplicatedMergeTree.cpp | 23 ++++++++++++++----- ...torageSystemReplicatedPartitionExports.cpp | 2 ++ .../StorageSystemReplicatedPartitionExports.h | 1 + 3 files changed, 20 insertions(+), 6 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 91016f9180d2..5db996593d11 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4993,34 +4993,44 @@ std::vector StorageReplicatedMergeTree::getPartit const auto parts_to_do = std::stoull(parts_to_do_string.c_str()); + std::string exception_replica; std::string last_exception; std::string exception_part; std::size_t exception_count = 0; const auto exceptions_per_replica_path = export_partition_path / "exceptions_per_replica"; - const auto exception_children = zk->getChildren(exceptions_per_replica_path); - for (const auto & exception_child : exception_children) + const auto exception_replicas = zk->getChildren(exceptions_per_replica_path); + for (const auto & replica : exception_replicas) { std::string exception_count_string; - if (!zk->tryGet(exceptions_per_replica_path / exception_child / "count", exception_count_string)) + if (!zk->tryGet(exceptions_per_replica_path / replica / "count", exception_count_string)) { - LOG_INFO(log, "Skipping {}: missing count", exception_child); + LOG_INFO(log, "Skipping {}: missing count", replica); continue; } exception_count += std::stoull(exception_count_string.c_str()); if (last_exception.empty()) { - const auto last_exception_path = exceptions_per_replica_path / exception_child / "last_exception"; + const auto last_exception_path = exceptions_per_replica_path / replica / "last_exception"; std::string last_exception_string; if (!zk->tryGet(last_exception_path / "exception", last_exception_string)) { LOG_INFO(log, "Skipping {}: missing last_exception/exception", last_exception_path); continue; } + + std::string exception_part_zk; + if (!zk->tryGet(last_exception_path / "part", exception_part_zk)) + { + LOG_INFO(log, "Skipping {}: missing exception part", last_exception_path); + continue; + } + + exception_replica = replica; last_exception = last_exception_string; - exception_part = last_exception_path / "part"; + exception_part = exception_part_zk; } } @@ -5036,6 +5046,7 @@ std::vector StorageReplicatedMergeTree::getPartit info.parts_to_do = parts_to_do; info.parts = metadata.parts; info.status = status; + info.exception_replica = exception_replica; info.last_exception = last_exception; info.exception_part = exception_part; info.exception_count = exception_count; diff --git a/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp b/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp index 94dabe8bf36a..018f0c8ffac7 100644 --- a/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp +++ b/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp @@ -31,6 +31,7 @@ ColumnsDescription StorageSystemReplicatedPartitionExports::getColumnsDescriptio {"parts_count", std::make_shared(), "Number of parts in the export."}, {"parts_to_do", std::make_shared(), "Number of parts pending to be exported."}, {"status", std::make_shared(), "Status of the export."}, + {"exception_replica", std::make_shared(), "Replica that caused the last exception"}, {"last_exception", std::make_shared(), "Last exception message of any part (not necessarily the last global exception)"}, {"exception_part", std::make_shared(), "Part that caused the last exception"}, {"exception_count", std::make_shared(), "Number of global exceptions"}, @@ -131,6 +132,7 @@ void StorageSystemReplicatedPartitionExports::fillData(MutableColumns & res_colu res_columns[i++]->insert(info.parts_count); res_columns[i++]->insert(info.parts_to_do); res_columns[i++]->insert(info.status); + res_columns[i++]->insert(info.exception_replica); res_columns[i++]->insert(info.last_exception); res_columns[i++]->insert(info.exception_part); res_columns[i++]->insert(info.exception_count); diff --git a/src/Storages/System/StorageSystemReplicatedPartitionExports.h b/src/Storages/System/StorageSystemReplicatedPartitionExports.h index de9b6bc1d7ff..de2547437c21 100644 --- a/src/Storages/System/StorageSystemReplicatedPartitionExports.h +++ b/src/Storages/System/StorageSystemReplicatedPartitionExports.h @@ -19,6 +19,7 @@ struct ReplicatedPartitionExportInfo size_t parts_to_do; std::vector parts; String status; + std::string exception_replica; std::string last_exception; std::string exception_part; size_t exception_count; From 3f3983ca3d027ea8210f2f9aa4147e36f7a800f9 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 17 Oct 2025 09:57:01 -0300 Subject: [PATCH 15/58] export_merge_tree_partition_force_export --- src/Common/ZooKeeper/ZooKeeper.h | 2 +- src/Core/Settings.cpp | 3 ++ src/Storages/StorageReplicatedMergeTree.cpp | 40 ++++++++++++--------- 3 files changed, 28 insertions(+), 17 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index fdb404fa7ae2..5c608fbf9611 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -380,8 +380,8 @@ class ZooKeeper Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL); using MultiGetChildrenResponse = MultiReadResponses; + using MultiTryGetChildrenResponse = MultiReadResponses; - template MultiGetChildrenResponse getChildren(TIter start, TIter end, Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 059d8702c3ce..9b7987e497eb 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -6873,6 +6873,9 @@ Use roaring bitmap for iceberg positional deletes. )", 0) \ DECLARE(Bool, export_merge_tree_part_overwrite_file_if_exists, false, R"( Overwrite file if it already exists when exporting a merge tree part +)", 0) \ + DECLARE(Bool, export_merge_tree_partition_force_export, false, R"( +Ignore existing partition export and overwrite the zookeeper entry )", 0) \ \ /* ####################################################### */ \ diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 5db996593d11..7fbefdd98083 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -193,6 +193,7 @@ namespace Setting extern const SettingsUInt64 select_sequential_consistency; extern const SettingsBool update_sequential_consistency; extern const SettingsBool allow_experimental_export_merge_tree_part; + extern const SettingsBool export_merge_tree_partition_force_export; } namespace MergeTreeSetting @@ -4406,9 +4407,6 @@ void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() /// If we have the cleanup lock, also remove stale entries from zk and local for (const auto & key : zk_children) { - if (!cleanup_lock_acquired && export_merge_tree_partition_task_entries.contains(key)) - continue; - const std::string entry_path = fs::path(exports_path) / key; std::string metadata_json; @@ -4417,6 +4415,19 @@ void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() LOG_INFO(log, "Skipping {}: missing metadata.json", key); continue; } + + const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + + const auto local_entry = export_merge_tree_partition_task_entries.find(key); + + /// If the zk entry has been replaced with export_merge_tree_partition_force_export, checking only for the export key is not enough + /// we need to make sure it is the same transaction id. If it is not, it needs to be replaced. + bool has_local_entry_and_is_up_to_date = local_entry != export_merge_tree_partition_task_entries.end() + && local_entry->second.manifest.transaction_id == metadata.transaction_id; + + /// If the entry is up to date and we don't have the cleanup lock, early exit, nothing to be done. + if (!cleanup_lock_acquired && has_local_entry_and_is_up_to_date) + continue; std::string status; if (!zk->tryGet(fs::path(entry_path) / "status", status)) @@ -4427,8 +4438,6 @@ void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() bool is_not_pending = status != "PENDING"; - const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); - if (cleanup_lock_acquired) { bool has_expired = metadata.create_time < now - 90; @@ -4448,7 +4457,7 @@ void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() continue; } - if (export_merge_tree_partition_task_entries.contains(key)) + if (has_local_entry_and_is_up_to_date) { LOG_INFO(log, "Skipping {}: already exists", key); continue; @@ -4488,9 +4497,8 @@ void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() } } - export_merge_tree_partition_task_entries.emplace( - key, - ExportReplicatedMergeTreePartitionTaskEntry {metadata, parts_to_do, std::move(part_references)}); + /// It is important to use the operator[] because it updates the existing entry if it already exists. + export_merge_tree_partition_task_entries[key] = ExportReplicatedMergeTreePartitionTaskEntry {metadata, parts_to_do, std::move(part_references)}; } /// Remove entries that were deleted by someone else @@ -8650,12 +8658,6 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & start_time: 123445 */ - /// maybe this should go in initialization somewhere else - if (!zookeeper->exists(exports_path)) - { - ops.emplace_back(zkutil::makeCreateRequest(exports_path, "", zkutil::CreateMode::Persistent)); - } - const auto export_key = partition_id + "_" + dest_storage_id.getNameForLogs(); const auto partition_exports_path = fs::path(exports_path) / export_key; @@ -8663,7 +8665,13 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & /// check if entry already exists if (zookeeper->exists(partition_exports_path)) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition {} already exported or it is being exported", partition_id); + if (!query_context->getSettingsRef()[Setting::export_merge_tree_partition_force_export]) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition {} already exported or it is being exported", partition_id); + } + + /// The check for existence and entry removal are not atomic, so this actually might fail. + ops.emplace_back(zkutil::makeRemoveRecursiveRequest(partition_exports_path, -1)); } ops.emplace_back(zkutil::makeCreateRequest(partition_exports_path, "", zkutil::CreateMode::Persistent)); From b89cd5ef93a73707b047e61b5d4d590b8c4b5211 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 21 Oct 2025 09:33:23 -0300 Subject: [PATCH 16/58] almost done with kill export partition --- .../InterpreterKillQueryQuery.cpp | 74 +++++++++++++++++ src/Parsers/ASTKillQueryQuery.cpp | 3 + src/Parsers/ASTKillQueryQuery.h | 1 + src/Parsers/ParserKillQueryQuery.cpp | 3 + src/Storages/IStorage.cpp | 5 ++ src/Storages/IStorage.h | 3 + src/Storages/MergeTree/MergeTreeData.cpp | 41 +++++++-- src/Storages/MergeTree/MergeTreeData.h | 16 ++-- ...nifest.h => MergeTreePartExportManifest.h} | 15 +++- ...rtStatus.h => MergeTreePartExportStatus.h} | 0 src/Storages/StorageReplicatedMergeTree.cpp | 83 +++++++++++++++---- src/Storages/StorageReplicatedMergeTree.h | 2 + 12 files changed, 212 insertions(+), 34 deletions(-) rename src/Storages/MergeTree/{MergeTreeExportManifest.h => MergeTreePartExportManifest.h} (84%) rename src/Storages/MergeTree/{MergeTreeExportStatus.h => MergeTreePartExportStatus.h} (100%) diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 51ea9051ddfc..3b68ef39b16c 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -250,6 +250,77 @@ BlockIO InterpreterKillQueryQuery::execute() break; } + case ASTKillQueryQuery::Type::ExportPartition: + { + Block exports_block = getSelectResult( + "source_database, source_table, transaction_id, destination_database, destination_table, partition_id", + "system.replicated_partition_exports"); + if (!exports_block) + return res_io; + + const ColumnString & src_db_col = typeid_cast(*exports_block.getByName("source_database").column); + const ColumnString & src_tbl_col = typeid_cast(*exports_block.getByName("source_table").column); + const ColumnString & dst_db_col = typeid_cast(*exports_block.getByName("destination_database").column); + const ColumnString & dst_tbl_col = typeid_cast(*exports_block.getByName("destination_table").column); + const ColumnString & tx_col = typeid_cast(*exports_block.getByName("transaction_id").column); + + auto header = exports_block.cloneEmpty(); + header.insert(0, {ColumnString::create(), std::make_shared(), "kill_status"}); + + MutableColumns res_columns = header.cloneEmptyColumns(); + auto table_id = StorageID::createEmpty(); + AccessRightsElements required_access_rights; + auto access = getContext()->getAccess(); + bool access_denied = false; + + for (size_t i = 0; i < exports_block.rows(); ++i) + { + const auto src_database = src_db_col.getDataAt(i).toString(); + const auto src_table = src_tbl_col.getDataAt(i).toString(); + const auto dst_database = dst_db_col.getDataAt(i).toString(); + const auto dst_table = dst_tbl_col.getDataAt(i).toString(); + + table_id = StorageID{src_database, src_table}; + auto transaction_id = tx_col.getDataAt(i).toString(); + + CancellationCode code = CancellationCode::Unknown; + if (!query.test) + { + auto storage = DatabaseCatalog::instance().tryGetTable(table_id, getContext()); + if (!storage) + code = CancellationCode::NotFound; + else + { + ASTAlterCommand alter_command{}; + alter_command.type = ASTAlterCommand::EXPORT_PARTITION; + alter_command.move_destination_type = DataDestinationType::TABLE; + alter_command.from_database = src_database; + alter_command.from_table = src_table; + alter_command.to_database = dst_database; + alter_command.to_table = dst_table; + + required_access_rights = InterpreterAlterQuery::getRequiredAccessForCommand( + alter_command, table_id.database_name, table_id.table_name); + if (!access->isGranted(required_access_rights)) + { + access_denied = true; + continue; + } + code = storage->killExportPartition(transaction_id); + } + } + + insertResultRow(i, code, exports_block, header, res_columns); + } + + if (res_columns[0]->empty() && access_denied) + throw Exception(ErrorCodes::ACCESS_DENIED, "Not allowed to kill export partition. " + "To execute this query, it's necessary to have the grant {}", required_access_rights.toString()); + + res_io.pipeline = QueryPipeline(Pipe(std::make_shared(header.cloneWithColumns(std::move(res_columns))))); + + break; + } case ASTKillQueryQuery::Type::Mutation: { Block mutations_block = getSelectResult("database, table, mutation_id, command", "system.mutations"); @@ -462,6 +533,9 @@ AccessRightsElements InterpreterKillQueryQuery::getRequiredAccessForDDLOnCluster | AccessType::ALTER_MATERIALIZE_COLUMN | AccessType::ALTER_MATERIALIZE_TTL ); + /// todo arthur think about this + else if (query.type == ASTKillQueryQuery::Type::ExportPartition) + required_access.emplace_back(AccessType::ALTER_EXPORT_PARTITION); return required_access; } diff --git a/src/Parsers/ASTKillQueryQuery.cpp b/src/Parsers/ASTKillQueryQuery.cpp index 0334b78d559e..9911e60b5ed9 100644 --- a/src/Parsers/ASTKillQueryQuery.cpp +++ b/src/Parsers/ASTKillQueryQuery.cpp @@ -27,6 +27,9 @@ void ASTKillQueryQuery::formatQueryImpl(WriteBuffer & ostr, const FormatSettings case Type::Transaction: ostr << "TRANSACTION"; break; + case Type::ExportPartition: + ostr << "EXPORT PARTITION"; + break; } formatOnCluster(ostr, settings); diff --git a/src/Parsers/ASTKillQueryQuery.h b/src/Parsers/ASTKillQueryQuery.h index 99a14c56d72b..13d2811534f0 100644 --- a/src/Parsers/ASTKillQueryQuery.h +++ b/src/Parsers/ASTKillQueryQuery.h @@ -13,6 +13,7 @@ class ASTKillQueryQuery : public ASTQueryWithOutput, public ASTQueryWithOnCluste { Query, /// KILL QUERY Mutation, /// KILL MUTATION + ExportPartition, /// KILL EXPORT_PARTITION PartMoveToShard, /// KILL PART_MOVE_TO_SHARD Transaction, /// KILL TRANSACTION }; diff --git a/src/Parsers/ParserKillQueryQuery.cpp b/src/Parsers/ParserKillQueryQuery.cpp index 55bd5100009e..7e06ae8d30b7 100644 --- a/src/Parsers/ParserKillQueryQuery.cpp +++ b/src/Parsers/ParserKillQueryQuery.cpp @@ -17,6 +17,7 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect ParserKeyword p_kill{Keyword::KILL}; ParserKeyword p_query{Keyword::QUERY}; ParserKeyword p_mutation{Keyword::MUTATION}; + ParserKeyword p_export_partition{Keyword::EXPORT_PARTITION}; ParserKeyword p_part_move_to_shard{Keyword::PART_MOVE_TO_SHARD}; ParserKeyword p_transaction{Keyword::TRANSACTION}; ParserKeyword p_on{Keyword::ON}; @@ -33,6 +34,8 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect query->type = ASTKillQueryQuery::Type::Query; else if (p_mutation.ignore(pos, expected)) query->type = ASTKillQueryQuery::Type::Mutation; + else if (p_export_partition.ignore(pos, expected)) + query->type = ASTKillQueryQuery::Type::ExportPartition; else if (p_part_move_to_shard.ignore(pos, expected)) query->type = ASTKillQueryQuery::Type::PartMoveToShard; else if (p_transaction.ignore(pos, expected)) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 215c832624c1..3766afafe6b9 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -321,6 +321,11 @@ CancellationCode IStorage::killPartMoveToShard(const UUID & /*task_uuid*/) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Part moves between shards are not supported by storage {}", getName()); } +CancellationCode IStorage::killExportPartition(const String & /*transaction_id*/) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Export partition is not supported by storage {}", getName()); +} + StorageID IStorage::getStorageID() const { std::lock_guard lock(id_mutex); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 2e2202fe02b6..397405621641 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -591,6 +591,9 @@ It is currently only implemented in StorageObjectStorage. virtual void setMutationCSN(const String & /*mutation_id*/, UInt64 /*csn*/); + /// Cancel a replicated partition export by transaction id. + virtual CancellationCode killExportPartition(const String & /*transaction_id*/); + /// Cancel a part move to shard. virtual CancellationCode killPartMoveToShard(const UUID & /*task_uuid*/); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6b2a1f4a48d2..8f9d549992c8 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6210,14 +6210,15 @@ void MergeTreeData::exportPartToTable(const PartitionCommand & command, ContextP const auto database_name = query_context->resolveDatabase(command.to_database); - exportPartToTable(part_name, StorageID{database_name, command.to_table}, query_context); + exportPartToTable(part_name, StorageID{database_name, command.to_table}, query_context->getCurrentQueryId(), query_context); } void MergeTreeData::exportPartToTable( const std::string & part_name, const StorageID & destination_storage_id, + const String & transaction_id, ContextPtr query_context, - std::function completion_callback) + std::function completion_callback) { auto dest_storage = DatabaseCatalog::instance().getTable(destination_storage_id, query_context); @@ -6250,9 +6251,10 @@ void MergeTreeData::exportPartToTable( part_name, getStorageID().getFullTableName()); { - MergeTreeExportManifest manifest( + MergeTreePartExportManifest manifest( dest_storage->getStorageID(), part, + transaction_id, query_context->getSettingsRef()[Setting::export_merge_tree_part_overwrite_file_if_exists], query_context->getSettingsRef()[Setting::output_format_parallel_formatting], completion_callback); @@ -6270,7 +6272,7 @@ void MergeTreeData::exportPartToTable( } void MergeTreeData::exportPartToTableImpl( - const MergeTreeExportManifest & manifest, + const MergeTreePartExportManifest & manifest, ContextPtr local_context) { auto metadata_snapshot = getInMemoryMetadataPtr(); @@ -6324,7 +6326,7 @@ void MergeTreeData::exportPartToTableImpl( export_manifests.erase(manifest); if (manifest.completion_callback) - manifest.completion_callback(MergeTreeExportManifest::CompletionCallbackResult::createFailure(e.message())); + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createFailure(e.message())); return; } @@ -6393,6 +6395,9 @@ void MergeTreeData::exportPartToTableImpl( pipeline.complete(sink); + /// oh boy, is there another way? + manifest.pipeline = &pipeline; + try { CompletedPipelineExecutor exec(pipeline); @@ -6416,7 +6421,7 @@ void MergeTreeData::exportPartToTableImpl( ProfileEvents::increment(ProfileEvents::PartsExportTotalMilliseconds, static_cast((*exports_list_entry)->elapsed * 1000)); if (manifest.completion_callback) - manifest.completion_callback(MergeTreeExportManifest::CompletionCallbackResult::createSuccess(destination_file_path)); + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createSuccess(destination_file_path)); } catch (const Exception & e) { @@ -6439,12 +6444,30 @@ void MergeTreeData::exportPartToTableImpl( export_manifests.erase(manifest); if (manifest.completion_callback) - manifest.completion_callback(MergeTreeExportManifest::CompletionCallbackResult::createFailure(e.message())); + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createFailure(e.message())); throw; } } +void MergeTreeData::killExportPart(const String & query_id) +{ + std::lock_guard lock(export_manifests_mutex); + + const auto it = std::find_if(export_manifests.begin(), export_manifests.end(), [&](const auto & manifest) + { + return manifest.query_id == query_id; + }); + + if (it == export_manifests.end()) + return; + + if (it->pipeline) + it->pipeline->cancel(); + + export_manifests.erase(it); +} + void MergeTreeData::movePartitionToShard(const ASTPtr & /*partition*/, bool /*move_part*/, const String & /*to*/, ContextPtr /*query_context*/) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MOVE PARTITION TO SHARD is not supported by storage {}", getName()); @@ -9284,7 +9307,9 @@ bool MergeTreeData::scheduleDataMovingJob(BackgroundJobsAssignee & assignee) } manifest.in_progress = assignee.scheduleMoveTask(std::make_shared( - [this, manifest] () mutable { + [this, &manifest] () mutable { + /// TODO arthur fix this: I need to be able to modify the real manifest + /// but grabbing it by reference is causing problems exportPartToTableImpl(manifest, getContext()); return true; }, diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 30b13ae600a4..b2b748c86ab5 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -38,8 +38,8 @@ #include #include #include -#include -#include +#include +#include #include #include @@ -986,8 +986,12 @@ class MergeTreeData : public IStorage, public WithMutableContext void exportPartToTable( const std::string & part_name, - const StorageID & destination_storage_id, ContextPtr query_context, - std::function completion_callback = {}); + const StorageID & destination_storage_id, + const String & transaction_id, + ContextPtr query_context, + std::function completion_callback = {}); + + void killExportPart(const String & query_id); virtual void exportPartitionToTable(const PartitionCommand &, ContextPtr) { @@ -995,7 +999,7 @@ class MergeTreeData : public IStorage, public WithMutableContext } void exportPartToTableImpl( - const MergeTreeExportManifest & manifest, + const MergeTreePartExportManifest & manifest, ContextPtr local_context); /// Checks that Partition could be dropped right now @@ -1263,7 +1267,7 @@ class MergeTreeData : public IStorage, public WithMutableContext mutable std::mutex export_manifests_mutex; - std::set export_manifests; + std::set export_manifests; PinnedPartUUIDsPtr getPinnedPartUUIDs() const; diff --git a/src/Storages/MergeTree/MergeTreeExportManifest.h b/src/Storages/MergeTree/MergeTreePartExportManifest.h similarity index 84% rename from src/Storages/MergeTree/MergeTreeExportManifest.h rename to src/Storages/MergeTree/MergeTreePartExportManifest.h index bcc480c8507e..3052bccec6bc 100644 --- a/src/Storages/MergeTree/MergeTreeExportManifest.h +++ b/src/Storages/MergeTree/MergeTreePartExportManifest.h @@ -1,10 +1,11 @@ #include #include +#include "QueryPipeline/QueryPipeline.h" namespace DB { -struct MergeTreeExportManifest +struct MergeTreePartExportManifest { using DataPartPtr = std::shared_ptr; @@ -30,14 +31,16 @@ struct MergeTreeExportManifest String exception; }; - MergeTreeExportManifest( + MergeTreePartExportManifest( const StorageID & destination_storage_id_, const DataPartPtr & data_part_, + const String & query_id_, bool overwrite_file_if_exists_, bool parallel_formatting_, std::function completion_callback_ = {}) : destination_storage_id(destination_storage_id_), data_part(data_part_), + query_id(query_id_), overwrite_file_if_exists(overwrite_file_if_exists_), parallel_formatting(parallel_formatting_), completion_callback(completion_callback_), @@ -45,6 +48,8 @@ struct MergeTreeExportManifest StorageID destination_storage_id; DataPartPtr data_part; + /// Used for killing the export. + String query_id; bool overwrite_file_if_exists; bool parallel_formatting; @@ -53,8 +58,10 @@ struct MergeTreeExportManifest time_t create_time; mutable bool in_progress = false; + /// Used for killing the export + mutable QueryPipeline * pipeline = nullptr; - bool operator<(const MergeTreeExportManifest & rhs) const + bool operator<(const MergeTreePartExportManifest & rhs) const { // Lexicographic comparison: first compare destination storage, then part name auto lhs_storage = destination_storage_id.getQualifiedName(); @@ -66,7 +73,7 @@ struct MergeTreeExportManifest return data_part->name < rhs.data_part->name; } - bool operator==(const MergeTreeExportManifest & rhs) const + bool operator==(const MergeTreePartExportManifest & rhs) const { return destination_storage_id.getQualifiedName() == rhs.destination_storage_id.getQualifiedName() && data_part->name == rhs.data_part->name; diff --git a/src/Storages/MergeTree/MergeTreeExportStatus.h b/src/Storages/MergeTree/MergeTreePartExportStatus.h similarity index 100% rename from src/Storages/MergeTree/MergeTreeExportStatus.h rename to src/Storages/MergeTree/MergeTreePartExportStatus.h diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7fbefdd98083..7bcbc6b67944 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -122,7 +122,6 @@ #include "QueryPipeline/QueryPlanResourceHolder.h" #include "Storages/ExportReplicatedMergeTreePartitionManifest.h" #include "Storages/ExportReplicatedMergeTreePartitionTaskEntry.h" -#include #include #include #include @@ -4503,7 +4502,27 @@ void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() /// Remove entries that were deleted by someone else std::erase_if(export_merge_tree_partition_task_entries, - [&](auto const & kv) { return !zk_children.contains(kv.first); }); + [&](auto const & kv) + { + if (zk_children.contains(kv.first)) + { + return false; + } + + const auto & transaction_id = kv.second.manifest.transaction_id; + LOG_INFO(log, "Export task {} was deleted, calling killExportPartition for transaction {}", kv.first, transaction_id); + + try + { + killExportPart(transaction_id); + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } + + return true; + }); if (cleanup_lock_acquired) { @@ -4841,9 +4860,10 @@ void StorageReplicatedMergeTree::selectPartsToExport() exportPartToTable( part_to_export.value(), destination_storage_id, + transaction_id, getContext(), [this, partition_id, transaction_id, exports_path, key, part_to_export, complete_part_export, partition_path, next_idx_path, next_idx, destination_storage] - (MergeTreeExportManifest::CompletionCallbackResult result) + (MergeTreePartExportManifest::CompletionCallbackResult result) { const auto zk_client = getZooKeeper(); if (result.success) @@ -8680,18 +8700,6 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & const auto parts = getDataPartsVectorInPartitionForInternalUsage(MergeTreeDataPartState::Active, partition_id, &data_parts_lock); - // const Strings parts = zookeeper->getChildren(fs::path(replica_path) / "parts"); - // const ActiveDataPartSet active_parts_set(format_version, parts); - // const auto part_infos = active_parts_set.getPartInfos(); - // std::vector parts_to_export; - // for (const auto & part : parts) - // { - // if (part_info.getPartitionId() == partition_id) - // { - // parts_to_export.push_back(part_info.getPartNameV1()); - // } - // } - if (parts.empty()) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition {} doesn't exist", partition_id); @@ -8707,7 +8715,7 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & ExportReplicatedMergeTreePartitionManifest manifest; - manifest.transaction_id = std::to_string(generateSnowflakeID()); + manifest.transaction_id = query_context->getCurrentQueryId(); manifest.partition_id = partition_id; manifest.destination_database = dest_database; manifest.destination_table = dest_table; @@ -10161,6 +10169,49 @@ CancellationCode StorageReplicatedMergeTree::killPartMoveToShard(const UUID & ta return part_moves_between_shards_orchestrator.killPartMoveToShard(task_uuid); } +CancellationCode StorageReplicatedMergeTree::killExportPartition(const String & transaction_id) +{ + auto zk = getZooKeeper(); + const auto exports_path = fs::path(zookeeper_path) / "exports"; + + const auto export_keys = zk->getChildren(exports_path); + String export_key_to_be_cancelled; + for (const auto & export_key : export_keys) + { + std::string status; + if (!zk->tryGet(fs::path(exports_path) / export_key / "status", status)) + continue; + if (status != "PENDING") + continue; + + std::string metadata_json; + if (!zk->tryGet(fs::path(exports_path) / export_key / "metadata.json", metadata_json)) + continue; + const auto manifest = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + if (manifest.transaction_id == transaction_id) + { + export_key_to_be_cancelled = export_key; + break; + } + } + + if (export_key_to_be_cancelled.empty()) + return CancellationCode::NotFound; + + try + { + /// Once the entry is removed from zk, the update task will be triggered in all replicas + /// The logic for cancelling individual part exports will be triggered in the update task. + zk->removeRecursive(fs::path(exports_path) / export_key_to_be_cancelled); + return CancellationCode::CancelSent; + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + return CancellationCode::CancelCannotBeSent; + } +} + void StorageReplicatedMergeTree::getCommitPartOps( Coordination::Requests & ops, const DataPartPtr & part, diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 9cdbf9c9dd10..3e7af55023d9 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -13,6 +13,7 @@ #include #include #include +#include "Interpreters/CancellationCode.h" #include #include #include @@ -936,6 +937,7 @@ class StorageReplicatedMergeTree final : public MergeTreeData void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr query_context) override; void movePartitionToShard(const ASTPtr & partition, bool move_part, const String & to, ContextPtr query_context) override; CancellationCode killPartMoveToShard(const UUID & task_uuid) override; + CancellationCode killExportPartition(const String & transaction_id) override; void fetchPartition( const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, From bb04fd961990813779040f1fe407c69e2fc5093c Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 22 Oct 2025 09:42:25 -0300 Subject: [PATCH 17/58] working kill export, update next idx upon lock and lock as many parts as one can at once --- src/Storages/MergeTree/ExportPartTask.cpp | 247 ++++++++++++++++++ src/Storages/MergeTree/ExportPartTask.h | 32 +++ src/Storages/MergeTree/MergeTreeData.cpp | 43 ++- src/Storages/MergeTree/MergeTreeData.h | 1 + .../MergeTree/MergeTreePartExportManifest.h | 9 +- src/Storages/StorageReplicatedMergeTree.cpp | 74 +++--- 6 files changed, 348 insertions(+), 58 deletions(-) create mode 100644 src/Storages/MergeTree/ExportPartTask.cpp create mode 100644 src/Storages/MergeTree/ExportPartTask.h diff --git a/src/Storages/MergeTree/ExportPartTask.cpp b/src/Storages/MergeTree/ExportPartTask.cpp new file mode 100644 index 000000000000..4b247946c76a --- /dev/null +++ b/src/Storages/MergeTree/ExportPartTask.cpp @@ -0,0 +1,247 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace ProfileEvents +{ + extern const Event PartsExportDuplicated; + extern const Event PartsExportFailures; + extern const Event PartsExports; + extern const Event PartsExportTotalMilliseconds; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_TABLE; + extern const int FILE_ALREADY_EXISTS; + extern const int LOGICAL_ERROR; +} + +namespace Setting +{ + extern const SettingsUInt64 min_bytes_to_use_direct_io; +} + +ExportPartTask::ExportPartTask(MergeTreeData & storage_, const MergeTreePartExportManifest & manifest_, ContextPtr context_) + : storage(storage_), + manifest(manifest_), + local_context(context_) +{ +} + +bool ExportPartTask::executeStep() +{ + auto metadata_snapshot = storage.getInMemoryMetadataPtr(); + Names columns_to_read = metadata_snapshot->getColumns().getNamesOfPhysical(); + StorageSnapshotPtr storage_snapshot = storage.getStorageSnapshot(metadata_snapshot, local_context); + + MergeTreeSequentialSourceType read_type = MergeTreeSequentialSourceType::Export; + + NamesAndTypesList partition_columns; + if (metadata_snapshot->hasPartitionKey()) + { + const auto & partition_key = metadata_snapshot->getPartitionKey(); + if (!partition_key.column_names.empty()) + partition_columns = partition_key.expression->getRequiredColumnsWithTypes(); + } + + auto block_with_partition_values = manifest.data_part->partition.getBlockWithPartitionValues(partition_columns); + + auto destination_storage = DatabaseCatalog::instance().tryGetTable(manifest.destination_storage_id, local_context); + if (!destination_storage) + { + std::lock_guard inner_lock(storage.export_manifests_mutex); + + const auto destination_storage_id_name = manifest.destination_storage_id.getNameForLogs(); + storage.export_manifests.erase(manifest); + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Failed to reconstruct destination storage: {}", destination_storage_id_name); + } + + SinkToStoragePtr sink; + std::string destination_file_path; + + try + { + auto context_copy = Context::createCopy(local_context); + context_copy->setSetting("output_format_parallel_formatting", manifest.parallel_formatting); + + sink = destination_storage->import( + manifest.data_part->name + "_" + manifest.data_part->checksums.getTotalChecksumHex(), + block_with_partition_values, + destination_file_path, + manifest.overwrite_file_if_exists, + context_copy); + } + catch (const Exception & e) + { + if (e.code() == ErrorCodes::FILE_ALREADY_EXISTS) + { + ProfileEvents::increment(ProfileEvents::PartsExportDuplicated); + } + + ProfileEvents::incrementNoTrace(ProfileEvents::PartsExportFailures); + + std::lock_guard inner_lock(storage.export_manifests_mutex); + storage.export_manifests.erase(manifest); + + if (manifest.completion_callback) + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createFailure(e.message())); + return false; + } + + bool apply_deleted_mask = true; + bool read_with_direct_io = local_context->getSettingsRef()[Setting::min_bytes_to_use_direct_io] > manifest.data_part->getBytesOnDisk(); + bool prefetch = false; + + MergeTreeData::IMutationsSnapshot::Params params + { + .metadata_version = metadata_snapshot->getMetadataVersion(), + .min_part_metadata_version = manifest.data_part->getMetadataVersion(), + }; + + auto mutations_snapshot = storage.getMutationsSnapshot(params); + + auto alter_conversions = MergeTreeData::getAlterConversionsForPart( + manifest.data_part, + mutations_snapshot, + local_context); + + QueryPlan plan_for_part; + + createReadFromPartStep( + read_type, + plan_for_part, + storage, + storage_snapshot, + RangesInDataPart(manifest.data_part), + alter_conversions, + nullptr, + columns_to_read, + nullptr, + apply_deleted_mask, + std::nullopt, + read_with_direct_io, + prefetch, + local_context, + getLogger("ExportPartition")); + + auto exports_list_entry = storage.getContext()->getExportsList().insert( + getStorageID(), + manifest.destination_storage_id, + manifest.data_part->getBytesOnDisk(), + manifest.data_part->name, + destination_file_path, + manifest.data_part->rows_count, + manifest.data_part->getBytesOnDisk(), + manifest.data_part->getBytesUncompressedOnDisk(), + manifest.create_time, + local_context); + + ThreadGroupSwitcher switcher((*exports_list_entry)->thread_group, ""); + + QueryPlanOptimizationSettings optimization_settings(local_context); + auto pipeline_settings = BuildQueryPipelineSettings(local_context); + auto builder = plan_for_part.buildQueryPipeline(optimization_settings, pipeline_settings); + + builder->setProgressCallback([&exports_list_entry](const Progress & progress) + { + (*exports_list_entry)->bytes_read_uncompressed += progress.read_bytes; + (*exports_list_entry)->rows_read += progress.read_rows; + (*exports_list_entry)->elapsed = (*exports_list_entry)->watch.elapsedSeconds(); + }); + + pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + + pipeline.complete(sink); + + try + { + CompletedPipelineExecutor exec(pipeline); + exec.execute(); + + std::lock_guard inner_lock(storage.export_manifests_mutex); + storage.writePartLog( + PartLogElement::Type::EXPORT_PART, + {}, + static_cast((*exports_list_entry)->elapsed * 1000000000), + manifest.data_part->name, + manifest.data_part, + {manifest.data_part}, + nullptr, + nullptr, + exports_list_entry.get()); + + storage.export_manifests.erase(manifest); + + ProfileEvents::increment(ProfileEvents::PartsExports); + ProfileEvents::increment(ProfileEvents::PartsExportTotalMilliseconds, static_cast((*exports_list_entry)->elapsed * 1000)); + + if (manifest.completion_callback) + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createSuccess(destination_file_path)); + } + catch (const Exception & e) + { + tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("while exporting the part {}. User should retry.", manifest.data_part->name)); + + ProfileEvents::increment(ProfileEvents::PartsExportFailures); + + std::lock_guard inner_lock(storage.export_manifests_mutex); + storage.writePartLog( + PartLogElement::Type::EXPORT_PART, + ExecutionStatus::fromCurrentException("", true), + static_cast((*exports_list_entry)->elapsed * 1000000000), + manifest.data_part->name, + manifest.data_part, + {manifest.data_part}, + nullptr, + nullptr, + exports_list_entry.get()); + + storage.export_manifests.erase(manifest); + + if (manifest.completion_callback) + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createFailure(e.message())); + + throw; + } + return false; +} + +void ExportPartTask::cancel() noexcept +{ + pipeline.cancel(); +} + +void ExportPartTask::onCompleted() +{ +} + +StorageID ExportPartTask::getStorageID() const +{ + return manifest.destination_storage_id; +} + +Priority ExportPartTask::getPriority() const +{ + return Priority{}; +} + +String ExportPartTask::getQueryId() const +{ + return manifest.query_id; +} + +} diff --git a/src/Storages/MergeTree/ExportPartTask.h b/src/Storages/MergeTree/ExportPartTask.h new file mode 100644 index 000000000000..790ef9f6ecba --- /dev/null +++ b/src/Storages/MergeTree/ExportPartTask.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class ExportPartTask : public IExecutableTask +{ +public: + explicit ExportPartTask( + MergeTreeData & storage_, + const MergeTreePartExportManifest & manifest_, + ContextPtr context_); + bool executeStep() override; + void onCompleted() override; + StorageID getStorageID() const override; + Priority getPriority() const override; + String getQueryId() const override; + + void cancel() noexcept override; + +private: + MergeTreeData & storage; + MergeTreePartExportManifest manifest; + ContextPtr local_context; + QueryPipeline pipeline; +}; + +} diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8f9d549992c8..916c6f0e32ee 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -24,6 +24,7 @@ #include #include #include +#include "Storages/MergeTree/ExportPartTask.h" #include #include #include @@ -6395,9 +6396,6 @@ void MergeTreeData::exportPartToTableImpl( pipeline.complete(sink); - /// oh boy, is there another way? - manifest.pipeline = &pipeline; - try { CompletedPipelineExecutor exec(pipeline); @@ -6454,18 +6452,16 @@ void MergeTreeData::killExportPart(const String & query_id) { std::lock_guard lock(export_manifests_mutex); - const auto it = std::find_if(export_manifests.begin(), export_manifests.end(), [&](const auto & manifest) + std::erase_if(export_manifests, [&](const auto & manifest) { - return manifest.query_id == query_id; + if (manifest.query_id == query_id) + { + if (manifest.task) + manifest.task->cancel(); + return true; + } + return false; }); - - if (it == export_manifests.end()) - return; - - if (it->pipeline) - it->pipeline->cancel(); - - export_manifests.erase(it); } void MergeTreeData::movePartitionToShard(const ASTPtr & /*partition*/, bool /*move_part*/, const String & /*to*/, ContextPtr /*query_context*/) @@ -9306,17 +9302,18 @@ bool MergeTreeData::scheduleDataMovingJob(BackgroundJobsAssignee & assignee) continue; } - manifest.in_progress = assignee.scheduleMoveTask(std::make_shared( - [this, &manifest] () mutable { - /// TODO arthur fix this: I need to be able to modify the real manifest - /// but grabbing it by reference is causing problems - exportPartToTableImpl(manifest, getContext()); - return true; - }, - moves_assignee_trigger, - getStorageID())); + auto task = std::make_shared(*this, manifest, getContext()); + + manifest.in_progress = assignee.scheduleMoveTask(task); + + if (!manifest.in_progress) + { + continue; + } - return manifest.in_progress; + manifest.task = task; + + return true; } return false; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index b2b748c86ab5..6c7d6c0218ba 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1362,6 +1362,7 @@ class MergeTreeData : public IStorage, public WithMutableContext friend class MergeTask; friend class IPartMetadataManager; friend class IMergedBlockOutputStream; // for access to log + friend class ExportPartTask; bool require_part_metadata; diff --git a/src/Storages/MergeTree/MergeTreePartExportManifest.h b/src/Storages/MergeTree/MergeTreePartExportManifest.h index 3052bccec6bc..dd55b21c8a23 100644 --- a/src/Storages/MergeTree/MergeTreePartExportManifest.h +++ b/src/Storages/MergeTree/MergeTreePartExportManifest.h @@ -1,10 +1,14 @@ +#pragma once + #include #include -#include "QueryPipeline/QueryPipeline.h" +#include namespace DB { +class ExportPartTask; + struct MergeTreePartExportManifest { using DataPartPtr = std::shared_ptr; @@ -58,8 +62,7 @@ struct MergeTreePartExportManifest time_t create_time; mutable bool in_progress = false; - /// Used for killing the export - mutable QueryPipeline * pipeline = nullptr; + mutable std::shared_ptr task = nullptr; bool operator<(const MergeTreePartExportManifest & rhs) const { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7bcbc6b67944..3f908e652585 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4688,21 +4688,6 @@ void StorageReplicatedMergeTree::selectPartsToExport() return false; } - // std::string parts_to_do_zk; - // if (!zk->tryGet(fs::path(export_partition_path) / "parts_to_do", parts_to_do_zk)) - // { - // LOG_INFO(log, "Failed to get parts_to_do, skipping"); - // return false; - // } - - // const auto parts_to_do = std::stoull(parts_to_do_zk.c_str()); - - // if (parts_to_do == 0) - // { - // LOG_INFO(log, "Skipping... Parts to do is 0, maybe someone else already completed it?"); - // return false; - // } - /// only try to lock it if the status is still PENDING /// if we did not check for status = pending, chances are some other replica completed and released the lock in the meantime ops.emplace_back(zkutil::makeCheckRequest(status_path, stat.version)); @@ -4733,9 +4718,31 @@ void StorageReplicatedMergeTree::selectPartsToExport() auto try_to_acquire_a_part = [&]( const ExportReplicatedMergeTreePartitionManifest & manifest, const std::string & partition_export_path, - const std::size_t next_idx, + std::size_t & next_idx, const ZooKeeperPtr & zk) -> std::optional { + const auto try_to_update_next_idx = [&](std::size_t local_next_idx) + { + /// Update next_idx - best effort, if it fails, that is ok + Coordination::Stat next_idx_stat; + std::string next_idx_string; + const auto next_idx_path = fs::path(partition_export_path) / "next_idx"; + + if (zk->tryGet(next_idx_path, next_idx_string, &next_idx_stat)) + { + const std::size_t next_idx_zk = std::stoul(next_idx_string.c_str()); + const std::size_t new_next_idx = std::max(next_idx_zk, local_next_idx + 1); + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(next_idx_path, next_idx_stat.version)); + ops.emplace_back(zkutil::makeSetRequest(next_idx_path, std::to_string(new_next_idx), next_idx_stat.version)); + Coordination::Responses responses; + if (zk->tryMulti(ops, responses) != Coordination::Error::ZOK) + { + LOG_INFO(log, "Updated next_idx to {}", new_next_idx); + } + } + }; + for (auto i = next_idx; i < manifest.number_of_parts; i++) { const auto part_path = fs::path(partition_export_path) / "parts" / manifest.parts[i]; @@ -4751,6 +4758,8 @@ void StorageReplicatedMergeTree::selectPartsToExport() if (lock_part(part_path, status_path, lock_path, replica_name, zk)) { + next_idx = i; + try_to_update_next_idx(i); return manifest.parts[i]; } } @@ -4772,6 +4781,8 @@ void StorageReplicatedMergeTree::selectPartsToExport() if (lock_part(part_path, status_path, lock_path, replica_name, zk)) { + next_idx = i; + try_to_update_next_idx(i); return manifest.parts[i]; } } @@ -4822,21 +4833,21 @@ void StorageReplicatedMergeTree::selectPartsToExport() LOG_INFO(log, "Skipping... Status is not PENDING"); continue; } - + const auto & manifest = task_entry.manifest; const auto & database = getContext()->resolveDatabase(manifest.destination_database); const auto & table = manifest.destination_table; - + const auto destination_storage_id = StorageID(QualifiedTableName {database, table}); - + const auto destination_storage = DatabaseCatalog::instance().tryGetTable(destination_storage_id, getContext()); - + if (!destination_storage) { LOG_INFO(log, "Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); continue; } - + const auto partition_path = fs::path(exports_path) / key; const auto next_idx_path = fs::path(partition_path) / "next_idx"; std::string next_idx_string; @@ -4845,16 +4856,13 @@ void StorageReplicatedMergeTree::selectPartsToExport() LOG_INFO(log, "Failed to get next_idx, skipping"); continue; } - - const auto next_idx = std::stoull(next_idx_string.c_str()); - - const auto part_to_export = try_to_acquire_a_part(manifest, partition_path, next_idx, zk); - - const auto partition_id = manifest.partition_id; - const auto transaction_id = manifest.transaction_id; - - if (part_to_export.has_value()) + + std::size_t next_idx = std::stoull(next_idx_string.c_str()); + while (const auto part_to_export = try_to_acquire_a_part(manifest, partition_path, next_idx, zk)) { + const auto partition_id = manifest.partition_id; + const auto transaction_id = manifest.transaction_id; + try { exportPartToTable( @@ -4969,11 +4977,14 @@ void StorageReplicatedMergeTree::selectPartsToExport() /// best-effort to remove the lock (actually, we should make sure the lock is released..) zk->tryRemove(fs::path(partition_path) / "parts" / part_to_export.value() / "lock"); + + /// todo arthur should I try to rewind the next_idx? /// re-run after some time export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); + + break; } - } } } @@ -4982,7 +4993,6 @@ void StorageReplicatedMergeTree::selectPartsToExport() tryLogCurrentException(log, __PRETTY_FUNCTION__); export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); } - // export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); } std::vector StorageReplicatedMergeTree::getPartitionExportsInfo() const From 55e7b9425f45ca9955c6cbb546d0553191339c11 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 22 Oct 2025 13:46:11 -0300 Subject: [PATCH 18/58] fix conflicts --- src/Interpreters/InterpreterKillQueryQuery.cpp | 4 ++-- src/Parsers/ASTAlterQuery.cpp | 6 +++--- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- .../0_stateless/03604_export_merge_tree_partition.reference | 1 + 4 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 3b68ef39b16c..bd91cf613acc 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -255,7 +255,7 @@ BlockIO InterpreterKillQueryQuery::execute() Block exports_block = getSelectResult( "source_database, source_table, transaction_id, destination_database, destination_table, partition_id", "system.replicated_partition_exports"); - if (!exports_block) + if (exports_block.empty()) return res_io; const ColumnString & src_db_col = typeid_cast(*exports_block.getByName("source_database").column); @@ -317,7 +317,7 @@ BlockIO InterpreterKillQueryQuery::execute() throw Exception(ErrorCodes::ACCESS_DENIED, "Not allowed to kill export partition. " "To execute this query, it's necessary to have the grant {}", required_access_rights.toString()); - res_io.pipeline = QueryPipeline(Pipe(std::make_shared(header.cloneWithColumns(std::move(res_columns))))); + res_io.pipeline = QueryPipeline(Pipe(std::make_shared(std::make_shared(header.cloneWithColumns(std::move(res_columns)))))); break; } diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index a777280bea69..57d81c92898f 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -380,14 +380,14 @@ void ASTAlterCommand::formatImpl(WriteBuffer & ostr, const FormatSettings & sett } else if (type == ASTAlterCommand::EXPORT_PARTITION) { - ostr << (settings.hilite ? hilite_keyword : "") << "EXPORT PARTITION " << (settings.hilite ? hilite_none : ""); + ostr << "EXPORT PARTITION "; partition->format(ostr, settings, state, frame); ostr << " TO TABLE "; if (!to_database.empty()) { - ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(to_database) << (settings.hilite ? hilite_none : "") << "."; + ostr << backQuoteIfNeed(to_database) << "."; } - ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(to_table) << (settings.hilite ? hilite_none : ""); + ostr << backQuoteIfNeed(to_table); } else if (type == ASTAlterCommand::REPLACE_PARTITION) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 3f908e652585..1c6543f001f2 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8701,7 +8701,7 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & } /// The check for existence and entry removal are not atomic, so this actually might fail. - ops.emplace_back(zkutil::makeRemoveRecursiveRequest(partition_exports_path, -1)); + ops.emplace_back(zkutil::makeRemoveRecursiveRequest(*zookeeper, partition_exports_path, -1)); } ops.emplace_back(zkutil::makeCreateRequest(partition_exports_path, "", zkutil::CreateMode::Persistent)); diff --git a/tests/queries/0_stateless/03604_export_merge_tree_partition.reference b/tests/queries/0_stateless/03604_export_merge_tree_partition.reference index 2bda7b6165a4..d48023362b99 100644 --- a/tests/queries/0_stateless/03604_export_merge_tree_partition.reference +++ b/tests/queries/0_stateless/03604_export_merge_tree_partition.reference @@ -11,6 +11,7 @@ Select from destination table 2 2020 3 2020 4 2021 +5 2021 Export partition 2022 Select from destination table again 1 2020 From abe14f33e21f63d9aa872373f504799bc506c950 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 23 Oct 2025 10:12:45 -0300 Subject: [PATCH 19/58] rmv from system.exports --- src/Storages/MergeTree/ExportList.cpp | 8 ++++++++ src/Storages/MergeTree/ExportList.h | 2 ++ src/Storages/MergeTree/MergeTreeData.cpp | 2 ++ 3 files changed, 12 insertions(+) diff --git a/src/Storages/MergeTree/ExportList.cpp b/src/Storages/MergeTree/ExportList.cpp index 0239f841dc69..a00e4774fee7 100644 --- a/src/Storages/MergeTree/ExportList.cpp +++ b/src/Storages/MergeTree/ExportList.cpp @@ -63,4 +63,12 @@ UInt64 ExportsListElement::getPeakMemoryUsage() const return thread_group->memory_tracker.getPeak(); } +void ExportsList::remove(const StorageID & source_table_id, const StorageID & destination_table_id, const String & part_name) +{ + std::erase_if(entries, [source_table_id, destination_table_id, part_name](const auto & entry) + { + return entry.source_table_id == source_table_id && entry.destination_table_id == destination_table_id && entry.part_name == part_name; + }); +} + } diff --git a/src/Storages/MergeTree/ExportList.h b/src/Storages/MergeTree/ExportList.h index ade18b69480c..28aecda7a137 100644 --- a/src/Storages/MergeTree/ExportList.h +++ b/src/Storages/MergeTree/ExportList.h @@ -83,6 +83,8 @@ class ExportsList final : public BackgroundProcessList; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 916c6f0e32ee..d472774813e0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6458,6 +6458,8 @@ void MergeTreeData::killExportPart(const String & query_id) { if (manifest.task) manifest.task->cancel(); + + getContext()->getExportsList().remove(getStorageID(), manifest.destination_storage_id, manifest.data_part->name); return true; } return false; From e22579816f45d7317342d919b27bafe98d527921 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 23 Oct 2025 10:48:51 -0300 Subject: [PATCH 20/58] add no fasttest --- tests/config/config.d/clusters.xml | 13 +++++++++++++ .../03604_export_merge_tree_partition.sh | 2 +- 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/tests/config/config.d/clusters.xml b/tests/config/config.d/clusters.xml index 1a507a76fd85..de2293cf458f 100644 --- a/tests/config/config.d/clusters.xml +++ b/tests/config/config.d/clusters.xml @@ -162,6 +162,19 @@ + + + false + + 127.0.0.1 + 9000 + + + 127.0.0.1 + 9100 + + + false diff --git a/tests/queries/0_stateless/03604_export_merge_tree_partition.sh b/tests/queries/0_stateless/03604_export_merge_tree_partition.sh index d27126469603..87503112aadb 100755 --- a/tests/queries/0_stateless/03604_export_merge_tree_partition.sh +++ b/tests/queries/0_stateless/03604_export_merge_tree_partition.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: replica, no-parallel, no-replicated-database +# Tags: no-fasttest, replica, no-parallel, no-replicated-database CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 0ca5e28bd92a32e100997cb3bc7fa806c5bc7b0d Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 23 Oct 2025 15:42:25 -0300 Subject: [PATCH 21/58] some adjustments --- src/Storages/MergeTree/ExportPartTask.cpp | 11 +- src/Storages/MergeTree/MergeTreeData.cpp | 176 ------------------ src/Storages/MergeTree/MergeTreeData.h | 4 - ...xport_merge_tree_part_to_object_storage.sh | 5 +- 4 files changed, 8 insertions(+), 188 deletions(-) diff --git a/src/Storages/MergeTree/ExportPartTask.cpp b/src/Storages/MergeTree/ExportPartTask.cpp index 4b247946c76a..4fe13d7a8425 100644 --- a/src/Storages/MergeTree/ExportPartTask.cpp +++ b/src/Storages/MergeTree/ExportPartTask.cpp @@ -50,16 +50,13 @@ bool ExportPartTask::executeStep() MergeTreeSequentialSourceType read_type = MergeTreeSequentialSourceType::Export; - NamesAndTypesList partition_columns; + Block block_with_partition_values; if (metadata_snapshot->hasPartitionKey()) { - const auto & partition_key = metadata_snapshot->getPartitionKey(); - if (!partition_key.column_names.empty()) - partition_columns = partition_key.expression->getRequiredColumnsWithTypes(); + /// todo arthur do I need to init minmax_idx? + block_with_partition_values = manifest.data_part->minmax_idx->getBlock(storage); } - auto block_with_partition_values = manifest.data_part->partition.getBlockWithPartitionValues(partition_columns); - auto destination_storage = DatabaseCatalog::instance().tryGetTable(manifest.destination_storage_id, local_context); if (!destination_storage) { @@ -92,7 +89,7 @@ bool ExportPartTask::executeStep() ProfileEvents::increment(ProfileEvents::PartsExportDuplicated); } - ProfileEvents::incrementNoTrace(ProfileEvents::PartsExportFailures); + ProfileEvents::increment(ProfileEvents::PartsExportFailures); std::lock_guard inner_lock(storage.export_manifests_mutex); storage.export_manifests.erase(manifest); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d472774813e0..0ad3d72328e6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6272,182 +6272,6 @@ void MergeTreeData::exportPartToTable( background_moves_assignee.trigger(); } -void MergeTreeData::exportPartToTableImpl( - const MergeTreePartExportManifest & manifest, - ContextPtr local_context) -{ - auto metadata_snapshot = getInMemoryMetadataPtr(); - Names columns_to_read = metadata_snapshot->getColumns().getNamesOfPhysical(); - StorageSnapshotPtr storage_snapshot = getStorageSnapshot(metadata_snapshot, local_context); - - MergeTreeSequentialSourceType read_type = MergeTreeSequentialSourceType::Export; - - Block block_with_partition_values; - if (metadata_snapshot->hasPartitionKey()) - { - /// todo arthur do I need to init minmax_idx? - block_with_partition_values = manifest.data_part->minmax_idx->getBlock(*this); - } - - auto destination_storage = DatabaseCatalog::instance().tryGetTable(manifest.destination_storage_id, getContext()); - if (!destination_storage) - { - std::lock_guard inner_lock(export_manifests_mutex); - - const auto destination_storage_id_name = manifest.destination_storage_id.getNameForLogs(); - export_manifests.erase(manifest); - throw Exception(ErrorCodes::UNKNOWN_TABLE, "Failed to reconstruct destination storage: {}", destination_storage_id_name); - } - - SinkToStoragePtr sink; - std::string destination_file_path; - - try - { - auto context_copy = Context::createCopy(local_context); - context_copy->setSetting("output_format_parallel_formatting", manifest.parallel_formatting); - - sink = destination_storage->import( - manifest.data_part->name + "_" + manifest.data_part->checksums.getTotalChecksumHex(), - block_with_partition_values, - destination_file_path, - manifest.overwrite_file_if_exists, - context_copy); - } - catch (const Exception & e) - { - if (e.code() == ErrorCodes::FILE_ALREADY_EXISTS) - { - ProfileEvents::increment(ProfileEvents::PartsExportDuplicated); - } - - ProfileEvents::increment(ProfileEvents::PartsExportFailures); - - std::lock_guard inner_lock(export_manifests_mutex); - export_manifests.erase(manifest); - - if (manifest.completion_callback) - manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createFailure(e.message())); - return; - } - - bool apply_deleted_mask = true; - bool read_with_direct_io = local_context->getSettingsRef()[Setting::min_bytes_to_use_direct_io] > manifest.data_part->getBytesOnDisk(); - bool prefetch = false; - - MergeTreeData::IMutationsSnapshot::Params params - { - .metadata_version = metadata_snapshot->getMetadataVersion(), - .min_part_metadata_version = manifest.data_part->getMetadataVersion(), - }; - - auto mutations_snapshot = getMutationsSnapshot(params); - - auto alter_conversions = MergeTreeData::getAlterConversionsForPart( - manifest.data_part, - mutations_snapshot, - local_context); - - QueryPlan plan_for_part; - - createReadFromPartStep( - read_type, - plan_for_part, - *this, - storage_snapshot, - RangesInDataPart(manifest.data_part), - alter_conversions, - nullptr, - columns_to_read, - nullptr, - apply_deleted_mask, - std::nullopt, - read_with_direct_io, - prefetch, - local_context, - getLogger("ExportPartition")); - - auto exports_list_entry = getContext()->getExportsList().insert( - getStorageID(), - manifest.destination_storage_id, - manifest.data_part->getBytesOnDisk(), - manifest.data_part->name, - destination_file_path, - manifest.data_part->rows_count, - manifest.data_part->getBytesOnDisk(), - manifest.data_part->getBytesUncompressedOnDisk(), - manifest.create_time, - local_context); - - ThreadGroupSwitcher switcher((*exports_list_entry)->thread_group, ""); - - QueryPlanOptimizationSettings optimization_settings(local_context); - auto pipeline_settings = BuildQueryPipelineSettings(local_context); - auto builder = plan_for_part.buildQueryPipeline(optimization_settings, pipeline_settings); - - builder->setProgressCallback([&exports_list_entry](const Progress & progress) - { - (*exports_list_entry)->bytes_read_uncompressed += progress.read_bytes; - (*exports_list_entry)->rows_read += progress.read_rows; - (*exports_list_entry)->elapsed = (*exports_list_entry)->watch.elapsedSeconds(); - }); - - auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); - - pipeline.complete(sink); - - try - { - CompletedPipelineExecutor exec(pipeline); - exec.execute(); - - std::lock_guard inner_lock(export_manifests_mutex); - writePartLog( - PartLogElement::Type::EXPORT_PART, - {}, - static_cast((*exports_list_entry)->elapsed * 1000000000), - manifest.data_part->name, - manifest.data_part, - {manifest.data_part}, - nullptr, - nullptr, - exports_list_entry.get()); - - export_manifests.erase(manifest); - - ProfileEvents::increment(ProfileEvents::PartsExports); - ProfileEvents::increment(ProfileEvents::PartsExportTotalMilliseconds, static_cast((*exports_list_entry)->elapsed * 1000)); - - if (manifest.completion_callback) - manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createSuccess(destination_file_path)); - } - catch (const Exception & e) - { - tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("while exporting the part {}. User should retry.", manifest.data_part->name)); - - ProfileEvents::increment(ProfileEvents::PartsExportFailures); - - std::lock_guard inner_lock(export_manifests_mutex); - writePartLog( - PartLogElement::Type::EXPORT_PART, - ExecutionStatus::fromCurrentException("", true), - static_cast((*exports_list_entry)->elapsed * 1000000000), - manifest.data_part->name, - manifest.data_part, - {manifest.data_part}, - nullptr, - nullptr, - exports_list_entry.get()); - - export_manifests.erase(manifest); - - if (manifest.completion_callback) - manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createFailure(e.message())); - - throw; - } -} - void MergeTreeData::killExportPart(const String & query_id) { std::lock_guard lock(export_manifests_mutex); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 6c7d6c0218ba..00d809f9f32e 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -998,10 +998,6 @@ class MergeTreeData : public IStorage, public WithMutableContext throw Exception(ErrorCodes::NOT_IMPLEMENTED, "EXPORT PARTITION is not implemented"); } - void exportPartToTableImpl( - const MergeTreePartExportManifest & manifest, - ContextPtr local_context); - /// Checks that Partition could be dropped right now /// Otherwise - throws an exception with detailed information. /// We do not use mutex because it is not very important that the size could change during the operation. diff --git a/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage.sh b/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage.sh index 515feb9676d4..ae7f05dac90a 100755 --- a/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage.sh +++ b/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage.sh @@ -49,6 +49,8 @@ echo "---- Export 2020_1_1_0 and 2021_2_2_0 to wildcard table" query "ALTER TABLE $mt_table EXPORT PART '2020_1_1_0' TO TABLE $s3_table_wildcard SETTINGS allow_experimental_export_merge_tree_part = 1" query "ALTER TABLE $mt_table EXPORT PART '2021_2_2_0' TO TABLE $s3_table_wildcard SETTINGS allow_experimental_export_merge_tree_part = 1" +sleep 3 + echo "---- Both data parts should appear" query "SELECT * FROM s3(s3_conn, filename='$s3_table_wildcard/**.parquet') ORDER BY id" @@ -60,13 +62,14 @@ query "SELECT * FROM s3(s3_conn, filename='$s3_table_wildcard/**.parquet') ORDER query "CREATE TABLE $mt_table_partition_expression_with_function (id UInt64, year UInt16) ENGINE = MergeTree() PARTITION BY toString(year) ORDER BY tuple()" query "CREATE TABLE $s3_table_wildcard_partition_expression_with_function (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$s3_table_wildcard_partition_expression_with_function/{_partition_id}/{_file}.parquet', format=Parquet, partition_strategy='wildcard') PARTITION BY toString(year)" -# insert query "INSERT INTO $mt_table_partition_expression_with_function VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)" echo "---- Export 2020_1_1_0 and 2021_2_2_0 to wildcard table with partition expression with function" query "ALTER TABLE $mt_table_partition_expression_with_function EXPORT PART 'cb217c742dc7d143b61583011996a160_1_1_0' TO TABLE $s3_table_wildcard_partition_expression_with_function SETTINGS allow_experimental_export_merge_tree_part = 1" query "ALTER TABLE $mt_table_partition_expression_with_function EXPORT PART '3be6d49ecf9749a383964bc6fab22d10_2_2_0' TO TABLE $s3_table_wildcard_partition_expression_with_function SETTINGS allow_experimental_export_merge_tree_part = 1" +sleep 1 + echo "---- Both data parts should appear" query "SELECT * FROM s3(s3_conn, filename='$s3_table_wildcard_partition_expression_with_function/**.parquet') ORDER BY id" From 63c48ceafcf7632e44f76c8216ec2b7a450acd8b Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 23 Oct 2025 19:27:11 -0300 Subject: [PATCH 22/58] silly change to force cicd rebuild --- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 226d5efcc286..aa7cf00b06a3 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -477,6 +477,7 @@ bool StorageObjectStorage::supportsImport() const return configuration->partition_strategy_type == PartitionStrategyFactory::StrategyType::HIVE; } + SinkToStoragePtr StorageObjectStorage::import( const std::string & file_name, Block & block_with_partition_values, From ee00ebb89d793013f17a8ce51dd3439222756a18 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 28 Oct 2025 09:46:02 +0100 Subject: [PATCH 23/58] remove kind of dead code --- src/Storages/StorageReplicatedMergeTree.cpp | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1c6543f001f2..34363c690fbf 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4588,15 +4588,6 @@ void StorageReplicatedMergeTree::selectPartsToExport() return false; } - std::string next_idx_string; - if (!zk->tryGet(next_idx_path, next_idx_string, &next_idx_stat)) - { - LOG_INFO(log, "Failed to get next_idx, skipping"); - return false; - } - - const std::size_t next_idx_zk = std::stoull(next_idx_string.c_str()); - std::size_t parts_to_do = std::stoull(parts_to_do_string.c_str()); if (parts_to_do == 0) @@ -4610,8 +4601,6 @@ void StorageReplicatedMergeTree::selectPartsToExport() Coordination::Requests ops; ops.emplace_back(zkutil::makeCheckRequest(lock_path, lock_stat.version)); ops.emplace_back(zkutil::makeCheckRequest(parts_to_do_path, parts_to_do_stat.version)); - ops.emplace_back(zkutil::makeCheckRequest(next_idx_path, next_idx_stat.version)); - ops.emplace_back(zkutil::makeSetRequest(next_idx_path, std::to_string(std::max(next_idx_zk, next_idx_local + 1)), next_idx_stat.version)); ops.emplace_back(zkutil::makeSetRequest(part_status_path, "COMPLETED", -1)); ops.emplace_back(zkutil::makeRemoveRequest(lock_path, lock_stat.version)); ops.emplace_back(zkutil::makeSetRequest(parts_to_do_path, std::to_string(parts_to_do), parts_to_do_stat.version)); From 5c61bd69ffa67a68806845af07d1b79427ea2973 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 28 Oct 2025 11:14:51 +0100 Subject: [PATCH 24/58] small tweaks for demo --- src/Storages/StorageReplicatedMergeTree.cpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 34363c690fbf..b4238f94ec27 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4439,7 +4439,7 @@ void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() if (cleanup_lock_acquired) { - bool has_expired = metadata.create_time < now - 90; + bool has_expired = metadata.create_time < now - 180; if (has_expired && is_not_pending) { @@ -4550,8 +4550,6 @@ void StorageReplicatedMergeTree::selectPartsToExport() const std::string & part_status_path, const std::string & parts_to_do_path, const std::string & lock_path, - const std::string & next_idx_path, - const std::size_t next_idx_local, const std::string & path_in_destination_storage_path, const StoragePtr & destination_storage, const std::string & transaction_id, @@ -4561,7 +4559,6 @@ void StorageReplicatedMergeTree::selectPartsToExport() /// todo arthur is it possible to grab stats using a multi-op? Coordination::Stat parts_to_do_stat; Coordination::Stat lock_stat; - Coordination::Stat next_idx_stat; std::string parts_to_do_string; int retries = 0; @@ -4600,6 +4597,8 @@ void StorageReplicatedMergeTree::selectPartsToExport() Coordination::Requests ops; ops.emplace_back(zkutil::makeCheckRequest(lock_path, lock_stat.version)); + + /// there is a problem here: if someone else completed a part export before, this thing will fail.. ops.emplace_back(zkutil::makeCheckRequest(parts_to_do_path, parts_to_do_stat.version)); ops.emplace_back(zkutil::makeSetRequest(part_status_path, "COMPLETED", -1)); ops.emplace_back(zkutil::makeRemoveRequest(lock_path, lock_stat.version)); @@ -4854,12 +4853,15 @@ void StorageReplicatedMergeTree::selectPartsToExport() try { + /// todo arthur temporary for hackathon + auto context_copy = Context::createCopy(getContext()); + context_copy->setSetting("export_merge_tree_part_overwrite_file_if_exists", true); exportPartToTable( part_to_export.value(), destination_storage_id, transaction_id, - getContext(), - [this, partition_id, transaction_id, exports_path, key, part_to_export, complete_part_export, partition_path, next_idx_path, next_idx, destination_storage] + context_copy, + [this, partition_id, transaction_id, exports_path, key, part_to_export, complete_part_export, partition_path, destination_storage] (MergeTreePartExportManifest::CompletionCallbackResult result) { const auto zk_client = getZooKeeper(); @@ -4871,8 +4873,6 @@ void StorageReplicatedMergeTree::selectPartsToExport() fs::path(partition_path) / "parts" / part_to_export.value() / "status", fs::path(partition_path) / "parts_to_do", fs::path(partition_path) / "parts" / part_to_export.value() / "lock", - next_idx_path, - next_idx, result.relative_path_in_destination_storage, destination_storage, transaction_id, From d609d04a1277c0d30d2f46bc6f06b4e5b74d2a95 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 28 Oct 2025 11:16:30 +0100 Subject: [PATCH 25/58] todo comment --- src/Storages/StorageReplicatedMergeTree.cpp | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b4238f94ec27..b9c05612dba5 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4598,7 +4598,19 @@ void StorageReplicatedMergeTree::selectPartsToExport() Coordination::Requests ops; ops.emplace_back(zkutil::makeCheckRequest(lock_path, lock_stat.version)); - /// there is a problem here: if someone else completed a part export before, this thing will fail.. + /* + todo arthur: + Taking a note here so we can eventually discuss: + + Marking an individual part export as completed should ideally not depend on race conditions. Right now it depends. For instance: + + replica1 finished part1, and it is about to mark it as completed. It queries parts_to_do and creates the request that updates parts_to_do and the status. Before it sends that query, replica2 does the same and is able to update parts_to_do and its part2 status. + + At this point, replica1 will faill the entire request, which includes status=completed. There are a few ways around it: + + Upon failure, release the lock and let someoe else (or the same replica in the future) retry that part even tho it had already succeeded. + Implement retry logic that detects if the failure was because of the parts_to_do version and keeps retrying for a while until it eventually release the lock + */ ops.emplace_back(zkutil::makeCheckRequest(parts_to_do_path, parts_to_do_stat.version)); ops.emplace_back(zkutil::makeSetRequest(part_status_path, "COMPLETED", -1)); ops.emplace_back(zkutil::makeRemoveRequest(lock_path, lock_stat.version)); From b571f5af62eef6634eb4748f4b9de43b5e722b05 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 3 Nov 2025 21:18:28 +0100 Subject: [PATCH 26/58] wip - refactor new design --- ...ortReplicatedMergeTreePartitionTaskEntry.h | 1 - src/Storages/MergeTree/ExportList.cpp | 8 - src/Storages/MergeTree/ExportList.h | 2 - src/Storages/MergeTree/ExportPartTask.cpp | 3 +- .../ExportPartitionManifestUpdatingTask.cpp | 226 ++++++ .../ExportPartitionManifestUpdatingTask.h | 19 + .../ExportPartitionTaskScheduler.cpp | 296 ++++++++ .../MergeTree/ExportPartitionTaskScheduler.h | 51 ++ .../MergeTree/ExportPartitionUtils.cpp | 79 ++ src/Storages/MergeTree/ExportPartitionUtils.h | 28 + src/Storages/MergeTree/MergeTreeData.cpp | 3 +- .../MergeTree/MergeTreePartExportManifest.h | 4 + .../ObjectStorage/StorageObjectStorage.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 683 ++---------------- src/Storages/StorageReplicatedMergeTree.h | 7 + .../__init__.py | 0 .../configs/named_collections.xml | 9 + .../test.py | 170 +++++ 18 files changed, 942 insertions(+), 649 deletions(-) create mode 100644 src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp create mode 100644 src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h create mode 100644 src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp create mode 100644 src/Storages/MergeTree/ExportPartitionTaskScheduler.h create mode 100644 src/Storages/MergeTree/ExportPartitionUtils.cpp create mode 100644 src/Storages/MergeTree/ExportPartitionUtils.h create mode 100644 tests/integration/test_export_replicated_mt_partition_to_object_storage/__init__.py create mode 100644 tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/named_collections.xml create mode 100644 tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py diff --git a/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h index bdaef2ec3fa7..cf3b14629fae 100644 --- a/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h +++ b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h @@ -10,7 +10,6 @@ struct ExportReplicatedMergeTreePartitionTaskEntry using DataPartPtr = std::shared_ptr; ExportReplicatedMergeTreePartitionManifest manifest; - std::size_t parts_to_do; /// References to the parts that should be exported /// This is used to prevent the parts from being deleted before finishing the export operation /// It does not mean this replica will export all the parts diff --git a/src/Storages/MergeTree/ExportList.cpp b/src/Storages/MergeTree/ExportList.cpp index a00e4774fee7..0239f841dc69 100644 --- a/src/Storages/MergeTree/ExportList.cpp +++ b/src/Storages/MergeTree/ExportList.cpp @@ -63,12 +63,4 @@ UInt64 ExportsListElement::getPeakMemoryUsage() const return thread_group->memory_tracker.getPeak(); } -void ExportsList::remove(const StorageID & source_table_id, const StorageID & destination_table_id, const String & part_name) -{ - std::erase_if(entries, [source_table_id, destination_table_id, part_name](const auto & entry) - { - return entry.source_table_id == source_table_id && entry.destination_table_id == destination_table_id && entry.part_name == part_name; - }); -} - } diff --git a/src/Storages/MergeTree/ExportList.h b/src/Storages/MergeTree/ExportList.h index 28aecda7a137..ade18b69480c 100644 --- a/src/Storages/MergeTree/ExportList.h +++ b/src/Storages/MergeTree/ExportList.h @@ -83,8 +83,6 @@ class ExportsList final : public BackgroundProcessList; diff --git a/src/Storages/MergeTree/ExportPartTask.cpp b/src/Storages/MergeTree/ExportPartTask.cpp index 4fe13d7a8425..a72fbb1ded96 100644 --- a/src/Storages/MergeTree/ExportPartTask.cpp +++ b/src/Storages/MergeTree/ExportPartTask.cpp @@ -74,6 +74,7 @@ bool ExportPartTask::executeStep() { auto context_copy = Context::createCopy(local_context); context_copy->setSetting("output_format_parallel_formatting", manifest.parallel_formatting); + context_copy->setSetting("output_format_parquet_parallel_encoding", manifest.parquet_parallel_formatting); sink = destination_storage->import( manifest.data_part->name + "_" + manifest.data_part->checksums.getTotalChecksumHex(), @@ -228,7 +229,7 @@ void ExportPartTask::onCompleted() StorageID ExportPartTask::getStorageID() const { - return manifest.destination_storage_id; + return storage.getStorageID(); } Priority ExportPartTask::getPriority() const diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp new file mode 100644 index 000000000000..3948ae9532be --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp @@ -0,0 +1,226 @@ +#include +#include +#include +#include "Storages/MergeTree/ExportPartitionUtils.h" +#include +#include +#include + +namespace DB +{ + +/// v2 of my initial design +/* +table_path/ + exports/ + / + metadata.json -> {tid, partition_id, destination_id, create_time, ttl} + parts/ + processing/ <-- not started, in progress + part_1.json -> {retry_count, max_retry_count, path_in_destination} + ... + part_n.json + processed/ + part_1.json -> {retry_count, max_retry_count, path_in_destination} + ... + part_n.json + locks + part_1 -> r1 + part_n -> rN + cleanup_lock <--- ephemeral + + One of the ideas behind this design is to reduce the number of required CAS loops. + It should work as follows: + + upon request, the structure should be created in zk in case it does not exist. + + once the task is published in zk, replicas are notified there is a new task and will fetch it. + + once they have it loaded locally, eventually the scheduler thread will run and try to lock individual parts in that task to export. + + the lock process is kind of the following: + + try to create an ephemeral node with the aprt name under the `locks` path. If it succeeded, the part is locked and the task will be scheduled within that replica. + + if it fails, it means the part is already locked by another replica. Try the next part. + + Once it completes, moves the part structure that lives under processing to processed with status either of failed or succeeded. If it failed, it'll also fail the entire task. + + Also, once it completes a local part, after moving it to processed (a transaction). It tries to read `processing` to check if it is empty. + + If it is empty, it means all parts have been exported and it is time to commit the export. Note that this is not transactional with the previous operation of moving the part to processed. + + So it means there is a chance the last part will be exported, but the server might die right before checking processing path and will never commit. For this, the cleanup thread also helps + + This is the overall idea, but please read the code to get a better understanding +*/ + +ExportPartitionManifestUpdatingTask::ExportPartitionManifestUpdatingTask(StorageReplicatedMergeTree & storage_) + : storage(storage_) +{ +} + +void ExportPartitionManifestUpdatingTask::run() +{ + std::lock_guard lock(storage.export_merge_tree_partition_mutex); + + auto zk = storage.getZooKeeper(); + + const std::string exports_path = fs::path(storage.zookeeper_path) / "exports"; + const std::string cleanup_lock_path = fs::path(storage.zookeeper_path) / "exports_cleanup_lock"; + + bool cleanup_lock_acquired = zk->tryCreate(cleanup_lock_path, "", ::zkutil::CreateMode::Ephemeral) == Coordination::Error::ZOK; + + if (cleanup_lock_acquired) + { + LOG_INFO(storage.log, "ExportPartition: Cleanup lock acquired, will remove stale entries"); + } + + Coordination::Stat stat; + const auto children = zk->getChildrenWatch(exports_path, &stat, storage.export_merge_tree_partition_watch_callback); + const std::unordered_set zk_children(children.begin(), children.end()); + + const auto now = time(nullptr); + + /// Load new entries + /// If we have the cleanup lock, also remove stale entries from zk and local + /// Upload dangling commit files if any + for (const auto & key : zk_children) + { + const std::string entry_path = fs::path(exports_path) / key; + + std::string metadata_json; + if (!zk->tryGet(fs::path(entry_path) / "metadata.json", metadata_json)) + { + LOG_INFO(storage.log, "ExportPartition: Skipping {}: missing metadata.json", key); + continue; + } + + const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + + const auto local_entry = storage.export_merge_tree_partition_task_entries.find(key); + + /// If the zk entry has been replaced with export_merge_tree_partition_force_export, checking only for the export key is not enough + /// we need to make sure it is the same transaction id. If it is not, it needs to be replaced. + bool has_local_entry_and_is_up_to_date = local_entry != storage.export_merge_tree_partition_task_entries.end() + && local_entry->second.manifest.transaction_id == metadata.transaction_id; + + /// If the entry is up to date and we don't have the cleanup lock, early exit, nothing to be done. + if (!cleanup_lock_acquired && has_local_entry_and_is_up_to_date) + continue; + + std::string status; + if (!zk->tryGet(fs::path(entry_path) / "status", status)) + { + LOG_INFO(storage.log, "ExportPartition: Skipping {}: missing status", key); + continue; + } + + bool is_not_pending = status != "PENDING"; + + if (cleanup_lock_acquired) + { + bool has_expired = metadata.create_time < now - 180; + + if (has_expired && is_not_pending) + { + zk->tryRemoveRecursive(fs::path(entry_path)); + storage.export_merge_tree_partition_task_entries.erase(key); + LOG_INFO(storage.log, "ExportPartition: Removed {}: expired", key); + continue; + } + } + + if (is_not_pending) + { + LOG_INFO(storage.log, "ExportPartition: Skipping {}: status is not PENDING", key); + continue; + } + + + if (cleanup_lock_acquired) + { + std::vector parts_in_processing_or_pending; + if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(entry_path) / "processing", parts_in_processing_or_pending)) + { + LOG_INFO(storage.log, "ExportPartition: Failed to get parts in processing or pending, skipping"); + continue; + } + + if (parts_in_processing_or_pending.empty()) + { + LOG_INFO(storage.log, "ExportPartition: Cleanup found PENDING for {} with all parts exported, try to fix it by committing the export", entry_path); + + const auto destination_storage_id = StorageID(QualifiedTableName {metadata.destination_database, metadata.destination_table}); + const auto destination_storage = DatabaseCatalog::instance().tryGetTable(destination_storage_id, storage.getContext()); + if (!destination_storage) + { + LOG_INFO(storage.log, "ExportPartition: Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); + continue; + } + + /// it sounds like a replica exported the last part, but was not able to commit the export. Try to fix it + ExportPartitionUtils::commit(metadata, destination_storage, zk, storage.log.load(), entry_path, storage.getContext()); + } + } + + if (has_local_entry_and_is_up_to_date) + { + LOG_INFO(storage.log, "ExportPartition: Skipping {}: already exists", key); + continue; + } + + std::vector part_references; + + for (const auto & part_name : metadata.parts) + { + if (const auto part = storage.getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated})) + { + part_references.push_back(part); + } + } + + /// It is important to use the operator[] because it updates the existing entry if it already exists. + storage.export_merge_tree_partition_task_entries[key] = ExportReplicatedMergeTreePartitionTaskEntry {metadata, std::move(part_references)}; + } + + /// Remove entries that were deleted by someone else + std::erase_if(storage.export_merge_tree_partition_task_entries, + [&](auto const & kv) + { + if (zk_children.contains(kv.first)) + { + return false; + } + + const auto & transaction_id = kv.second.manifest.transaction_id; + LOG_INFO(storage.log, "ExportPartition: Export task {} was deleted, calling killExportPartition for transaction {}", kv.first, transaction_id); + + try + { + storage.killExportPart(transaction_id); + } + catch (...) + { + tryLogCurrentException(storage.log, __PRETTY_FUNCTION__); + } + + return true; + }); + + if (cleanup_lock_acquired) + { + zk->tryRemove(cleanup_lock_path); + } + + /// todo arthur remember to sort the entries by create_time + // std::sort(storage.export_merge_tree_partition_task_entries.begin(), storage.export_merge_tree_partition_task_entries.end(), + // [](const auto & a, const auto & b) + // { + // return a.second.manifest.create_time < b.second.manifest.create_time; + // }); + + storage.export_merge_tree_partition_select_task->schedule(); +} + +} diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h new file mode 100644 index 000000000000..48f740504303 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h @@ -0,0 +1,19 @@ +#pragma once + +namespace DB +{ + +class StorageReplicatedMergeTree; + +class ExportPartitionManifestUpdatingTask +{ +public: + ExportPartitionManifestUpdatingTask(StorageReplicatedMergeTree & storage); + + void run(); + +private: + StorageReplicatedMergeTree & storage; +}; + +} diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp new file mode 100644 index 000000000000..9e7c2b1084e7 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -0,0 +1,296 @@ +#include +#include +#include +#include +#include "Common/ZooKeeper/Types.h" +#include "Storages/MergeTree/ExportPartitionUtils.h" + + +namespace DB +{ + +ExportPartitionTaskScheduler::ExportPartitionTaskScheduler(StorageReplicatedMergeTree & storage_) + : storage(storage_) +{ +} + +void ExportPartitionTaskScheduler::run() +{ + std::lock_guard lock(storage.export_merge_tree_partition_mutex); + + auto zk = storage.getZooKeeper(); + + for (const auto & [key, entry] : storage.export_merge_tree_partition_task_entries) + { + + const auto & manifest = entry.manifest; + const auto & database = storage.getContext()->resolveDatabase(manifest.destination_database); + const auto & table = manifest.destination_table; + + const auto destination_storage_id = StorageID(QualifiedTableName {database, table}); + + const auto destination_storage = DatabaseCatalog::instance().tryGetTable(destination_storage_id, storage.getContext()); + + if (!destination_storage) + { + LOG_INFO(storage.log, "ExportPartition: Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); + continue; + } + + std::string status; + if (!zk->tryGet(fs::path(storage.zookeeper_path) / "exports" / key / "status", status)) + { + LOG_INFO(storage.log, "ExportPartition: Failed to get status, skipping"); + continue; + } + + if (status != "PENDING") + { + LOG_INFO(storage.log, "ExportPartition: Skipping... Status is not PENDING"); + continue; + } + + std::vector parts_in_processing_or_pending; + + if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(storage.zookeeper_path) / "exports" / key / "processing", parts_in_processing_or_pending)) + { + LOG_INFO(storage.log, "ExportPartition: Failed to get parts in processing or pending, skipping"); + continue; + } + + if (parts_in_processing_or_pending.empty()) + { + LOG_INFO(storage.log, "ExportPartition: No parts in processing or pending, skipping"); + continue; + } + + std::vector locked_parts; + + if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(storage.zookeeper_path) / "exports" / key / "locks", locked_parts)) + { + LOG_INFO(storage.log, "ExportPartition: Failed to get locked parts, skipping"); + continue; + } + + std::unordered_set locked_parts_set(locked_parts.begin(), locked_parts.end()); + + for (const auto & zk_part_name : parts_in_processing_or_pending) + { + if (locked_parts_set.contains(zk_part_name)) + { + LOG_INFO(storage.log, "ExportPartition: Part {} is locked, skipping", zk_part_name); + continue; + } + + const auto part = storage.getPartIfExists(zk_part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + if (!part) + { + LOG_INFO(storage.log, "ExportPartition: Part {} not found locally, skipping", zk_part_name); + continue; + } + + if (Coordination::Error::ZOK != zk->tryCreate(fs::path(storage.zookeeper_path) / "exports" / key / "locks" / zk_part_name, storage.replica_name, zkutil::CreateMode::Ephemeral)) + { + LOG_INFO(storage.log, "ExportPartition: Failed to lock part {}, skipping", zk_part_name); + continue; + } + + try + { + storage.exportPartToTable( + part->name, + destination_storage_id, + manifest.transaction_id, + storage.getContext(), + [this, key, zk_part_name, manifest, destination_storage] + (MergeTreePartExportManifest::CompletionCallbackResult result) + { + handlePartExportCompletion(key, zk_part_name, manifest, destination_storage, result); + }); + } + catch (const Exception &) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + zk->tryRemove(fs::path(storage.zookeeper_path) / "exports" / key / "locks" / zk_part_name); + /// todo arthur re-schedule this so we can try later + /// we should not increment retry_count because the node might just be full + } + } + } +} + +void ExportPartitionTaskScheduler::handlePartExportCompletion( + const std::string & export_key, + const std::string & part_name, + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const MergeTreePartExportManifest::CompletionCallbackResult & result) +{ + const auto export_path = fs::path(storage.zookeeper_path) / "exports" / export_key; + const auto processing_parts_path = export_path / "processing"; + const auto processed_part_path = export_path / "processed" / part_name; + const auto zk = storage.getZooKeeper(); + + if (result.success) + { + handlePartExportSuccess(manifest, destination_storage, processing_parts_path, processed_part_path, part_name, export_path, zk, result.relative_path_in_destination_storage); + } + else + { + handlePartExportFailure(processing_parts_path, processed_part_path, part_name, export_path, zk, result.exception); + } +} + +void ExportPartitionTaskScheduler::handlePartExportSuccess( + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const std::filesystem::path & processing_parts_path, + const std::filesystem::path & processed_part_path, + const std::string & part_name, + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk, + const String & relative_path_in_destination_storage +) +{ + LOG_INFO(storage.log, "ExportPartition: Part {} exported successfully", relative_path_in_destination_storage); + + Coordination::Stat locked_by_stat; + std::string locked_by; + + if (!zk->tryGet(export_path / "locks" / part_name, locked_by, &locked_by_stat)) + { + LOG_INFO(storage.log, "ExportPartition: Part {} is not locked by any replica, will not commit or set it as completed", part_name); + return; + } + + /// Is this a good idea? what if the file we just pushed to s3 ends up triggering an exception in the replica that actually locks the part and it does not commit? + /// I guess we should not throw if file already exists for export partition, hard coded. + if (locked_by != storage.replica_name) + { + LOG_INFO(storage.log, "ExportPartition: Part {} is locked by another replica, will not commit or set it as completed", part_name); + return; + } + + Coordination::Requests requests; + + if (zk->isFeatureEnabled(DB::KeeperFeatureFlag::REMOVE_RECURSIVE)) + { + requests.emplace_back(zkutil::makeRemoveRecursiveRequest(*zk, processing_parts_path / part_name, -1)); + } + else + { + // Remove children before parent (order matters for multi operations) + // Maybe a ls + multi rm.. + requests.emplace_back(zkutil::makeRemoveRequest(processing_parts_path / part_name / "retry_count", -1)); + requests.emplace_back(zkutil::makeRemoveRequest(processing_parts_path / part_name / "max_retry", -1)); + requests.emplace_back(zkutil::makeRemoveRequest(processing_parts_path / part_name / "status", -1)); + requests.emplace_back(zkutil::makeRemoveRequest(processing_parts_path / part_name, -1)); + } + + requests.emplace_back(zkutil::makeCreateRequest(processed_part_path, "", zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeCreateRequest(processed_part_path / "path", relative_path_in_destination_storage, zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeCreateRequest(processed_part_path / "status", "COMPLETED", zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeCreateRequest(processed_part_path / "finished_by", storage.replica_name, zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeRemoveRequest(export_path / "locks" / part_name, locked_by_stat.version)); + + Coordination::Responses responses; + if (Coordination::Error::ZOK != zk->tryMulti(requests, responses)) + { + /// todo arthur remember what to do here + LOG_INFO(storage.log, "ExportPartition: Failed to update export path, skipping"); + return; + } + + LOG_INFO(storage.log, "ExportPartition: Marked part export {} as completed", part_name); + + Strings parts_in_processing_or_pending; + if (Coordination::Error::ZOK != zk->tryGetChildren(export_path / "processing", parts_in_processing_or_pending)) + { + LOG_INFO(storage.log, "ExportPartition: Failed to get parts in processing or pending, will not try to commit export partition"); + return; + } + + if (!parts_in_processing_or_pending.empty()) + { + LOG_INFO(storage.log, "ExportPartition: There are still parts in processing or pending, will not try to commit export partition"); + return; + } + + LOG_INFO(storage.log, "ExportPartition: All parts are processed, will try to commit export partition"); + + ExportPartitionUtils::commit(manifest, destination_storage, zk, storage.log.load(), export_path, storage.getContext()); +} + +void ExportPartitionTaskScheduler::handlePartExportFailure( + const std::filesystem::path & processing_parts_path, + const std::filesystem::path & processed_part_path, + const std::string & part_name, + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk, + const String & exception +) +{ + tryLogCurrentException(__PRETTY_FUNCTION__); + + Coordination::Requests ops; + + const auto processing_part_path = processing_parts_path / part_name; + std::string retry_count_string; + if (zk->tryGet(processing_part_path / "retry_count", retry_count_string)) + { + std::size_t retry_count = std::stoull(retry_count_string.c_str()) + 1; + + + ops.emplace_back(zkutil::makeSetRequest(processing_part_path / "retry_count", std::to_string(retry_count), -1)); + ops.emplace_back(zkutil::makeRemoveRequest(export_path / "locks" / part_name, -1)); + + if (retry_count >= 3) + { + /// remove from processing and create in processed with status FAILED + + ops.emplace_back(zkutil::makeRemoveRequest(processing_parts_path / part_name, -1)); + ops.emplace_back(zkutil::makeCreateRequest(processed_part_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(processed_part_path / "status", "FAILED", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(processed_part_path / "finished_by", storage.replica_name, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeSetRequest(export_path / "status", "FAILED", -1)); + + LOG_INFO(storage.log, "ExportPartition: Marked part export {} as failed", part_name); + } + + std::size_t num_exceptions = 0; + + const auto exceptions_per_replica_path = export_path / "exceptions_per_replica" / storage.replica_name; + const auto count_path = exceptions_per_replica_path / "count"; + const auto last_exception_path = exceptions_per_replica_path / "last_exception"; + + if (zk->exists(exceptions_per_replica_path)) + { + std::string num_exceptions_string; + zk->tryGet(count_path, num_exceptions_string); + num_exceptions = std::stoull(num_exceptions_string.c_str()); + + ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "part", part_name, -1)); + ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "exception", exception, -1)); + } + else + { + ops.emplace_back(zkutil::makeCreateRequest(exceptions_per_replica_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(count_path, "0", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "part", part_name, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "exception", exception, zkutil::CreateMode::Persistent)); + } + + num_exceptions++; + ops.emplace_back(zkutil::makeSetRequest(count_path, std::to_string(num_exceptions), -1)); + + Coordination::Responses responses; + if (Coordination::Error::ZOK != zk->tryMulti(ops, responses)) + { + LOG_INFO(storage.log, "ExportPartition: All failure mechanism failed, will not try to update it"); + return; + } + } +} + +} diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.h b/src/Storages/MergeTree/ExportPartitionTaskScheduler.h new file mode 100644 index 000000000000..9c7a2e00ebb8 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.h @@ -0,0 +1,51 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class StorageReplicatedMergeTree; + +struct ExportReplicatedMergeTreePartitionManifest; + +/// todo arthur remember to add check(lock, version) when updating stuff because maybe if we believe we have the lock, we might not actually have it +class ExportPartitionTaskScheduler +{ +public: + ExportPartitionTaskScheduler(StorageReplicatedMergeTree & storage); + + void run(); +private: + StorageReplicatedMergeTree & storage; + + /// todo arthur maybe it is invalid to grab the manifst here + void handlePartExportCompletion( + const std::string & export_key, + const std::string & part_name, + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const MergeTreePartExportManifest::CompletionCallbackResult & result); + + void handlePartExportSuccess( + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const std::filesystem::path & processing_parts_path, + const std::filesystem::path & processed_part_path, + const std::string & part_name, + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk, + const String & relative_path_in_destination_storage + ); + + void handlePartExportFailure( + const std::filesystem::path & processing_parts_path, + const std::filesystem::path & processed_part_path, + const std::string & part_name, + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk, + const String & exception); +}; + +} diff --git a/src/Storages/MergeTree/ExportPartitionUtils.cpp b/src/Storages/MergeTree/ExportPartitionUtils.cpp new file mode 100644 index 000000000000..6af6ad05e711 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionUtils.cpp @@ -0,0 +1,79 @@ +#include +#include +#include +#include "Storages/ExportReplicatedMergeTreePartitionManifest.h" +#include "Storages/StorageReplicatedMergeTree.h" +#include + +namespace DB +{ + +namespace fs = std::filesystem; + +namespace ExportPartitionUtils +{ + std::vector getExportedPaths(const LoggerPtr & log, const zkutil::ZooKeeperPtr & zk, const std::string & export_path) + { + std::vector exported_paths; + + LOG_INFO(log, "ExportPartition: Getting exported paths for {}", export_path); + + std::vector parts_children; + if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(export_path) / "processed", parts_children)) + { + /// todo arthur do something here + LOG_INFO(log, "ExportPartition: Failed to get parts children, exiting"); + return exported_paths; + } + + for (const auto & part_child : parts_children) + { + std::string path_in_destination_storage; + + if (zk->tryGet(fs::path(export_path) / "parts" / part_child / "path", path_in_destination_storage)) + { + LOG_INFO(log, "ExportPartition: Failed to get path in destination storage for part {}, skipping", part_child); + continue; + } + + exported_paths.push_back(path_in_destination_storage); + } + + return exported_paths; + } + + void commit( + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const zkutil::ZooKeeperPtr & zk, + const LoggerPtr & log, + const std::string & entry_path, + const ContextPtr & context + ) + { + const auto exported_paths = ExportPartitionUtils::getExportedPaths(log, zk, entry_path); + + if (exported_paths.size() == manifest.parts.size()) + { + LOG_INFO(log, "ExportPartition: Exported paths size matches parts size, commit the export"); + destination_storage->commitExportPartitionTransaction(manifest.transaction_id, manifest.partition_id, exported_paths, context); + + LOG_INFO(log, "ExportPartition: Committed export, mark as completed"); + if (Coordination::Error::ZOK == zk->trySet(fs::path(entry_path) / "status", "COMPLETED", -1)) + { + LOG_INFO(log, "ExportPartition: Marked export as completed"); + } + else + { + LOG_INFO(log, "ExportPartition: Failed to mark export as completed, will not try to fix it"); + } + } + else + { + LOG_INFO(log, "ExportPartition: Skipping {}: exported paths size does not match parts size, this is a BUG", entry_path); + } + } + +} + +} diff --git a/src/Storages/MergeTree/ExportPartitionUtils.h b/src/Storages/MergeTree/ExportPartitionUtils.h new file mode 100644 index 000000000000..40fe04a5bfd3 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionUtils.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include +#include +#include +#include "Storages/IStorage.h" + +namespace DB +{ + +struct ExportReplicatedMergeTreePartitionManifest; + +namespace ExportPartitionUtils +{ + std::vector getExportedPaths(const LoggerPtr & log, const zkutil::ZooKeeperPtr & zk, const std::string & export_path); + + void commit( + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const zkutil::ZooKeeperPtr & zk, + const LoggerPtr & log, + const std::string & entry_path, + const ContextPtr & context + ); +} + +} diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 0ad3d72328e6..2bfa7c346242 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -214,6 +214,7 @@ namespace Setting extern const SettingsUInt64 min_bytes_to_use_direct_io; extern const SettingsBool export_merge_tree_part_overwrite_file_if_exists; extern const SettingsBool output_format_parallel_formatting; + extern const SettingsBool output_format_parquet_parallel_encoding; } namespace MergeTreeSetting @@ -6258,6 +6259,7 @@ void MergeTreeData::exportPartToTable( transaction_id, query_context->getSettingsRef()[Setting::export_merge_tree_part_overwrite_file_if_exists], query_context->getSettingsRef()[Setting::output_format_parallel_formatting], + query_context->getSettingsRef()[Setting::output_format_parquet_parallel_encoding], completion_callback); std::lock_guard lock(export_manifests_mutex); @@ -6283,7 +6285,6 @@ void MergeTreeData::killExportPart(const String & query_id) if (manifest.task) manifest.task->cancel(); - getContext()->getExportsList().remove(getStorageID(), manifest.destination_storage_id, manifest.data_part->name); return true; } return false; diff --git a/src/Storages/MergeTree/MergeTreePartExportManifest.h b/src/Storages/MergeTree/MergeTreePartExportManifest.h index dd55b21c8a23..4aa2f1d46ac7 100644 --- a/src/Storages/MergeTree/MergeTreePartExportManifest.h +++ b/src/Storages/MergeTree/MergeTreePartExportManifest.h @@ -41,12 +41,14 @@ struct MergeTreePartExportManifest const String & query_id_, bool overwrite_file_if_exists_, bool parallel_formatting_, + bool parquet_parallel_formatting_, std::function completion_callback_ = {}) : destination_storage_id(destination_storage_id_), data_part(data_part_), query_id(query_id_), overwrite_file_if_exists(overwrite_file_if_exists_), parallel_formatting(parallel_formatting_), + parquet_parallel_formatting(parquet_parallel_formatting_), completion_callback(completion_callback_), create_time(time(nullptr)) {} @@ -56,6 +58,8 @@ struct MergeTreePartExportManifest String query_id; bool overwrite_file_if_exists; bool parallel_formatting; + /// parquet has a different setting for parallel formatting + bool parquet_parallel_formatting; std::function completion_callback; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index aa7cf00b06a3..52d99a0fda46 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -508,7 +508,7 @@ SinkToStoragePtr StorageObjectStorage::import( destination_file_path, object_storage, configuration, - format_settings, + std::nullopt, /// passing nullopt here so we can rebuild it based on the context received in the import function std::make_shared(getInMemoryMetadataPtr()->getSampleBlock()), local_context); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b9c05612dba5..b3d5895021dd 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -470,6 +470,10 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( /// Will be activated by restarting thread. mutations_finalizing_task->deactivate(); + export_merge_tree_partition_manifest_updater = std::make_shared(*this); + + export_merge_tree_partition_task_scheduler = std::make_shared(*this); + export_merge_tree_partition_updating_task = getContext()->getSchedulePool().createTask( getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_updating_task)", [this] { exportMergeTreePartitionUpdatingTask(); }); @@ -4379,621 +4383,32 @@ void StorageReplicatedMergeTree::mutationsFinalizingTask() } void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() -{ +{ try { - std::lock_guard lock(export_merge_tree_partition_mutex); - - auto zk = getZooKeeper(); - - const std::string exports_path = fs::path(zookeeper_path) / "exports"; - const std::string cleanup_lock_path = fs::path(zookeeper_path) / "exports_cleanup_lock"; - - bool cleanup_lock_acquired = zk->tryCreate(cleanup_lock_path, "", zkutil::CreateMode::Ephemeral) == Coordination::Error::ZOK; - - if (cleanup_lock_acquired) - { - LOG_INFO(log, "Cleanup lock acquired, will remove stale entries"); - } - - Coordination::Stat stat; - const auto children = zk->getChildrenWatch(exports_path, &stat, export_merge_tree_partition_watch_callback); - const std::unordered_set zk_children(children.begin(), children.end()); - - const auto now = time(nullptr); - - /// Load new entries - /// If we have the cleanup lock, also remove stale entries from zk and local - for (const auto & key : zk_children) - { - const std::string entry_path = fs::path(exports_path) / key; - - std::string metadata_json; - if (!zk->tryGet(fs::path(entry_path) / "metadata.json", metadata_json)) - { - LOG_INFO(log, "Skipping {}: missing metadata.json", key); - continue; - } - - const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); - - const auto local_entry = export_merge_tree_partition_task_entries.find(key); - - /// If the zk entry has been replaced with export_merge_tree_partition_force_export, checking only for the export key is not enough - /// we need to make sure it is the same transaction id. If it is not, it needs to be replaced. - bool has_local_entry_and_is_up_to_date = local_entry != export_merge_tree_partition_task_entries.end() - && local_entry->second.manifest.transaction_id == metadata.transaction_id; - - /// If the entry is up to date and we don't have the cleanup lock, early exit, nothing to be done. - if (!cleanup_lock_acquired && has_local_entry_and_is_up_to_date) - continue; - - std::string status; - if (!zk->tryGet(fs::path(entry_path) / "status", status)) - { - LOG_INFO(log, "Skipping {}: missing status", key); - continue; - } - - bool is_not_pending = status != "PENDING"; - - if (cleanup_lock_acquired) - { - bool has_expired = metadata.create_time < now - 180; - - if (has_expired && is_not_pending) - { - zk->tryRemoveRecursive(fs::path(entry_path)); - export_merge_tree_partition_task_entries.erase(key); - LOG_INFO(log, "Removed {}: expired", key); - continue; - } - } - - if (is_not_pending) - { - LOG_INFO(log, "Skipping {}: status is not PENDING", key); - continue; - } - - if (has_local_entry_and_is_up_to_date) - { - LOG_INFO(log, "Skipping {}: already exists", key); - continue; - } - - std::string parts_to_do_str; - if (!zk->tryGet(fs::path(entry_path) / "parts_to_do", parts_to_do_str)) - { - LOG_INFO(log, "Skipping {}: no parts_to_do", key); - continue; - } - - uint64_t parts_to_do = 0; - try - { - parts_to_do = std::stoull(parts_to_do_str); - } - catch (...) - { - LOG_WARNING(log, "Skipping {}: invalid parts_to_do='{}'", key, parts_to_do_str); - continue; - } - - if (parts_to_do == 0) - { - LOG_INFO(log, "Skipping {}: parts_to_do is 0", key); - continue; - } - - std::vector part_references; - - for (const auto & part_name : metadata.parts) - { - if (const auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated})) - { - part_references.push_back(part); - } - } - - /// It is important to use the operator[] because it updates the existing entry if it already exists. - export_merge_tree_partition_task_entries[key] = ExportReplicatedMergeTreePartitionTaskEntry {metadata, parts_to_do, std::move(part_references)}; - } - - /// Remove entries that were deleted by someone else - std::erase_if(export_merge_tree_partition_task_entries, - [&](auto const & kv) - { - if (zk_children.contains(kv.first)) - { - return false; - } - - const auto & transaction_id = kv.second.manifest.transaction_id; - LOG_INFO(log, "Export task {} was deleted, calling killExportPartition for transaction {}", kv.first, transaction_id); - - try - { - killExportPart(transaction_id); - } - catch (...) - { - tryLogCurrentException(log, __PRETTY_FUNCTION__); - } - - return true; - }); - - if (cleanup_lock_acquired) - { - zk->tryRemove(cleanup_lock_path); - } - - export_merge_tree_partition_select_task->schedule(); + export_merge_tree_partition_manifest_updater->run(); } catch (...) { tryLogCurrentException(log, __PRETTY_FUNCTION__); } - + export_merge_tree_partition_updating_task->scheduleAfter(30 * 1000); } void StorageReplicatedMergeTree::selectPartsToExport() { - const auto exports_path = fs::path(zookeeper_path) / "exports"; - - auto complete_part_export = [&]( - const std::string & export_partition_path, - const std::string & part_path, - const std::string & part_status_path, - const std::string & parts_to_do_path, - const std::string & lock_path, - const std::string & path_in_destination_storage_path, - const StoragePtr & destination_storage, - const std::string & transaction_id, - const std::string & partition_id, - const ZooKeeperPtr & zk) -> bool - { - /// todo arthur is it possible to grab stats using a multi-op? - Coordination::Stat parts_to_do_stat; - Coordination::Stat lock_stat; - std::string parts_to_do_string; - - int retries = 0; - const int max_retries = 3; - while (retries < max_retries) - { - if (!zk->tryGet(parts_to_do_path, parts_to_do_string, &parts_to_do_stat)) - { - LOG_INFO(log, "Failed to get parts_to_do, skipping"); - return false; - } - - std::string locked_by; - - if (!zk->tryGet(lock_path, locked_by, &lock_stat)) - { - LOG_INFO(log, "Failed to get locked_by, skipping"); - return false; - } - - if (locked_by != replica_name) - { - LOG_INFO(log, "Skipping... Locked by {}, not by {}", locked_by, replica_name); - return false; - } - - std::size_t parts_to_do = std::stoull(parts_to_do_string.c_str()); - - if (parts_to_do == 0) - { - LOG_INFO(log, "Skipping... Parts to do is 0, maybe someone else already completed it? that sounds weird"); - return false; - } - - parts_to_do--; - - Coordination::Requests ops; - ops.emplace_back(zkutil::makeCheckRequest(lock_path, lock_stat.version)); - - /* - todo arthur: - Taking a note here so we can eventually discuss: - - Marking an individual part export as completed should ideally not depend on race conditions. Right now it depends. For instance: - - replica1 finished part1, and it is about to mark it as completed. It queries parts_to_do and creates the request that updates parts_to_do and the status. Before it sends that query, replica2 does the same and is able to update parts_to_do and its part2 status. - - At this point, replica1 will faill the entire request, which includes status=completed. There are a few ways around it: - - Upon failure, release the lock and let someoe else (or the same replica in the future) retry that part even tho it had already succeeded. - Implement retry logic that detects if the failure was because of the parts_to_do version and keeps retrying for a while until it eventually release the lock - */ - ops.emplace_back(zkutil::makeCheckRequest(parts_to_do_path, parts_to_do_stat.version)); - ops.emplace_back(zkutil::makeSetRequest(part_status_path, "COMPLETED", -1)); - ops.emplace_back(zkutil::makeRemoveRequest(lock_path, lock_stat.version)); - ops.emplace_back(zkutil::makeSetRequest(parts_to_do_path, std::to_string(parts_to_do), parts_to_do_stat.version)); - ops.emplace_back(zkutil::makeCreateRequest(fs::path(part_path) / "finished_by", replica_name, zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(fs::path(part_path) / "path_in_destination_storage", path_in_destination_storage_path, zkutil::CreateMode::Persistent)); - - if (parts_to_do == 0) - { - /// loop over all parts under `/parts` and grab all paths in destination storage - Strings exported_paths; - const auto parts_path = fs::path(export_partition_path) / "parts"; - Strings parts = zk->getChildren(parts_path); - - for (const auto & part : parts) - { - std::string path_in_destination_storage; - const auto path_in_destination_storage_zk_path = fs::path(parts_path) / part / "path_in_destination_storage"; - - if (zk->tryGet(path_in_destination_storage_zk_path, path_in_destination_storage)) - { - exported_paths.push_back(path_in_destination_storage); - } - else - { - /// todo arthur what should I do here? - LOG_WARNING(log, "Failed to get path_in_destination_storage for part {} in export", part); - } - } - - LOG_INFO(log, "Collected {} exported paths for export", exported_paths.size()); - - /// manually add the export we just finished because it is not zk yet - exported_paths.push_back(path_in_destination_storage_path); - - destination_storage->commitExportPartitionTransaction(transaction_id, partition_id, exported_paths, getContext()); - ops.emplace_back(zkutil::makeSetRequest(fs::path(export_partition_path) / "status", "COMPLETED", -1)); - } - - Coordination::Responses responses; - if (zk->tryMulti(ops, responses) == Coordination::Error::ZOK) - { - return true; - } - - retries++; - } - - return false; - }; - - auto lock_part = [&]( - const std::string & part_path, - const std::string & status_path, - const std::string & lock_path, - const std::string & node_name, - const ZooKeeperPtr & zk) -> bool - { - Coordination::Requests ops; - - /// if the part path exists, it can be one of the following: - /// 1. PENDING and Locked - Some replica is working on it - we should skip - /// 2. PENDING and unlocked - Whoever was working on it died - we should acquire the lock - /// 3. COMPLETED and unlocked - We should skip - if (zk->exists(part_path)) - { - Coordination::Stat stat; - std::string status; - - if (zk->tryGet(status_path, status, &stat)) - { - if (status != "PENDING") - { - LOG_INFO(log, "Skipping... Status is not PENDING"); - return false; - } - - /// only try to lock it if the status is still PENDING - /// if we did not check for status = pending, chances are some other replica completed and released the lock in the meantime - ops.emplace_back(zkutil::makeCheckRequest(status_path, stat.version)); - /// todo do I need to "re-set" the status to PENDING? I don't think so. - ops.emplace_back(zkutil::makeCreateRequest(lock_path, node_name, zkutil::CreateMode::Ephemeral)); - /// no need to update retry count here. - } - else - { - LOG_INFO(log, "Skipping... Failed to get status, probably killed"); - return false; - } - } - else - { - /// if the node does not exist, just create everything from scratch. - ops.emplace_back(zkutil::makeCreateRequest(part_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(status_path, "PENDING", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(lock_path, node_name, zkutil::CreateMode::Ephemeral)); - ops.emplace_back(zkutil::makeCreateRequest(fs::path(part_path) / "retry_count", "0", zkutil::CreateMode::Persistent)); - } - - Coordination::Responses responses; - - return zk->tryMulti(ops, responses) == Coordination::Error::ZOK; - }; - - auto try_to_acquire_a_part = [&]( - const ExportReplicatedMergeTreePartitionManifest & manifest, - const std::string & partition_export_path, - std::size_t & next_idx, - const ZooKeeperPtr & zk) -> std::optional - { - const auto try_to_update_next_idx = [&](std::size_t local_next_idx) - { - /// Update next_idx - best effort, if it fails, that is ok - Coordination::Stat next_idx_stat; - std::string next_idx_string; - const auto next_idx_path = fs::path(partition_export_path) / "next_idx"; - - if (zk->tryGet(next_idx_path, next_idx_string, &next_idx_stat)) - { - const std::size_t next_idx_zk = std::stoul(next_idx_string.c_str()); - const std::size_t new_next_idx = std::max(next_idx_zk, local_next_idx + 1); - Coordination::Requests ops; - ops.emplace_back(zkutil::makeCheckRequest(next_idx_path, next_idx_stat.version)); - ops.emplace_back(zkutil::makeSetRequest(next_idx_path, std::to_string(new_next_idx), next_idx_stat.version)); - Coordination::Responses responses; - if (zk->tryMulti(ops, responses) != Coordination::Error::ZOK) - { - LOG_INFO(log, "Updated next_idx to {}", new_next_idx); - } - } - }; - - for (auto i = next_idx; i < manifest.number_of_parts; i++) - { - const auto part_path = fs::path(partition_export_path) / "parts" / manifest.parts[i]; - const auto lock_path = fs::path(part_path) / "lock"; - const auto status_path = fs::path(part_path) / "status"; - - const auto part = getPartIfExists(manifest.parts[i], {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); - if (!part) - { - LOG_INFO(log, "Skipping... Part {} not found locally", manifest.parts[i]); - continue; - } - - if (lock_part(part_path, status_path, lock_path, replica_name, zk)) - { - next_idx = i; - try_to_update_next_idx(i); - return manifest.parts[i]; - } - } - - /// failed to lock a part in the range of `next_idx...number_of_parts` - /// now try the full scan `0...next_idx` - for (auto i = 0u; i < next_idx; i++) - { - const auto part_path = fs::path(partition_export_path) / "parts" / manifest.parts[i]; - const auto lock_path = fs::path(part_path) / "lock"; - const auto status_path = fs::path(part_path) / "status"; - - const auto part = getPartIfExists(manifest.parts[i], {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); - if (!part) - { - LOG_INFO(log, "Skipping... Part {} not found locally", manifest.parts[i]); - continue; - } - - if (lock_part(part_path, status_path, lock_path, replica_name, zk)) - { - next_idx = i; - try_to_update_next_idx(i); - return manifest.parts[i]; - } - } - - return std::nullopt; - }; - try { - const auto zk = getZooKeeper(); - - std::lock_guard lock(export_merge_tree_partition_mutex); - - for (auto & [key, task_entry] : export_merge_tree_partition_task_entries) - { - /// this sounds impossible, but just in case - if (task_entry.parts_to_do == 0) - { - LOG_INFO(log, "Already completed, skipping"); - continue; - } - - std::string parts_to_do_string; - if (!zk->tryGet(fs::path(exports_path) / key / "parts_to_do", parts_to_do_string)) - { - LOG_INFO(log, "Failed to get parts_to_do, skipping"); - continue; - } - - const auto parts_to_do = std::stoull(parts_to_do_string.c_str()); - task_entry.parts_to_do = parts_to_do; - - if (task_entry.parts_to_do == 0) - { - LOG_INFO(log, "Already completed, skipping"); - continue; - } - - std::string status; - if (!zk->tryGet(fs::path(exports_path) / key / "status", status)) - { - LOG_INFO(log, "Failed to get status, skipping"); - continue; - } - - if (status != "PENDING") - { - LOG_INFO(log, "Skipping... Status is not PENDING"); - continue; - } - - const auto & manifest = task_entry.manifest; - const auto & database = getContext()->resolveDatabase(manifest.destination_database); - const auto & table = manifest.destination_table; - - const auto destination_storage_id = StorageID(QualifiedTableName {database, table}); - - const auto destination_storage = DatabaseCatalog::instance().tryGetTable(destination_storage_id, getContext()); - - if (!destination_storage) - { - LOG_INFO(log, "Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); - continue; - } - - const auto partition_path = fs::path(exports_path) / key; - const auto next_idx_path = fs::path(partition_path) / "next_idx"; - std::string next_idx_string; - if (!zk->tryGet(next_idx_path, next_idx_string)) - { - LOG_INFO(log, "Failed to get next_idx, skipping"); - continue; - } - - std::size_t next_idx = std::stoull(next_idx_string.c_str()); - while (const auto part_to_export = try_to_acquire_a_part(manifest, partition_path, next_idx, zk)) - { - const auto partition_id = manifest.partition_id; - const auto transaction_id = manifest.transaction_id; - - try - { - /// todo arthur temporary for hackathon - auto context_copy = Context::createCopy(getContext()); - context_copy->setSetting("export_merge_tree_part_overwrite_file_if_exists", true); - exportPartToTable( - part_to_export.value(), - destination_storage_id, - transaction_id, - context_copy, - [this, partition_id, transaction_id, exports_path, key, part_to_export, complete_part_export, partition_path, destination_storage] - (MergeTreePartExportManifest::CompletionCallbackResult result) - { - const auto zk_client = getZooKeeper(); - if (result.success) - { - complete_part_export( - partition_path, - fs::path(partition_path) / "parts" / part_to_export.value(), - fs::path(partition_path) / "parts" / part_to_export.value() / "status", - fs::path(partition_path) / "parts_to_do", - fs::path(partition_path) / "parts" / part_to_export.value() / "lock", - result.relative_path_in_destination_storage, - destination_storage, - transaction_id, - partition_id, - zk_client); - - /// maybe get up to date from complete_parts_export? - std::lock_guard inner_lock(export_merge_tree_partition_mutex); - export_merge_tree_partition_task_entries[key].parts_to_do--; - - if (export_merge_tree_partition_task_entries[key].parts_to_do == 0) - { - export_merge_tree_partition_task_entries.erase(key); - } - } - else - { - /// increment retry_count - /// if above threshhold, fail the entire export - hopefully it is safe to do so :D - /// I could also leave this for the cleanup thread, but will do it here for now. - - Coordination::Requests ops; - - std::string retry_count_string; - if (zk_client->tryGet(fs::path(partition_path) / "parts" / part_to_export.value() / "retry_count", retry_count_string)) - { - std::size_t retry_count = std::stoull(retry_count_string.c_str()) + 1; - - //// todo arthur unhardcode this - if (retry_count >= 3) - { - /// instead of removing the entire partition, just mark the status as failed - ops.emplace_back(zkutil::makeSetRequest(fs::path(partition_path) / "status", "FAILED", -1)); - /// mark the part as failed as well - ops.emplace_back(zkutil::makeSetRequest(fs::path(partition_path) / "parts" / part_to_export.value() / "status", "FAILED", -1)); - } - else - { - ops.emplace_back(zkutil::makeSetRequest(fs::path(partition_path) / "parts" / part_to_export.value() / "retry_count", std::to_string(retry_count), -1)); - } - - ops.emplace_back(zkutil::makeRemoveRequest(fs::path(partition_path) / "parts" / part_to_export.value() / "lock", -1)); - } - else - { - LOG_INFO(log, "Failed to get retry_count, will not try to update it"); - ops.emplace_back(zkutil::makeRemoveRequest(fs::path(partition_path) / "parts" / part_to_export.value() / "lock", -1)); - } - - std::size_t num_exceptions = 0; - - const auto exceptions_per_replica_path = fs::path(partition_path) / "exceptions_per_replica" / replica_name; - const auto count_path = exceptions_per_replica_path / "count"; - const auto last_exception_path = exceptions_per_replica_path / "last_exception"; - - if (zk_client->exists(exceptions_per_replica_path)) - { - std::string num_exceptions_string; - zk_client->tryGet(count_path, num_exceptions_string); - num_exceptions = std::stoull(num_exceptions_string.c_str()); - - ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "part", part_to_export.value(), -1)); - ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "exception", result.exception, -1)); - } - else - { - ops.emplace_back(zkutil::makeCreateRequest(exceptions_per_replica_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(count_path, "0", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(last_exception_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "part", part_to_export.value(), zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "exception", result.exception, zkutil::CreateMode::Persistent)); - } - - num_exceptions++; - ops.emplace_back(zkutil::makeSetRequest(count_path, std::to_string(num_exceptions), -1)); - - Coordination::Responses responses; - if (zk_client->tryMulti(ops, responses) != Coordination::Error::ZOK) - { - LOG_INFO(log, "All failure mechanism failed, will not try to update it"); - return; - } - - } - }); - } - catch (...) - { - /// failed to schedule the part export - tryLogCurrentException(log, __PRETTY_FUNCTION__); - - /// best-effort to remove the lock (actually, we should make sure the lock is released..) - zk->tryRemove(fs::path(partition_path) / "parts" / part_to_export.value() / "lock"); - - /// todo arthur should I try to rewind the next_idx? - - /// re-run after some time - export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); - - break; - } - } - } + export_merge_tree_partition_task_scheduler->run(); } catch (...) { tryLogCurrentException(log, __PRETTY_FUNCTION__); - export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); } + + export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); } std::vector StorageReplicatedMergeTree::getPartitionExportsInfo() const @@ -5016,21 +4431,21 @@ std::vector StorageReplicatedMergeTree::getPartit continue; } - std::string parts_to_do_string; - if (!zk->tryGet(export_partition_path / "parts_to_do", parts_to_do_string)) + std::string status; + if (!zk->tryGet(export_partition_path / "status", status)) { - LOG_INFO(log, "Skipping {}: missing parts_to_do", child); + LOG_INFO(log, "Skipping {}: missing status", child); continue; } - std::string status; - if (!zk->tryGet(export_partition_path / "status", status)) + std::vector processing_parts; + if (Coordination::Error::ZOK != zk->tryGetChildren(export_partition_path / "processing", processing_parts)) { - LOG_INFO(log, "Skipping {}: missing status", child); + LOG_INFO(log, "Skipping {}: missing processing parts", child); continue; } - const auto parts_to_do = std::stoull(parts_to_do_string.c_str()); + const auto parts_to_do = processing_parts.size(); std::string exception_replica; std::string last_exception; @@ -5048,6 +4463,7 @@ std::vector StorageReplicatedMergeTree::getPartit LOG_INFO(log, "Skipping {}: missing count", replica); continue; } + exception_count += std::stoull(exception_count_string.c_str()); if (last_exception.empty()) @@ -8663,32 +8079,6 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & const auto exports_path = fs::path(zookeeper_path) / "exports"; Coordination::Requests ops; - /* - exports - partition_id + target storage id: (or may be hash is safer?) - metadata (znode) <- immutable, every replica read it once to get full meta, znode mtime - is a timestamp - parition id - destination id - source replica - number of parts: 100 - list of parts: <- processed in strict order - 2020_0_0_1 - 2020_1_1_1 - ... - parts_to_do: 100 (znode) - exceptions_per_replica (znode) - replica1: - num_exceptions: 1 - last_exception (znode, znode mtime - is a timestamp of last exception) - part: - exception - parts/ - part_name/ <-- the value of that znode is pending initially, and finished later. - lock = ephemeral, when processing - replica: r1 - start_time: 123445 - */ - const auto export_key = partition_id + "_" + dest_storage_id.getNameForLogs(); const auto partition_exports_path = fs::path(exports_path) / export_key; @@ -8741,22 +8131,45 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest( - fs::path(partition_exports_path) / "parts_to_do", - std::to_string(part_names.size()), + fs::path(partition_exports_path) / "exceptions_per_replica", + "", zkutil::CreateMode::Persistent)); - + ops.emplace_back(zkutil::makeCreateRequest( - fs::path(partition_exports_path) / "next_idx", - "0", + fs::path(partition_exports_path) / "processing", + "", zkutil::CreateMode::Persistent)); + for (const auto & part : part_names) + { + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "processing" / part, + "", + zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "processing" / part / "max_retry", + "3", + zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "processing" / part / "status", + "PENDING", + zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "processing" / part / "retry_count", + "0", + zkutil::CreateMode::Persistent)); + } + ops.emplace_back(zkutil::makeCreateRequest( - fs::path(partition_exports_path) / "exceptions_per_replica", + fs::path(partition_exports_path) / "processed", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest( - fs::path(partition_exports_path) / "parts", + fs::path(partition_exports_path) / "locks", "", zkutil::CreateMode::Persistent)); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 3e7af55023d9..7cc5f9658eed 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -14,6 +14,8 @@ #include #include #include "Interpreters/CancellationCode.h" +#include "Storages/MergeTree/ExportPartitionManifestUpdatingTask.h" +#include "Storages/MergeTree/ExportPartitionTaskScheduler.h" #include #include #include @@ -402,6 +404,8 @@ class StorageReplicatedMergeTree final : public MergeTreeData friend class MergeFromLogEntryTask; friend class MutateFromLogEntryTask; friend class ReplicatedMergeMutateTaskBase; + friend class ExportPartitionManifestUpdatingTask; + friend class ExportPartitionTaskScheduler; using MergeStrategyPicker = ReplicatedMergeTreeMergeStrategyPicker; using LogEntry = ReplicatedMergeTreeLogEntry; @@ -517,6 +521,9 @@ class StorageReplicatedMergeTree final : public MergeTreeData BackgroundSchedulePoolTaskHolder mutations_finalizing_task; BackgroundSchedulePoolTaskHolder export_merge_tree_partition_updating_task; + std::shared_ptr export_merge_tree_partition_manifest_updater; + + std::shared_ptr export_merge_tree_partition_task_scheduler; Coordination::WatchCallbackPtr export_merge_tree_partition_watch_callback; diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/__init__.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/named_collections.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/named_collections.xml new file mode 100644 index 000000000000..d46920b7ba88 --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/named_collections.xml @@ -0,0 +1,9 @@ + + + + http://minio1:9001/root/data + minio + ClickHouse_Minio_P@ssw0rd + + + \ No newline at end of file diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py new file mode 100644 index 000000000000..9f3e39bf9699 --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -0,0 +1,170 @@ +import logging +import pytest +import random +import string +import time +from typing import Optional + +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "replica1", + main_configs=["configs/named_collections.xml"], + user_configs=[], + with_minio=True, + stay_alive=True, + with_zookeeper=True, + ) + cluster.add_instance( + "replica2", + main_configs=["configs/named_collections.xml"], + user_configs=[], + with_minio=True, + stay_alive=True, + with_zookeeper=True, + ) + # node that does not participate in the export, but will have visibility over the s3 table + cluster.add_instance( + "watcher_node", + main_configs=["configs/named_collections.xml"], + user_configs=[], + with_minio=True, + ) + logging.info("Starting cluster...") + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def create_s3_table(node, s3_table): + node.query(f"CREATE TABLE {s3_table} (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='{s3_table}', format=Parquet, partition_strategy='hive') PARTITION BY year") + + +def create_tables_and_insert_data(node, mt_table, s3_table, replica_name): + node.query(f"CREATE TABLE {mt_table} (id UInt64, year UInt16) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{mt_table}', '{replica_name}') PARTITION BY year ORDER BY tuple()") + node.query(f"INSERT INTO {mt_table} VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)") + + create_s3_table(node, s3_table) + + +def test_restart_nodes_during_export(cluster): + node = cluster.instances["replica1"] + node2 = cluster.instances["replica2"] + watcher_node = cluster.instances["watcher_node"] + + mt_table = "disaster_mt_table" + s3_table = "disaster_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") + create_s3_table(watcher_node, s3_table) + + # Add network delays so we can kill the node during the export + with PartitionManager() as pm: + pm.add_network_delay(node, delay_ms=1000) + + export_queries = f""" + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2020' TO TABLE {s3_table} + SETTINGS allow_experimental_export_merge_tree_part=1; + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2021' TO TABLE {s3_table} + SETTINGS allow_experimental_export_merge_tree_part=1; + """ + + node.query(export_queries) + + node.stop_clickhouse(kill=True) + node2.stop_clickhouse(kill=True) + + assert watcher_node.query(f"SELECT count() FROM {s3_table} where year = 2020") == '0\n', "Partition 2020 was written to S3 during network delay crash" + + assert watcher_node.query(f"SELECT count() FROM {s3_table} where year = 2021") == '0\n', "Partition 2021 was written to S3 during network delay crash" + + # start the nodes, they should finish the export + node.start_clickhouse() + node2.start_clickhouse() + + time.sleep(5) + + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") != f'0\n', "Export of partition 2020 did not resume after crash" + + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2021") != f'0\n', "Export of partition 2021 did not resume after crash" + + +def test_kill_export(cluster): + node = cluster.instances["replica1"] + node2 = cluster.instances["replica2"] + watcher_node = cluster.instances["watcher_node"] + + mt_table = "kill_export_mt_table" + s3_table = "kill_export_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") + + with PartitionManager() as pm: + pm.add_network_delay(node, delay_ms=1000) + + export_queries = f""" + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2020' TO TABLE {s3_table} + SETTINGS allow_experimental_export_merge_tree_part=1; + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2021' TO TABLE {s3_table} + SETTINGS allow_experimental_export_merge_tree_part=1; + """ + + node.query(export_queries) + + # kill only 2020, 2021 should still finish + node.query(f"KILL EXPORT PARTITION WHERE partition_id = '2020'") + + # wait for 2021 to finish + time.sleep(5) + + # checking for the commit file because maybe the data file was too fast? + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '0\n', "Partition 2020 was written to S3, it was not killed as expected" + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2021_*', format=LineAsString)") != f'0\n', "Partition 2021 was not written to S3, but it should have been" + + +def test_drop_table_during_export(cluster): + node = cluster.instances["replica1"] + # node2 = cluster.instances["replica2"] + watcher_node = cluster.instances["watcher_node"] + + mt_table = "drop_table_during_export_mt_table" + s3_table = "drop_table_during_export_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + # create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") + create_s3_table(watcher_node, s3_table) + + with PartitionManager() as pm: + pm.add_network_delay(node, delay_ms=1000) + + export_queries = f""" + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2020' TO TABLE {s3_table} + SETTINGS allow_experimental_export_merge_tree_part=1; + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2021' TO TABLE {s3_table} + SETTINGS allow_experimental_export_merge_tree_part=1; + """ + + node.query(export_queries) + + # I think this will actually wait until background operations are finished + node.query(f"DROP TABLE {mt_table} SYNC") + # this will not wait, but the pointer the background task holds is still valid, so the write will finish + node.query(f"DROP TABLE {s3_table}") + + time.sleep(5) + + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_*', format=LineAsString)") != '0\n', "Background operations finished even after the tables were dropped" From 44874313b798028249edc150fb17279e4763119e Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 5 Nov 2025 00:28:26 +0100 Subject: [PATCH 27/58] looking good before ordering --- src/Core/Settings.cpp | 3 + ...portReplicatedMergeTreePartitionManifest.h | 4 +- .../ExportPartitionTaskScheduler.cpp | 43 +-- .../MergeTree/ExportPartitionTaskScheduler.h | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 7 +- .../configs/users.d/profile.xml | 8 + .../test.py | 248 +++++++++++++++++- 7 files changed, 291 insertions(+), 26 deletions(-) create mode 100644 tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/users.d/profile.xml diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 9b7987e497eb..2620b15a7532 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -6876,6 +6876,9 @@ Overwrite file if it already exists when exporting a merge tree part )", 0) \ DECLARE(Bool, export_merge_tree_partition_force_export, false, R"( Ignore existing partition export and overwrite the zookeeper entry +)", 0) \ + DECLARE(UInt64, export_merge_tree_partition_max_retries, 3, R"( +Maximum number of retries for exporting a merge tree part in an export partition task )", 0) \ \ /* ####################################################### */ \ diff --git a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h index 11bdb45e5bc3..1c8556e570f4 100644 --- a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h +++ b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h @@ -19,6 +19,7 @@ struct ExportReplicatedMergeTreePartitionManifest size_t number_of_parts; std::vector parts; time_t create_time; + size_t max_retries; std::string toJsonString() const { @@ -36,6 +37,7 @@ struct ExportReplicatedMergeTreePartitionManifest json.set("parts", parts_array); json.set("create_time", create_time); + json.set("max_retries", max_retries); std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); Poco::JSON::Stringifier::stringify(json, oss); @@ -55,7 +57,7 @@ struct ExportReplicatedMergeTreePartitionManifest manifest.destination_table = json->getValue("destination_table"); manifest.source_replica = json->getValue("source_replica"); manifest.number_of_parts = json->getValue("number_of_parts"); - + manifest.max_retries = json->getValue("max_retries"); auto parts_array = json->getArray("parts"); for (size_t i = 0; i < parts_array->size(); ++i) manifest.parts.push_back(parts_array->getElement(static_cast(i))); diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp index 9e7c2b1084e7..84db08a4c3aa 100644 --- a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -22,7 +22,6 @@ void ExportPartitionTaskScheduler::run() for (const auto & [key, entry] : storage.export_merge_tree_partition_task_entries) { - const auto & manifest = entry.manifest; const auto & database = storage.getContext()->resolveDatabase(manifest.destination_database); const auto & table = manifest.destination_table; @@ -112,11 +111,13 @@ void ExportPartitionTaskScheduler::run() { tryLogCurrentException(__PRETTY_FUNCTION__); zk->tryRemove(fs::path(storage.zookeeper_path) / "exports" / key / "locks" / zk_part_name); - /// todo arthur re-schedule this so we can try later /// we should not increment retry_count because the node might just be full } } } + + /// maybe we failed to schedule or failed to export, need to retry eventually + storage.export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); } void ExportPartitionTaskScheduler::handlePartExportCompletion( @@ -137,7 +138,7 @@ void ExportPartitionTaskScheduler::handlePartExportCompletion( } else { - handlePartExportFailure(processing_parts_path, processed_part_path, part_name, export_path, zk, result.exception); + handlePartExportFailure(processing_parts_path, part_name, export_path, zk, result.exception, manifest.max_retries); } } @@ -182,7 +183,6 @@ void ExportPartitionTaskScheduler::handlePartExportSuccess( // Remove children before parent (order matters for multi operations) // Maybe a ls + multi rm.. requests.emplace_back(zkutil::makeRemoveRequest(processing_parts_path / part_name / "retry_count", -1)); - requests.emplace_back(zkutil::makeRemoveRequest(processing_parts_path / part_name / "max_retry", -1)); requests.emplace_back(zkutil::makeRemoveRequest(processing_parts_path / part_name / "status", -1)); requests.emplace_back(zkutil::makeRemoveRequest(processing_parts_path / part_name, -1)); } @@ -223,15 +223,30 @@ void ExportPartitionTaskScheduler::handlePartExportSuccess( void ExportPartitionTaskScheduler::handlePartExportFailure( const std::filesystem::path & processing_parts_path, - const std::filesystem::path & processed_part_path, const std::string & part_name, const std::filesystem::path & export_path, const zkutil::ZooKeeperPtr & zk, - const String & exception + const String & exception, + size_t max_retries ) { tryLogCurrentException(__PRETTY_FUNCTION__); + Coordination::Stat locked_by_stat; + std::string locked_by; + + if (!zk->tryGet(export_path / "locks" / part_name, locked_by, &locked_by_stat)) + { + LOG_INFO(storage.log, "ExportPartition: Part {} is not locked by any replica, will not increment error counts", part_name); + return; + } + + if (locked_by != storage.replica_name) + { + LOG_INFO(storage.log, "ExportPartition: Part {} is locked by another replica, will not increment error counts", part_name); + return; + } + Coordination::Requests ops; const auto processing_part_path = processing_parts_path / part_name; @@ -240,21 +255,17 @@ void ExportPartitionTaskScheduler::handlePartExportFailure( { std::size_t retry_count = std::stoull(retry_count_string.c_str()) + 1; - ops.emplace_back(zkutil::makeSetRequest(processing_part_path / "retry_count", std::to_string(retry_count), -1)); - ops.emplace_back(zkutil::makeRemoveRequest(export_path / "locks" / part_name, -1)); + ops.emplace_back(zkutil::makeRemoveRequest(export_path / "locks" / part_name, locked_by_stat.version)); - if (retry_count >= 3) + if (retry_count >= max_retries) { - /// remove from processing and create in processed with status FAILED - - ops.emplace_back(zkutil::makeRemoveRequest(processing_parts_path / part_name, -1)); - ops.emplace_back(zkutil::makeCreateRequest(processed_part_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(processed_part_path / "status", "FAILED", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(processed_part_path / "finished_by", storage.replica_name, zkutil::CreateMode::Persistent)); + /// just set status in processing_part_path and finished_by + ops.emplace_back(zkutil::makeSetRequest(processing_part_path / "status", "FAILED", -1)); + ops.emplace_back(zkutil::makeCreateRequest(processing_part_path / "finished_by", storage.replica_name, zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeSetRequest(export_path / "status", "FAILED", -1)); - LOG_INFO(storage.log, "ExportPartition: Marked part export {} as failed", part_name); + LOG_INFO(storage.log, "ExportPartition: Retry count limit exceeded for part {}, will try to fail the entire task", part_name); } std::size_t num_exceptions = 0; diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.h b/src/Storages/MergeTree/ExportPartitionTaskScheduler.h index 9c7a2e00ebb8..ee5d03dd5bed 100644 --- a/src/Storages/MergeTree/ExportPartitionTaskScheduler.h +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.h @@ -41,11 +41,11 @@ class ExportPartitionTaskScheduler void handlePartExportFailure( const std::filesystem::path & processing_parts_path, - const std::filesystem::path & processed_part_path, const std::string & part_name, const std::filesystem::path & export_path, const zkutil::ZooKeeperPtr & zk, - const String & exception); + const String & exception, + size_t max_retries); }; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b3d5895021dd..8a041af3de83 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -193,6 +193,7 @@ namespace Setting extern const SettingsBool update_sequential_consistency; extern const SettingsBool allow_experimental_export_merge_tree_part; extern const SettingsBool export_merge_tree_partition_force_export; + extern const SettingsUInt64 export_merge_tree_partition_max_retries; } namespace MergeTreeSetting @@ -8124,6 +8125,7 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & manifest.number_of_parts = part_names.size(); manifest.parts = part_names; manifest.create_time = time(nullptr); + manifest.max_retries = query_context->getSettingsRef()[Setting::export_merge_tree_partition_max_retries]; ops.emplace_back(zkutil::makeCreateRequest( fs::path(partition_exports_path) / "metadata.json", @@ -8147,11 +8149,6 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest( - fs::path(partition_exports_path) / "processing" / part / "max_retry", - "3", - zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest( fs::path(partition_exports_path) / "processing" / part / "status", "PENDING", diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/users.d/profile.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/users.d/profile.xml new file mode 100644 index 000000000000..518f29708929 --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/users.d/profile.xml @@ -0,0 +1,8 @@ + + + + 3 + + + + diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index 9f3e39bf9699..3ef25629e40f 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -15,7 +15,7 @@ def cluster(): cluster.add_instance( "replica1", main_configs=["configs/named_collections.xml"], - user_configs=[], + user_configs=["configs/users.d/profile.xml"], with_minio=True, stay_alive=True, with_zookeeper=True, @@ -23,7 +23,7 @@ def cluster(): cluster.add_instance( "replica2", main_configs=["configs/named_collections.xml"], - user_configs=[], + user_configs=["configs/users.d/profile.xml"], with_minio=True, stay_alive=True, with_zookeeper=True, @@ -168,3 +168,247 @@ def test_drop_table_during_export(cluster): time.sleep(5) assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_*', format=LineAsString)") != '0\n', "Background operations finished even after the tables were dropped" + + +def test_kill_export_by_table(cluster): + node = cluster.instances["replica1"] + + mt_table = "kill_granularity_by_table_mt" + s3_table = "kill_granularity_by_table_s3" + alt_mt_table = "kill_granularity_by_table_alt_mt" + alt_s3_table = "kill_granularity_by_table_alt_s3" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + create_tables_and_insert_data(node, alt_mt_table, alt_s3_table, "replica1") + + # Slow down network so we can issue KILL mid-flight + with PartitionManager() as pm: + pm.add_network_delay(node, delay_ms=3000) + + # Start two exports for the same table and one export for another table concurrently + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1;" + ) + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2021' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1;" + ) + node.query( + f"ALTER TABLE {alt_mt_table} EXPORT PARTITION ID '2020' TO TABLE {alt_s3_table} SETTINGS allow_experimental_export_merge_tree_part=1;" + ) + + # Kill all exports for the first table only + node.query(f"KILL EXPORT PARTITION WHERE source_table = '{mt_table}'") + + # Give some time for effects to propagate + time.sleep(5) + + # The killed table should have no commit for either partition + assert ( + node.query( + f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_*', format=LineAsString)" + ) + == '0\n' + ), "Partition 2020 was written to S3, but KILL by table should have stopped it" + + # The alternate table (not killed) should complete + assert ( + node.query( + f"SELECT count() FROM s3(s3_conn, filename='{alt_s3_table}/commit_*', format=LineAsString)" + ) + != '0\n' + ), "Alternate table export was affected by KILL on a different table" + + +def test_concurrent_exports_to_different_targets(cluster): + node = cluster.instances["replica1"] + + mt_table = "concurrent_diff_targets_mt_table" + s3_table_a = "concurrent_diff_targets_s3_a" + s3_table_b = "concurrent_diff_targets_s3_b" + + create_tables_and_insert_data(node, mt_table, s3_table_a, "replica1") + create_s3_table(node, s3_table_b) + + # Launch two exports of the same partition to two different S3 tables concurrently + with PartitionManager() as pm: + pm.add_network_delay(node, delay_ms=1000) + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table_a} SETTINGS allow_experimental_export_merge_tree_part=1;" + ) + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table_b} SETTINGS allow_experimental_export_merge_tree_part=1;" + ) + + time.sleep(5) + + # Both targets should receive the same data independently + assert node.query(f"SELECT count() FROM {s3_table_a} WHERE year = 2020") == '3\n', "First target did not receive expected rows" + assert node.query(f"SELECT count() FROM {s3_table_b} WHERE year = 2020") == '3\n', "Second target did not receive expected rows" + + # And both should have a commit marker + assert node.query( + f"SELECT count() FROM s3(s3_conn, filename='{s3_table_a}/commit_2020_*', format=LineAsString)" + ) != '0\n', "Commit file missing for first target" + assert node.query( + f"SELECT count() FROM s3(s3_conn, filename='{s3_table_b}/commit_2020_*', format=LineAsString)" + ) != '0\n', "Commit file missing for second target" + + +def test_failure_is_logged_in_system_table(cluster): + node = cluster.instances["replica1"] + + mt_table = "failure_is_logged_in_system_table_mt_table" + s3_table = "failure_is_logged_in_system_table_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # Block traffic to/from MinIO to force upload errors and retries, following existing S3 tests style + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "destination": node.ip_address, + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_responses) + + # Also block requests to MinIO (destination: MinIO, destination_port: 9001) with REJECT to fail fast + pm_rule_reject_requests = { + "destination": minio_ip, + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_requests) + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_max_retries=1;" + ) + + # Wait so that the export fails + time.sleep(5) + + # Network restored; verify the export is marked as FAILED in the system table + # Also verify we captured at least one exception and no commit file exists + status = node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) + + assert status.strip() == "FAILED", f"Expected FAILED status, got: {status!r}" + + exception_count = node.query( + f""" + SELECT any(exception_count) FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) + assert int(exception_count.strip()) > 0, "Expected non-zero exception_count in system.replicated_partition_exports" + + # No commit should have been produced for this partition + assert node.query( + f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)" + ) == '0\n', "Commit file exists despite forced S3 failures" + + +def test_inject_short_living_failures(cluster): + node = cluster.instances["replica1"] + + mt_table = "inject_short_living_failures_mt_table" + s3_table = "inject_short_living_failures_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # Block traffic to/from MinIO to force upload errors and retries, following existing S3 tests style + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "destination": node.ip_address, + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_responses) + + # Also block requests to MinIO (destination: MinIO, destination_port: 9001) with REJECT to fail fast + pm_rule_reject_requests = { + "destination": minio_ip, + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_requests) + + # set big max_retries so that the export does not fail completely + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_max_retries=100;" + ) + + # wait only for a second to get at least one failure, but not enough to finish the export + time.sleep(5) + + # wait for the export to finish + time.sleep(5) + + # Assert the export succeeded + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") == '3\n', "Export did not succeed" + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '1\n', "Export did not succeed" + + # check system.replicated_partition_exports for the export + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) == "COMPLETED\n", "Export should be marked as COMPLETED" + + exception_count = node.query( + f""" + SELECT exception_count FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) + assert int(exception_count.strip()) >= 1, "Expected at least one exception" + + +# def test_source_mutations_during_export_snapshot(cluster): +# node = cluster.instances["replica1"] + +# mt_table = "mutations_snapshot_mt_table" +# s3_table = "mutations_snapshot_s3_table" + +# create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + +# # Ensure export sees a consistent snapshot at start time even if we mutate the source later +# with PartitionManager() as pm: +# pm.add_network_delay(node, delay_ms=5000) + +# # Start export of 2020 +# node.query( +# f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1;" +# ) + +# # Mutate the source after export started (delete the same partition) +# node.query(f"ALTER TABLE {mt_table} DROP COLUMN id") + +# # assert the mutation has been applied AND the data has not been exported yet +# assert node.query(f"SELECT count() FROM {mt_table} WHERE year = 2020") == '0\n', "Mutation has not been applied" +# assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") == '0\n', "Data has been exported" + +# # Wait for export to finish and then verify destination still reflects the original snapshot (3 rows) +# time.sleep(5) +# assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") == '3\n', "Export did not preserve snapshot at start time after source mutation" From f23ed2d985b72daf68331377fc3860761d382a0e Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 5 Nov 2025 01:23:02 +0100 Subject: [PATCH 28/58] preserve order of tasks.. --- ...ortReplicatedMergeTreePartitionTaskEntry.h | 37 ++++++++++ .../ExportPartitionManifestUpdatingTask.cpp | 67 +++++++++-------- .../ExportPartitionTaskScheduler.cpp | 4 +- .../tests/gtest_export_partition_ordering.cpp | 74 +++++++++++++++++++ src/Storages/StorageReplicatedMergeTree.cpp | 4 +- src/Storages/StorageReplicatedMergeTree.h | 5 +- 6 files changed, 156 insertions(+), 35 deletions(-) create mode 100644 src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp diff --git a/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h index cf3b14629fae..07aad0aacb8a 100644 --- a/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h +++ b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h @@ -2,6 +2,11 @@ #include #include +#include "Core/QualifiedTableName.h" +#include +#include +#include +#include namespace DB { @@ -15,6 +20,38 @@ struct ExportReplicatedMergeTreePartitionTaskEntry /// It does not mean this replica will export all the parts /// There is also a chance this replica does not contain a given part and it is totally ok. std::vector part_references; + + std::string getCompositeKey() const + { + const auto qualified_table_name = QualifiedTableName {manifest.destination_database, manifest.destination_table}; + return manifest.partition_id + "_" + qualified_table_name.getFullName(); + } + + /// Get create_time for sorted iteration + time_t getCreateTime() const + { + return manifest.create_time; + } }; +struct ExportPartitionTaskEntryTagByCompositeKey {}; +struct ExportPartitionTaskEntryTagByCreateTime {}; + +// Multi-index container for export partition task entries +// - Index 0 (TagByCompositeKey): hashed_unique on composite key for O(1) lookup +// - Index 1 (TagByCreateTime): ordered_non_unique on create_time for sorted iteration +using ExportPartitionTaskEntriesContainer = boost::multi_index_container< + ExportReplicatedMergeTreePartitionTaskEntry, + boost::multi_index::indexed_by< + boost::multi_index::hashed_unique< + boost::multi_index::tag, + boost::multi_index::const_mem_fun + >, + boost::multi_index::ordered_non_unique< + boost::multi_index::tag, + boost::multi_index::const_mem_fun + > + > +>; + } diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp index 3948ae9532be..2299f0f2d373 100644 --- a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp @@ -98,12 +98,13 @@ void ExportPartitionManifestUpdatingTask::run() const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); - const auto local_entry = storage.export_merge_tree_partition_task_entries.find(key); + auto & entries_by_key = storage.export_merge_tree_partition_task_entries_by_key; + const auto local_entry = entries_by_key.find(key); /// If the zk entry has been replaced with export_merge_tree_partition_force_export, checking only for the export key is not enough /// we need to make sure it is the same transaction id. If it is not, it needs to be replaced. - bool has_local_entry_and_is_up_to_date = local_entry != storage.export_merge_tree_partition_task_entries.end() - && local_entry->second.manifest.transaction_id == metadata.transaction_id; + bool has_local_entry_and_is_up_to_date = local_entry != entries_by_key.end() + && local_entry->manifest.transaction_id == metadata.transaction_id; /// If the entry is up to date and we don't have the cleanup lock, early exit, nothing to be done. if (!cleanup_lock_acquired && has_local_entry_and_is_up_to_date) @@ -125,7 +126,9 @@ void ExportPartitionManifestUpdatingTask::run() if (has_expired && is_not_pending) { zk->tryRemoveRecursive(fs::path(entry_path)); - storage.export_merge_tree_partition_task_entries.erase(key); + auto it = entries_by_key.find(key); + if (it != entries_by_key.end()) + entries_by_key.erase(it); LOG_INFO(storage.log, "ExportPartition: Removed {}: expired", key); continue; } @@ -180,46 +183,46 @@ void ExportPartitionManifestUpdatingTask::run() } } - /// It is important to use the operator[] because it updates the existing entry if it already exists. - storage.export_merge_tree_partition_task_entries[key] = ExportReplicatedMergeTreePartitionTaskEntry {metadata, std::move(part_references)}; + /// Insert or update entry. The multi_index container automatically maintains both indexes. + auto entry = ExportReplicatedMergeTreePartitionTaskEntry {metadata, std::move(part_references)}; + auto it = entries_by_key.find(key); + if (it != entries_by_key.end()) + entries_by_key.replace(it, entry); + else + entries_by_key.insert(entry); } /// Remove entries that were deleted by someone else - std::erase_if(storage.export_merge_tree_partition_task_entries, - [&](auto const & kv) + auto & entries_by_key = storage.export_merge_tree_partition_task_entries_by_key; + for (auto it = entries_by_key.begin(); it != entries_by_key.end();) + { + const auto & key = it->getCompositeKey(); + if (zk_children.contains(key)) { - if (zk_children.contains(kv.first)) - { - return false; - } + ++it; + continue; + } - const auto & transaction_id = kv.second.manifest.transaction_id; - LOG_INFO(storage.log, "ExportPartition: Export task {} was deleted, calling killExportPartition for transaction {}", kv.first, transaction_id); - - try - { - storage.killExportPart(transaction_id); - } - catch (...) - { - tryLogCurrentException(storage.log, __PRETTY_FUNCTION__); - } + const auto & transaction_id = it->manifest.transaction_id; + LOG_INFO(storage.log, "ExportPartition: Export task {} was deleted, calling killExportPartition for transaction {}", key, transaction_id); + + try + { + storage.killExportPart(transaction_id); + } + catch (...) + { + tryLogCurrentException(storage.log, __PRETTY_FUNCTION__); + } - return true; - }); + it = entries_by_key.erase(it); + } if (cleanup_lock_acquired) { zk->tryRemove(cleanup_lock_path); } - /// todo arthur remember to sort the entries by create_time - // std::sort(storage.export_merge_tree_partition_task_entries.begin(), storage.export_merge_tree_partition_task_entries.end(), - // [](const auto & a, const auto & b) - // { - // return a.second.manifest.create_time < b.second.manifest.create_time; - // }); - storage.export_merge_tree_partition_select_task->schedule(); } diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp index 84db08a4c3aa..3e127250a529 100644 --- a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -20,9 +20,11 @@ void ExportPartitionTaskScheduler::run() auto zk = storage.getZooKeeper(); - for (const auto & [key, entry] : storage.export_merge_tree_partition_task_entries) + // Iterate sorted by create_time + for (const auto & entry : storage.export_merge_tree_partition_task_entries_by_create_time) { const auto & manifest = entry.manifest; + const auto key = entry.getCompositeKey(); const auto & database = storage.getContext()->resolveDatabase(manifest.destination_database); const auto & table = manifest.destination_table; diff --git a/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp b/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp new file mode 100644 index 000000000000..679092251b99 --- /dev/null +++ b/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp @@ -0,0 +1,74 @@ +#include +#include + +namespace DB +{ + +// Test fixture that uses the shared container definition +class ExportPartitionOrderingTest : public ::testing::Test +{ +protected: + ExportPartitionTaskEntriesContainer container; + ExportPartitionTaskEntriesContainer::index::type & by_key; + ExportPartitionTaskEntriesContainer::index::type & by_create_time; + + ExportPartitionOrderingTest() + : by_key(container.get()) + , by_create_time(container.get()) + { + } +}; + +TEST_F(ExportPartitionOrderingTest, IterationOrderMatchesCreateTime) +{ + // Create entries with different create_times (in reverse order) + time_t base_time = 1000; + + ExportReplicatedMergeTreePartitionManifest manifest1; + manifest1.partition_id = "2020"; + manifest1.destination_database = "db1"; + manifest1.destination_table = "table1"; + manifest1.create_time = base_time + 300; // Latest + + ExportReplicatedMergeTreePartitionManifest manifest2; + manifest2.partition_id = "2021"; + manifest2.destination_database = "db1"; + manifest2.destination_table = "table1"; + manifest2.create_time = base_time + 100; // Middle + + ExportReplicatedMergeTreePartitionManifest manifest3; + manifest3.partition_id = "2022"; + manifest3.destination_database = "db1"; + manifest3.destination_table = "table1"; + manifest3.create_time = base_time; // Oldest + + ExportReplicatedMergeTreePartitionTaskEntry entry1{manifest1, {}}; + ExportReplicatedMergeTreePartitionTaskEntry entry2{manifest2, {}}; + ExportReplicatedMergeTreePartitionTaskEntry entry3{manifest3, {}}; + + // Insert in reverse order + by_key.insert(entry1); + by_key.insert(entry2); + by_key.insert(entry3); + + // Verify iteration order matches create_time (ascending) + auto it = by_create_time.begin(); + ASSERT_NE(it, by_create_time.end()); + EXPECT_EQ(it->manifest.partition_id, "2022"); // Oldest first + EXPECT_EQ(it->manifest.create_time, base_time); + + ++it; + ASSERT_NE(it, by_create_time.end()); + EXPECT_EQ(it->manifest.partition_id, "2021"); + EXPECT_EQ(it->manifest.create_time, base_time + 100); + + ++it; + ASSERT_NE(it, by_create_time.end()); + EXPECT_EQ(it->manifest.partition_id, "2020"); + EXPECT_EQ(it->manifest.create_time, base_time + 300); + + ++it; + EXPECT_EQ(it, by_create_time.end()); +} + +} diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 8a041af3de83..296d5288b7fa 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -425,6 +425,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , merge_strategy_picker(*this) , queue(*this, merge_strategy_picker) , fetcher(*this) + , export_merge_tree_partition_task_entries_by_key(export_merge_tree_partition_task_entries.get()) + , export_merge_tree_partition_task_entries_by_create_time(export_merge_tree_partition_task_entries.get()) , cleanup_thread(*this) , async_block_ids_cache(*this) , part_check_thread(*this) @@ -8080,7 +8082,7 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & const auto exports_path = fs::path(zookeeper_path) / "exports"; Coordination::Requests ops; - const auto export_key = partition_id + "_" + dest_storage_id.getNameForLogs(); + const auto export_key = partition_id + "_" + dest_storage_id.getQualifiedName().getFullName(); const auto partition_exports_path = fs::path(exports_path) / export_key; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 7cc5f9658eed..ab7a49398ace 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -531,8 +531,11 @@ class StorageReplicatedMergeTree final : public MergeTreeData BackgroundSchedulePoolTaskHolder export_merge_tree_partition_select_task; + ExportPartitionTaskEntriesContainer export_merge_tree_partition_task_entries; - std::unordered_map export_merge_tree_partition_task_entries; + // Convenience references to indexes + ExportPartitionTaskEntriesContainer::index::type & export_merge_tree_partition_task_entries_by_key; + ExportPartitionTaskEntriesContainer::index::type & export_merge_tree_partition_task_entries_by_create_time; /// A thread that removes old parts, log entries, and blocks. ReplicatedMergeTreeCleanupThread cleanup_thread; From fb2d7f7e0abef479362e8f6d27798b2b904af948 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 5 Nov 2025 12:16:07 +0100 Subject: [PATCH 29/58] increase some sleeps to try to make the test more stable? --- src/Core/SettingsChangesHistory.cpp | 2 ++ .../test.py | 16 ++++++---------- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 9edb667d20b5..066380c8f68d 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -47,6 +47,8 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"allow_retries_in_cluster_requests", false, false, "New setting"}, {"object_storage_remote_initiator", false, false, "New setting."}, {"allow_experimental_export_merge_tree_part", false, true, "Turned ON by default for Antalya."}, + {"export_merge_tree_partition_force_export", false, false, "New setting."}, + {"export_merge_tree_partition_max_retries", 3, 3, "New setting."}, }); addSettingsChanges(settings_changes_history, "25.8", { diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index 3ef25629e40f..f47a5f6dc113 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -67,7 +67,7 @@ def test_restart_nodes_during_export(cluster): # Add network delays so we can kill the node during the export with PartitionManager() as pm: - pm.add_network_delay(node, delay_ms=1000) + pm.add_network_delay(node, delay_ms=5000) export_queries = f""" ALTER TABLE {mt_table} @@ -110,7 +110,7 @@ def test_kill_export(cluster): create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") with PartitionManager() as pm: - pm.add_network_delay(node, delay_ms=1000) + pm.add_network_delay(node, delay_ms=5000) export_queries = f""" ALTER TABLE {mt_table} @@ -183,17 +183,13 @@ def test_kill_export_by_table(cluster): # Slow down network so we can issue KILL mid-flight with PartitionManager() as pm: - pm.add_network_delay(node, delay_ms=3000) + pm.add_network_delay(node, delay_ms=5000) # Start two exports for the same table and one export for another table concurrently node.query( - f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1;" - ) - node.query( - f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2021' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1;" - ) - node.query( - f"ALTER TABLE {alt_mt_table} EXPORT PARTITION ID '2020' TO TABLE {alt_s3_table} SETTINGS allow_experimental_export_merge_tree_part=1;" + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1; + ALTER TABLE {mt_table} EXPORT PARTITION ID '2021' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1; + ALTER TABLE {alt_mt_table} EXPORT PARTITION ID '2020' TO TABLE {alt_s3_table} SETTINGS allow_experimental_export_merge_tree_part=1;" ) # Kill all exports for the first table only From ee2abd01b4f52169df03bb9dd272630c84f2d153 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 5 Nov 2025 15:30:45 +0100 Subject: [PATCH 30/58] implement ttl that depends on cleanup, try to make tests more stable --- src/Core/Settings.cpp | 4 + ...portReplicatedMergeTreePartitionManifest.h | 3 + .../ExportPartitionManifestUpdatingTask.cpp | 64 +++++++--- .../ExportPartitionTaskScheduler.cpp | 42 +++---- .../tests/gtest_export_partition_ordering.cpp | 2 - src/Storages/StorageReplicatedMergeTree.cpp | 2 + .../test.py | 115 ++++++++++++++---- 7 files changed, 166 insertions(+), 66 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 2620b15a7532..39548489a35e 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -6879,6 +6879,10 @@ Ignore existing partition export and overwrite the zookeeper entry )", 0) \ DECLARE(UInt64, export_merge_tree_partition_max_retries, 3, R"( Maximum number of retries for exporting a merge tree part in an export partition task +)", 0) \ + DECLARE(UInt64, export_merge_tree_partition_manifest_ttl, 180, R"( +Determines how long the manifest will live in ZooKeeper. It prevents the same partition from being exported twice to the same destination. +This setting does not affect / delete in progress tasks. It'll only cleanup the completed ones. )", 0) \ \ /* ####################################################### */ \ diff --git a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h index 1c8556e570f4..da905b98dac4 100644 --- a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h +++ b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h @@ -20,6 +20,7 @@ struct ExportReplicatedMergeTreePartitionManifest std::vector parts; time_t create_time; size_t max_retries; + size_t ttl_seconds; std::string toJsonString() const { @@ -38,6 +39,7 @@ struct ExportReplicatedMergeTreePartitionManifest json.set("create_time", create_time); json.set("max_retries", max_retries); + json.set("ttl_seconds", ttl_seconds); std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); Poco::JSON::Stringifier::stringify(json, oss); @@ -63,6 +65,7 @@ struct ExportReplicatedMergeTreePartitionManifest manifest.parts.push_back(parts_array->getElement(static_cast(i))); manifest.create_time = json->getValue("create_time"); + manifest.ttl_seconds = json->getValue("ttl_seconds"); return manifest; } }; diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp index 2299f0f2d373..059134c312e7 100644 --- a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp @@ -55,6 +55,35 @@ table_path/ This is the overall idea, but please read the code to get a better understanding */ +struct CleanupLockRAII +{ + CleanupLockRAII(const zkutil::ZooKeeperPtr & zk_, const std::string & cleanup_lock_path_, const std::string & replica_name_, const LoggerPtr & log_) + : cleanup_lock_path(cleanup_lock_path_), zk(zk_), replica_name(replica_name_), log(log_) + { + is_locked = zk->tryCreate(cleanup_lock_path, replica_name, ::zkutil::CreateMode::Ephemeral) == Coordination::Error::ZOK; + + if (is_locked) + { + LOG_INFO(log, "ExportPartition Manifest Updating Task: Cleanup lock acquired, will remove stale entries"); + } + } + + ~CleanupLockRAII() + { + if (is_locked) + { + LOG_INFO(log, "ExportPartition Manifest Updating Task: Releasing cleanup lock"); + zk->tryRemove(cleanup_lock_path); + } + } + + bool is_locked; + std::string cleanup_lock_path; + zkutil::ZooKeeperPtr zk; + std::string replica_name; + LoggerPtr log; +}; + ExportPartitionManifestUpdatingTask::ExportPartitionManifestUpdatingTask(StorageReplicatedMergeTree & storage_) : storage(storage_) { @@ -69,12 +98,7 @@ void ExportPartitionManifestUpdatingTask::run() const std::string exports_path = fs::path(storage.zookeeper_path) / "exports"; const std::string cleanup_lock_path = fs::path(storage.zookeeper_path) / "exports_cleanup_lock"; - bool cleanup_lock_acquired = zk->tryCreate(cleanup_lock_path, "", ::zkutil::CreateMode::Ephemeral) == Coordination::Error::ZOK; - - if (cleanup_lock_acquired) - { - LOG_INFO(storage.log, "ExportPartition: Cleanup lock acquired, will remove stale entries"); - } + CleanupLockRAII cleanup_lock(zk, cleanup_lock_path, storage.replica_name, storage.log.load()); Coordination::Stat stat; const auto children = zk->getChildrenWatch(exports_path, &stat, storage.export_merge_tree_partition_watch_callback); @@ -92,7 +116,7 @@ void ExportPartitionManifestUpdatingTask::run() std::string metadata_json; if (!zk->tryGet(fs::path(entry_path) / "metadata.json", metadata_json)) { - LOG_INFO(storage.log, "ExportPartition: Skipping {}: missing metadata.json", key); + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: missing metadata.json", key); continue; } @@ -107,21 +131,21 @@ void ExportPartitionManifestUpdatingTask::run() && local_entry->manifest.transaction_id == metadata.transaction_id; /// If the entry is up to date and we don't have the cleanup lock, early exit, nothing to be done. - if (!cleanup_lock_acquired && has_local_entry_and_is_up_to_date) + if (!cleanup_lock.is_locked && has_local_entry_and_is_up_to_date) continue; std::string status; if (!zk->tryGet(fs::path(entry_path) / "status", status)) { - LOG_INFO(storage.log, "ExportPartition: Skipping {}: missing status", key); + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: missing status", key); continue; } bool is_not_pending = status != "PENDING"; - if (cleanup_lock_acquired) + if (cleanup_lock.is_locked) { - bool has_expired = metadata.create_time < now - 180; + bool has_expired = metadata.create_time < now - static_cast(metadata.ttl_seconds); if (has_expired && is_not_pending) { @@ -129,36 +153,36 @@ void ExportPartitionManifestUpdatingTask::run() auto it = entries_by_key.find(key); if (it != entries_by_key.end()) entries_by_key.erase(it); - LOG_INFO(storage.log, "ExportPartition: Removed {}: expired", key); + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Removed {}: expired", key); continue; } } if (is_not_pending) { - LOG_INFO(storage.log, "ExportPartition: Skipping {}: status is not PENDING", key); + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: status is not PENDING", key); continue; } - if (cleanup_lock_acquired) + if (cleanup_lock.is_locked) { std::vector parts_in_processing_or_pending; if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(entry_path) / "processing", parts_in_processing_or_pending)) { - LOG_INFO(storage.log, "ExportPartition: Failed to get parts in processing or pending, skipping"); + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Failed to get parts in processing or pending, skipping"); continue; } if (parts_in_processing_or_pending.empty()) { - LOG_INFO(storage.log, "ExportPartition: Cleanup found PENDING for {} with all parts exported, try to fix it by committing the export", entry_path); + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Cleanup found PENDING for {} with all parts exported, try to fix it by committing the export", entry_path); const auto destination_storage_id = StorageID(QualifiedTableName {metadata.destination_database, metadata.destination_table}); const auto destination_storage = DatabaseCatalog::instance().tryGetTable(destination_storage_id, storage.getContext()); if (!destination_storage) { - LOG_INFO(storage.log, "ExportPartition: Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); continue; } @@ -169,7 +193,7 @@ void ExportPartitionManifestUpdatingTask::run() if (has_local_entry_and_is_up_to_date) { - LOG_INFO(storage.log, "ExportPartition: Skipping {}: already exists", key); + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: already exists", key); continue; } @@ -204,7 +228,7 @@ void ExportPartitionManifestUpdatingTask::run() } const auto & transaction_id = it->manifest.transaction_id; - LOG_INFO(storage.log, "ExportPartition: Export task {} was deleted, calling killExportPartition for transaction {}", key, transaction_id); + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Export task {} was deleted, calling killExportPartition for transaction {}", key, transaction_id); try { @@ -218,7 +242,7 @@ void ExportPartitionManifestUpdatingTask::run() it = entries_by_key.erase(it); } - if (cleanup_lock_acquired) + if (cleanup_lock.is_locked) { zk->tryRemove(cleanup_lock_path); } diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp index 3e127250a529..bdc00d0fa598 100644 --- a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -34,20 +34,20 @@ void ExportPartitionTaskScheduler::run() if (!destination_storage) { - LOG_INFO(storage.log, "ExportPartition: Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); continue; } std::string status; if (!zk->tryGet(fs::path(storage.zookeeper_path) / "exports" / key / "status", status)) { - LOG_INFO(storage.log, "ExportPartition: Failed to get status, skipping"); + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get status, skipping"); continue; } if (status != "PENDING") { - LOG_INFO(storage.log, "ExportPartition: Skipping... Status is not PENDING"); + LOG_INFO(storage.log, "ExportPartition scheduler task: Skipping... Status is not PENDING"); continue; } @@ -55,13 +55,13 @@ void ExportPartitionTaskScheduler::run() if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(storage.zookeeper_path) / "exports" / key / "processing", parts_in_processing_or_pending)) { - LOG_INFO(storage.log, "ExportPartition: Failed to get parts in processing or pending, skipping"); + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get parts in processing or pending, skipping"); continue; } if (parts_in_processing_or_pending.empty()) { - LOG_INFO(storage.log, "ExportPartition: No parts in processing or pending, skipping"); + LOG_INFO(storage.log, "ExportPartition scheduler task: No parts in processing or pending, skipping"); continue; } @@ -69,7 +69,7 @@ void ExportPartitionTaskScheduler::run() if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(storage.zookeeper_path) / "exports" / key / "locks", locked_parts)) { - LOG_INFO(storage.log, "ExportPartition: Failed to get locked parts, skipping"); + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get locked parts, skipping"); continue; } @@ -79,20 +79,20 @@ void ExportPartitionTaskScheduler::run() { if (locked_parts_set.contains(zk_part_name)) { - LOG_INFO(storage.log, "ExportPartition: Part {} is locked, skipping", zk_part_name); + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is locked, skipping", zk_part_name); continue; } const auto part = storage.getPartIfExists(zk_part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); if (!part) { - LOG_INFO(storage.log, "ExportPartition: Part {} not found locally, skipping", zk_part_name); + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} not found locally, skipping", zk_part_name); continue; } if (Coordination::Error::ZOK != zk->tryCreate(fs::path(storage.zookeeper_path) / "exports" / key / "locks" / zk_part_name, storage.replica_name, zkutil::CreateMode::Ephemeral)) { - LOG_INFO(storage.log, "ExportPartition: Failed to lock part {}, skipping", zk_part_name); + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to lock part {}, skipping", zk_part_name); continue; } @@ -155,14 +155,14 @@ void ExportPartitionTaskScheduler::handlePartExportSuccess( const String & relative_path_in_destination_storage ) { - LOG_INFO(storage.log, "ExportPartition: Part {} exported successfully", relative_path_in_destination_storage); + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} exported successfully", relative_path_in_destination_storage); Coordination::Stat locked_by_stat; std::string locked_by; if (!zk->tryGet(export_path / "locks" / part_name, locked_by, &locked_by_stat)) { - LOG_INFO(storage.log, "ExportPartition: Part {} is not locked by any replica, will not commit or set it as completed", part_name); + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is not locked by any replica, will not commit or set it as completed", part_name); return; } @@ -170,7 +170,7 @@ void ExportPartitionTaskScheduler::handlePartExportSuccess( /// I guess we should not throw if file already exists for export partition, hard coded. if (locked_by != storage.replica_name) { - LOG_INFO(storage.log, "ExportPartition: Part {} is locked by another replica, will not commit or set it as completed", part_name); + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is locked by another replica, will not commit or set it as completed", part_name); return; } @@ -199,26 +199,26 @@ void ExportPartitionTaskScheduler::handlePartExportSuccess( if (Coordination::Error::ZOK != zk->tryMulti(requests, responses)) { /// todo arthur remember what to do here - LOG_INFO(storage.log, "ExportPartition: Failed to update export path, skipping"); + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to update export path, skipping"); return; } - LOG_INFO(storage.log, "ExportPartition: Marked part export {} as completed", part_name); + LOG_INFO(storage.log, "ExportPartition scheduler task: Marked part export {} as completed", part_name); Strings parts_in_processing_or_pending; if (Coordination::Error::ZOK != zk->tryGetChildren(export_path / "processing", parts_in_processing_or_pending)) { - LOG_INFO(storage.log, "ExportPartition: Failed to get parts in processing or pending, will not try to commit export partition"); + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get parts in processing or pending, will not try to commit export partition"); return; } if (!parts_in_processing_or_pending.empty()) { - LOG_INFO(storage.log, "ExportPartition: There are still parts in processing or pending, will not try to commit export partition"); + LOG_INFO(storage.log, "ExportPartition scheduler task: There are still parts in processing or pending, will not try to commit export partition"); return; } - LOG_INFO(storage.log, "ExportPartition: All parts are processed, will try to commit export partition"); + LOG_INFO(storage.log, "ExportPartition scheduler task: All parts are processed, will try to commit export partition"); ExportPartitionUtils::commit(manifest, destination_storage, zk, storage.log.load(), export_path, storage.getContext()); } @@ -239,13 +239,13 @@ void ExportPartitionTaskScheduler::handlePartExportFailure( if (!zk->tryGet(export_path / "locks" / part_name, locked_by, &locked_by_stat)) { - LOG_INFO(storage.log, "ExportPartition: Part {} is not locked by any replica, will not increment error counts", part_name); + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is not locked by any replica, will not increment error counts", part_name); return; } if (locked_by != storage.replica_name) { - LOG_INFO(storage.log, "ExportPartition: Part {} is locked by another replica, will not increment error counts", part_name); + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is locked by another replica, will not increment error counts", part_name); return; } @@ -267,7 +267,7 @@ void ExportPartitionTaskScheduler::handlePartExportFailure( ops.emplace_back(zkutil::makeCreateRequest(processing_part_path / "finished_by", storage.replica_name, zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeSetRequest(export_path / "status", "FAILED", -1)); - LOG_INFO(storage.log, "ExportPartition: Retry count limit exceeded for part {}, will try to fail the entire task", part_name); + LOG_INFO(storage.log, "ExportPartition scheduler task: Retry count limit exceeded for part {}, will try to fail the entire task", part_name); } std::size_t num_exceptions = 0; @@ -300,7 +300,7 @@ void ExportPartitionTaskScheduler::handlePartExportFailure( Coordination::Responses responses; if (Coordination::Error::ZOK != zk->tryMulti(ops, responses)) { - LOG_INFO(storage.log, "ExportPartition: All failure mechanism failed, will not try to update it"); + LOG_INFO(storage.log, "ExportPartition scheduler task: All failure mechanism failed, will not try to update it"); return; } } diff --git a/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp b/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp index 679092251b99..480f76db4d99 100644 --- a/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp +++ b/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp @@ -4,7 +4,6 @@ namespace DB { -// Test fixture that uses the shared container definition class ExportPartitionOrderingTest : public ::testing::Test { protected: @@ -21,7 +20,6 @@ class ExportPartitionOrderingTest : public ::testing::Test TEST_F(ExportPartitionOrderingTest, IterationOrderMatchesCreateTime) { - // Create entries with different create_times (in reverse order) time_t base_time = 1000; ExportReplicatedMergeTreePartitionManifest manifest1; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 296d5288b7fa..1de791e89d44 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -194,6 +194,7 @@ namespace Setting extern const SettingsBool allow_experimental_export_merge_tree_part; extern const SettingsBool export_merge_tree_partition_force_export; extern const SettingsUInt64 export_merge_tree_partition_max_retries; + extern const SettingsUInt64 export_merge_tree_partition_manifest_ttl; } namespace MergeTreeSetting @@ -8128,6 +8129,7 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & manifest.parts = part_names; manifest.create_time = time(nullptr); manifest.max_retries = query_context->getSettingsRef()[Setting::export_merge_tree_partition_max_retries]; + manifest.ttl_seconds = query_context->getSettingsRef()[Setting::export_merge_tree_partition_manifest_ttl]; ops.emplace_back(zkutil::makeCreateRequest( fs::path(partition_exports_path) / "metadata.json", diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index f47a5f6dc113..dc5109392d31 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -65,23 +65,51 @@ def test_restart_nodes_during_export(cluster): create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") create_s3_table(watcher_node, s3_table) - # Add network delays so we can kill the node during the export + # Block S3/MinIO requests to keep exports alive via retry mechanism + # This allows ZooKeeper operations to proceed quickly + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + with PartitionManager() as pm: - pm.add_network_delay(node, delay_ms=5000) + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses_node1 = { + "destination": node.ip_address, + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_responses_node1) + + pm_rule_reject_responses_node2 = { + "destination": node2.ip_address, + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_responses_node2) + + # Block requests to MinIO (destination: MinIO, destination_port: minio_port) + pm_rule_reject_requests = { + "destination": minio_ip, + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_requests) export_queries = f""" ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} - SETTINGS allow_experimental_export_merge_tree_part=1; + SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_max_retries = 50; ALTER TABLE {mt_table} EXPORT PARTITION ID '2021' TO TABLE {s3_table} - SETTINGS allow_experimental_export_merge_tree_part=1; + SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_max_retries = 50; """ node.query(export_queries) - - node.stop_clickhouse(kill=True) - node2.stop_clickhouse(kill=True) + + # wait for the exports to start + time.sleep(3) + + node.stop_clickhouse(kill=True) + node2.stop_clickhouse(kill=True) assert watcher_node.query(f"SELECT count() FROM {s3_table} where year = 2020") == '0\n', "Partition 2020 was written to S3 during network delay crash" @@ -109,22 +137,42 @@ def test_kill_export(cluster): create_tables_and_insert_data(node, mt_table, s3_table, "replica1") create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") + # Block S3/MinIO requests to keep exports alive via retry mechanism + # This allows ZooKeeper operations (KILL) to proceed quickly + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + with PartitionManager() as pm: - pm.add_network_delay(node, delay_ms=5000) + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "destination": node.ip_address, + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_responses) + + # Block requests to MinIO (destination: MinIO, destination_port: minio_port) + pm_rule_reject_requests = { + "destination": minio_ip, + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_requests) export_queries = f""" ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} - SETTINGS allow_experimental_export_merge_tree_part=1; + SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_max_retries = 50; ALTER TABLE {mt_table} EXPORT PARTITION ID '2021' TO TABLE {s3_table} - SETTINGS allow_experimental_export_merge_tree_part=1; + SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_max_retries = 50; """ node.query(export_queries) - # kill only 2020, 2021 should still finish - node.query(f"KILL EXPORT PARTITION WHERE partition_id = '2020'") + # Kill only 2020 while S3 is blocked - retry mechanism keeps exports alive + # ZooKeeper operations (KILL) proceed quickly since only S3 is blocked + node.query(f"KILL EXPORT PARTITION WHERE partition_id = '2020'") # wait for 2021 to finish time.sleep(5) @@ -160,10 +208,10 @@ def test_drop_table_during_export(cluster): node.query(export_queries) - # I think this will actually wait until background operations are finished - node.query(f"DROP TABLE {mt_table} SYNC") - # this will not wait, but the pointer the background task holds is still valid, so the write will finish - node.query(f"DROP TABLE {s3_table}") + # I think this will actually wait until background operations are finished + node.query(f"DROP TABLE {mt_table} SYNC") + # this will not wait, but the pointer the background task holds is still valid, so the write will finish + node.query(f"DROP TABLE {s3_table}") time.sleep(5) @@ -181,19 +229,40 @@ def test_kill_export_by_table(cluster): create_tables_and_insert_data(node, mt_table, s3_table, "replica1") create_tables_and_insert_data(node, alt_mt_table, alt_s3_table, "replica1") - # Slow down network so we can issue KILL mid-flight + # Block S3/MinIO requests to keep exports alive via retry mechanism + # This allows ZooKeeper operations (KILL) to proceed quickly + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + with PartitionManager() as pm: - pm.add_network_delay(node, delay_ms=5000) + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "destination": node.ip_address, + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_responses) + + # Block requests to MinIO (destination: MinIO, destination_port: minio_port) + pm_rule_reject_requests = { + "destination": minio_ip, + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_requests) # Start two exports for the same table and one export for another table concurrently node.query( - f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1; - ALTER TABLE {mt_table} EXPORT PARTITION ID '2021' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1; - ALTER TABLE {alt_mt_table} EXPORT PARTITION ID '2020' TO TABLE {alt_s3_table} SETTINGS allow_experimental_export_merge_tree_part=1;" + f""" + ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_max_retries = 50; + ALTER TABLE {mt_table} EXPORT PARTITION ID '2021' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_max_retries = 50; + ALTER TABLE {alt_mt_table} EXPORT PARTITION ID '2020' TO TABLE {alt_s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_max_retries = 50; + """ ) - # Kill all exports for the first table only - node.query(f"KILL EXPORT PARTITION WHERE source_table = '{mt_table}'") + # Kill all exports for the first table only while S3 is blocked + # Retry mechanism keeps exports alive, ZooKeeper operations proceed quickly + node.query(f"KILL EXPORT PARTITION WHERE source_table = '{mt_table}'") # Give some time for effects to propagate time.sleep(5) From d6e4226ad5dfa34a5670130f970239900a638d1d Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 5 Nov 2025 19:07:16 +0100 Subject: [PATCH 31/58] improve ttl so it does not depend on cleanup, add test for ttl --- src/Core/SettingsChangesHistory.cpp | 1 + src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 32 +++++++++++++++--- .../test.py | 33 +++++++++++++++++++ 4 files changed, 62 insertions(+), 6 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 066380c8f68d..38e399f7e2ae 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -49,6 +49,7 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"allow_experimental_export_merge_tree_part", false, true, "Turned ON by default for Antalya."}, {"export_merge_tree_partition_force_export", false, false, "New setting."}, {"export_merge_tree_partition_max_retries", 3, 3, "New setting."}, + {"export_merge_tree_partition_manifest_ttl", 180, 180, "New setting."}, }); addSettingsChanges(settings_changes_history, "25.8", { diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 00d809f9f32e..abeda51be05e 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -995,7 +995,7 @@ class MergeTreeData : public IStorage, public WithMutableContext virtual void exportPartitionToTable(const PartitionCommand &, ContextPtr) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "EXPORT PARTITION is not implemented"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "EXPORT PARTITION is not implemented for engine {}", getName()); } /// Checks that Partition could be dropped right now diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1de791e89d44..7a6c24a141a3 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8081,7 +8081,6 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & const String partition_id = getPartitionIDFromQuery(command.partition, query_context); const auto exports_path = fs::path(zookeeper_path) / "exports"; - Coordination::Requests ops; const auto export_key = partition_id + "_" + dest_storage_id.getQualifiedName().getFullName(); @@ -8090,15 +8089,38 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & /// check if entry already exists if (zookeeper->exists(partition_exports_path)) { - if (!query_context->getSettingsRef()[Setting::export_merge_tree_partition_force_export]) + LOG_INFO(log, "Export with key {} is already exported or it is being exported. Checking if it has expired so that we can overwrite it.", export_key); + + bool has_expired = false; + + if (zookeeper->exists(fs::path(partition_exports_path) / "metadata.json")) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition {} already exported or it is being exported", partition_id); + std::string metadata_json; + if (zookeeper->tryGet(fs::path(partition_exports_path) / "metadata.json", metadata_json)) + { + const auto manifest = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + if (static_cast(manifest.create_time + manifest.ttl_seconds) < time(nullptr)) + { + has_expired = true; + } + } } - /// The check for existence and entry removal are not atomic, so this actually might fail. - ops.emplace_back(zkutil::makeRemoveRecursiveRequest(*zookeeper, partition_exports_path, -1)); + if (!has_expired && !query_context->getSettingsRef()[Setting::export_merge_tree_partition_force_export]) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Export with key {} already exported or it is being exported, and it has not expired. Set `export_merge_tree_partition_force_export` to overwrite it.", export_key); + } + + LOG_INFO(log, "Overwriting export with key {}", export_key); + + /// Not putting in ops (same transaction) because we can't construct a "tryRemoveRecursive" request. + /// It is possible that the zk being used does not support RemoveRecursive requests. + /// It is ok for this to be non transactional. Worst case scenario an on-going export is going to be killed and a new task won't be scheduled. + zookeeper->tryRemoveRecursive(partition_exports_path); } + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCreateRequest(partition_exports_path, "", zkutil::CreateMode::Persistent)); auto data_parts_lock = lockParts(); diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index dc5109392d31..52173e534f51 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -450,6 +450,39 @@ def test_inject_short_living_failures(cluster): assert int(exception_count.strip()) >= 1, "Expected at least one exception" +def test_export_ttl(cluster): + node = cluster.instances["replica1"] + + mt_table = "export_ttl_mt_table" + s3_table = "export_ttl_s3_table" + + expiration_time = 5 + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # start export + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_manifest_ttl={expiration_time};") + + # assert that I get an error when trying to export the same partition again, query_and_get_error + error = node.query_and_get_error(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1;") + assert "Export with key" in error, "Expected error about expired export" + + # wait for the export to finish and for the manifest to expire + time.sleep(expiration_time) + + # assert that the export succeeded, check the commit file + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '1\n', "Export did not succeed" + + # start export again + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1") + + # wait for the export to finish + time.sleep(expiration_time) + + # assert that the export succeeded, check the commit file + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '1\n', "Export did not succeed" + + # def test_source_mutations_during_export_snapshot(cluster): # node = cluster.instances["replica1"] From c2b5d8427f0badd3e8f863011f3b5c0b578565af Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 5 Nov 2025 20:03:24 +0100 Subject: [PATCH 32/58] cleanup the code in updating task --- .../ExportPartitionManifestUpdatingTask.cpp | 179 ++++++++++++------ .../ExportPartitionManifestUpdatingTask.h | 14 ++ 2 files changed, 130 insertions(+), 63 deletions(-) diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp index 059134c312e7..e932fd77fd25 100644 --- a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp @@ -84,6 +84,71 @@ struct CleanupLockRAII LoggerPtr log; }; +namespace +{ + /* + Remove expired entries and fix non-committed exports that have already exported all parts. + + Return values: + - true: the cleanup was successful, the entry is removed from the entries_by_key container and the function returns true. Proceed to the next entry. + - false: the cleanup was not successful, the entry is not removed from the entries_by_key container and the function returns false. + */ + bool tryCleanup( + const zkutil::ZooKeeperPtr & zk, + const std::string & entry_path, + const LoggerPtr & log, + const ContextPtr & context, + const std::string & key, + const ExportReplicatedMergeTreePartitionManifest & metadata, + const time_t now, + const bool is_pending, + auto & entries_by_key + ) + { + bool has_expired = metadata.create_time < now - static_cast(metadata.ttl_seconds); + + if (has_expired && !is_pending) + { + zk->tryRemoveRecursive(fs::path(entry_path)); + auto it = entries_by_key.find(key); + if (it != entries_by_key.end()) + entries_by_key.erase(it); + LOG_INFO(log, "ExportPartition Manifest Updating Task: Removed {}: expired", key); + + return true; + } + else if (is_pending) + { + std::vector parts_in_processing_or_pending; + if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(entry_path) / "processing", parts_in_processing_or_pending)) + { + LOG_INFO(log, "ExportPartition Manifest Updating Task: Failed to get parts in processing or pending, skipping"); + return false; + } + + if (parts_in_processing_or_pending.empty()) + { + LOG_INFO(log, "ExportPartition Manifest Updating Task: Cleanup found PENDING for {} with all parts exported, try to fix it by committing the export", entry_path); + + const auto destination_storage_id = StorageID(QualifiedTableName {metadata.destination_database, metadata.destination_table}); + const auto destination_storage = DatabaseCatalog::instance().tryGetTable(destination_storage_id, context); + if (!destination_storage) + { + LOG_INFO(log, "ExportPartition Manifest Updating Task: Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); + return false; + } + + /// it sounds like a replica exported the last part, but was not able to commit the export. Try to fix it + ExportPartitionUtils::commit(metadata, destination_storage, zk, log, entry_path, context); + + return true; + } + } + + return false; + } +} + ExportPartitionManifestUpdatingTask::ExportPartitionManifestUpdatingTask(StorageReplicatedMergeTree & storage_) : storage(storage_) { @@ -106,6 +171,8 @@ void ExportPartitionManifestUpdatingTask::run() const auto now = time(nullptr); + auto & entries_by_key = storage.export_merge_tree_partition_task_entries_by_key; + /// Load new entries /// If we have the cleanup lock, also remove stale entries from zk and local /// Upload dangling commit files if any @@ -122,7 +189,6 @@ void ExportPartitionManifestUpdatingTask::run() const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); - auto & entries_by_key = storage.export_merge_tree_partition_task_entries_by_key; const auto local_entry = entries_by_key.find(key); /// If the zk entry has been replaced with export_merge_tree_partition_force_export, checking only for the export key is not enough @@ -141,83 +207,77 @@ void ExportPartitionManifestUpdatingTask::run() continue; } - bool is_not_pending = status != "PENDING"; + bool is_pending = status == "PENDING"; + /// if we have the cleanup lock, try to cleanup + /// if we successfully cleaned it up, early exit if (cleanup_lock.is_locked) { - bool has_expired = metadata.create_time < now - static_cast(metadata.ttl_seconds); - - if (has_expired && is_not_pending) - { - zk->tryRemoveRecursive(fs::path(entry_path)); - auto it = entries_by_key.find(key); - if (it != entries_by_key.end()) - entries_by_key.erase(it); - LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Removed {}: expired", key); + bool cleanup_successful = tryCleanup( + zk, + entry_path, + storage.log.load(), + storage.getContext(), + key, + metadata, + now, + is_pending, entries_by_key); + + if (cleanup_successful) continue; - } } - if (is_not_pending) + if (!is_pending) { LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: status is not PENDING", key); continue; } - - if (cleanup_lock.is_locked) - { - std::vector parts_in_processing_or_pending; - if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(entry_path) / "processing", parts_in_processing_or_pending)) - { - LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Failed to get parts in processing or pending, skipping"); - continue; - } - - if (parts_in_processing_or_pending.empty()) - { - LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Cleanup found PENDING for {} with all parts exported, try to fix it by committing the export", entry_path); - - const auto destination_storage_id = StorageID(QualifiedTableName {metadata.destination_database, metadata.destination_table}); - const auto destination_storage = DatabaseCatalog::instance().tryGetTable(destination_storage_id, storage.getContext()); - if (!destination_storage) - { - LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); - continue; - } - - /// it sounds like a replica exported the last part, but was not able to commit the export. Try to fix it - ExportPartitionUtils::commit(metadata, destination_storage, zk, storage.log.load(), entry_path, storage.getContext()); - } - } - if (has_local_entry_and_is_up_to_date) { LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: already exists", key); continue; } - std::vector part_references; + addTask(metadata, key, entries_by_key); + } + + /// Remove entries that were deleted by someone else + removeStaleEntries(zk_children, entries_by_key); + + storage.export_merge_tree_partition_select_task->schedule(); +} + +void ExportPartitionManifestUpdatingTask::addTask( + const ExportReplicatedMergeTreePartitionManifest & metadata, + const std::string & key, + auto & entries_by_key +) +{ + std::vector part_references; - for (const auto & part_name : metadata.parts) + for (const auto & part_name : metadata.parts) + { + if (const auto part = storage.getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated})) { - if (const auto part = storage.getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated})) - { - part_references.push_back(part); - } + part_references.push_back(part); } - - /// Insert or update entry. The multi_index container automatically maintains both indexes. - auto entry = ExportReplicatedMergeTreePartitionTaskEntry {metadata, std::move(part_references)}; - auto it = entries_by_key.find(key); - if (it != entries_by_key.end()) - entries_by_key.replace(it, entry); - else - entries_by_key.insert(entry); } - /// Remove entries that were deleted by someone else - auto & entries_by_key = storage.export_merge_tree_partition_task_entries_by_key; + /// Insert or update entry. The multi_index container automatically maintains both indexes. + auto entry = ExportReplicatedMergeTreePartitionTaskEntry {metadata, std::move(part_references)}; + auto it = entries_by_key.find(key); + if (it != entries_by_key.end()) + entries_by_key.replace(it, entry); + else + entries_by_key.insert(entry); +} + +void ExportPartitionManifestUpdatingTask::removeStaleEntries( + const std::unordered_set & zk_children, + auto & entries_by_key +) +{ for (auto it = entries_by_key.begin(); it != entries_by_key.end();) { const auto & key = it->getCompositeKey(); @@ -241,13 +301,6 @@ void ExportPartitionManifestUpdatingTask::run() it = entries_by_key.erase(it); } - - if (cleanup_lock.is_locked) - { - zk->tryRemove(cleanup_lock_path); - } - - storage.export_merge_tree_partition_select_task->schedule(); } } diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h index 48f740504303..fa694299fc8a 100644 --- a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h @@ -1,9 +1,12 @@ #pragma once +#include +#include namespace DB { class StorageReplicatedMergeTree; +struct ExportReplicatedMergeTreePartitionManifest; class ExportPartitionManifestUpdatingTask { @@ -14,6 +17,17 @@ class ExportPartitionManifestUpdatingTask private: StorageReplicatedMergeTree & storage; + + void addTask( + const ExportReplicatedMergeTreePartitionManifest & metadata, + const std::string & key, + auto & entries_by_key + ); + + void removeStaleEntries( + const std::unordered_set & zk_children, + auto & entries_by_key + ); }; } From 601be4cbde1f44a29dbe78afb30a922c1a027314 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 5 Nov 2025 20:09:09 +0100 Subject: [PATCH 33/58] rmv useless change --- src/Common/ZooKeeper/ZooKeeper.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 5c608fbf9611..fdb404fa7ae2 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -380,8 +380,8 @@ class ZooKeeper Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL); using MultiGetChildrenResponse = MultiReadResponses; - using MultiTryGetChildrenResponse = MultiReadResponses; + template MultiGetChildrenResponse getChildren(TIter start, TIter end, Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL) From d9796b48646b77359d1a29944cd0760ef95cd8b2 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 6 Nov 2025 13:06:04 +0100 Subject: [PATCH 34/58] Cancel background exports instead of waiting for them upon drop table --- src/Storages/MergeTree/ExportPartTask.cpp | 9 +- .../test.py | 88 +++++++++++++++++-- 2 files changed, 88 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/ExportPartTask.cpp b/src/Storages/MergeTree/ExportPartTask.cpp index a72fbb1ded96..a735ffe20edb 100644 --- a/src/Storages/MergeTree/ExportPartTask.cpp +++ b/src/Storages/MergeTree/ExportPartTask.cpp @@ -168,8 +168,15 @@ bool ExportPartTask::executeStep() try { CompletedPipelineExecutor exec(pipeline); - exec.execute(); + auto is_cancelled_callback = [this]() + { + return storage.parts_mover.moves_blocker.isCancelled(); + }; + + exec.setCancelCallback(is_cancelled_callback, 100); + + exec.execute(); std::lock_guard inner_lock(storage.export_manifests_mutex); storage.writePartLog( PartLogElement::Type::EXPORT_PART, diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index 52173e534f51..97056dd1dfd3 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -182,20 +182,39 @@ def test_kill_export(cluster): assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2021_*', format=LineAsString)") != f'0\n', "Partition 2021 was not written to S3, but it should have been" -def test_drop_table_during_export(cluster): +def test_drop_source_table_during_export(cluster): node = cluster.instances["replica1"] # node2 = cluster.instances["replica2"] watcher_node = cluster.instances["watcher_node"] - mt_table = "drop_table_during_export_mt_table" - s3_table = "drop_table_during_export_s3_table" + mt_table = "drop_source_table_during_export_mt_table" + s3_table = "drop_source_table_during_export_s3_table" create_tables_and_insert_data(node, mt_table, s3_table, "replica1") # create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") create_s3_table(watcher_node, s3_table) + # Block S3/MinIO requests to keep exports alive via retry mechanism + # This allows ZooKeeper operations (KILL) to proceed quickly + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + with PartitionManager() as pm: - pm.add_network_delay(node, delay_ms=1000) + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "destination": node.ip_address, + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_responses) + + # Block requests to MinIO (destination: MinIO, destination_port: minio_port) + pm_rule_reject_requests = { + "destination": minio_ip, + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_requests) export_queries = f""" ALTER TABLE {mt_table} @@ -208,14 +227,67 @@ def test_drop_table_during_export(cluster): node.query(export_queries) - # I think this will actually wait until background operations are finished - node.query(f"DROP TABLE {mt_table} SYNC") - # this will not wait, but the pointer the background task holds is still valid, so the write will finish + # This should kill the background operations and drop the table + node.query(f"DROP TABLE {mt_table}") + + # Sleep some time to let the export finish (assuming it was not properly cancelled) + time.sleep(5) + + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_*', format=LineAsString)") == '0\n', "Background operations completed even with the table dropped" + + +def test_drop_destination_table_during_export(cluster): + node = cluster.instances["replica1"] + # node2 = cluster.instances["replica2"] + watcher_node = cluster.instances["watcher_node"] + + mt_table = "drop_destination_table_during_export_mt_table" + s3_table = "drop_destination_table_during_export_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + create_s3_table(watcher_node, s3_table) + + # Block S3/MinIO requests to keep exports alive via retry mechanism + # This allows ZooKeeper operations (KILL) to proceed quickly + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "destination": node.ip_address, + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_responses) + + # Block requests to MinIO (destination: MinIO, destination_port: minio_port) + pm_rule_reject_requests = { + "destination": minio_ip, + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_requests) + + export_queries = f""" + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2020' TO TABLE {s3_table} + SETTINGS allow_experimental_export_merge_tree_part=1; + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2021' TO TABLE {s3_table} + SETTINGS allow_experimental_export_merge_tree_part=1; + """ + + node.query(export_queries) + + # The pointer to the destination table is still valid, so the write will continue node.query(f"DROP TABLE {s3_table}") + # give some time for the export to finish time.sleep(5) - assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_*', format=LineAsString)") != '0\n', "Background operations finished even after the tables were dropped" + # not sure this is the expected behavior, but adding until we make a decision + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_*', format=LineAsString)") != '0\n', "Background operations did not complete after dropping the destination table" def test_kill_export_by_table(cluster): From 26bd6137dc8f4719e0675adbd2f4403a764e691f Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 6 Nov 2025 14:17:34 +0100 Subject: [PATCH 35/58] rmv bad ex log --- src/Storages/MergeTree/ExportPartTask.cpp | 2 ++ src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp | 2 -- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/ExportPartTask.cpp b/src/Storages/MergeTree/ExportPartTask.cpp index 4edab9902dee..36c9dfe95072 100644 --- a/src/Storages/MergeTree/ExportPartTask.cpp +++ b/src/Storages/MergeTree/ExportPartTask.cpp @@ -10,6 +10,7 @@ #include #include #include +#include "Common/Exception.h" #include namespace ProfileEvents @@ -86,6 +87,7 @@ bool ExportPartTask::executeStep() } catch (const Exception & e) { + tryLogCurrentException(__PRETTY_FUNCTION__); if (e.code() == ErrorCodes::FILE_ALREADY_EXISTS) { ProfileEvents::increment(ProfileEvents::PartsExportDuplicated); diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp index bdc00d0fa598..35d310395dc9 100644 --- a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -232,8 +232,6 @@ void ExportPartitionTaskScheduler::handlePartExportFailure( size_t max_retries ) { - tryLogCurrentException(__PRETTY_FUNCTION__); - Coordination::Stat locked_by_stat; std::string locked_by; From 803a91aa2cf3b9714dca4982ced7759e346d822d Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 7 Nov 2025 14:38:16 +0100 Subject: [PATCH 36/58] preserve some settings in export partition zk task --- .../ExportReplicatedMergeTreePartitionManifest.h | 10 +++++++++- .../MergeTree/ExportPartitionTaskScheduler.cpp | 14 +++++++++++++- src/Storages/StorageReplicatedMergeTree.cpp | 6 ++++++ 3 files changed, 28 insertions(+), 2 deletions(-) diff --git a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h index da905b98dac4..a8ea4dc474b3 100644 --- a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h +++ b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h @@ -21,6 +21,9 @@ struct ExportReplicatedMergeTreePartitionManifest time_t create_time; size_t max_retries; size_t ttl_seconds; + size_t max_threads; + bool parallel_formatting; + bool parquet_parallel_encoding; std::string toJsonString() const { @@ -36,7 +39,9 @@ struct ExportReplicatedMergeTreePartitionManifest for (const auto & part : parts) parts_array->add(part); json.set("parts", parts_array); - + json.set("parallel_formatting", parallel_formatting); + json.set("max_threads", max_threads); + json.set("parquet_parallel_encoding", parquet_parallel_encoding); json.set("create_time", create_time); json.set("max_retries", max_retries); json.set("ttl_seconds", ttl_seconds); @@ -66,6 +71,9 @@ struct ExportReplicatedMergeTreePartitionManifest manifest.create_time = json->getValue("create_time"); manifest.ttl_seconds = json->getValue("ttl_seconds"); + manifest.max_threads = json->getValue("max_threads"); + manifest.parallel_formatting = json->getValue("parallel_formatting"); + manifest.parquet_parallel_encoding = json->getValue("parquet_parallel_encoding"); return manifest; } }; diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp index 35d310395dc9..dcda713f5b39 100644 --- a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -9,6 +9,18 @@ namespace DB { +namespace +{ + ContextPtr getContextCopyWithTaskSettings(const ContextPtr & context, const ExportReplicatedMergeTreePartitionManifest & manifest) + { + auto context_copy = Context::createCopy(context); + context_copy->setSetting("output_format_parallel_formatting", manifest.parallel_formatting); + context_copy->setSetting("output_format_parquet_parallel_encoding", manifest.parquet_parallel_encoding); + context_copy->setSetting("max_threads", manifest.max_threads); + return context_copy; + } +} + ExportPartitionTaskScheduler::ExportPartitionTaskScheduler(StorageReplicatedMergeTree & storage_) : storage(storage_) { @@ -102,7 +114,7 @@ void ExportPartitionTaskScheduler::run() part->name, destination_storage_id, manifest.transaction_id, - storage.getContext(), + getContextCopyWithTaskSettings(storage.getContext(), manifest), [this, key, zk_part_name, manifest, destination_storage] (MergeTreePartExportManifest::CompletionCallbackResult result) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7a6c24a141a3..d396c4a594c6 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -195,6 +195,9 @@ namespace Setting extern const SettingsBool export_merge_tree_partition_force_export; extern const SettingsUInt64 export_merge_tree_partition_max_retries; extern const SettingsUInt64 export_merge_tree_partition_manifest_ttl; + extern const SettingsBool output_format_parallel_formatting; + extern const SettingsBool output_format_parquet_parallel_encoding; + extern const SettingsMaxThreads max_threads; } namespace MergeTreeSetting @@ -8152,6 +8155,9 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & manifest.create_time = time(nullptr); manifest.max_retries = query_context->getSettingsRef()[Setting::export_merge_tree_partition_max_retries]; manifest.ttl_seconds = query_context->getSettingsRef()[Setting::export_merge_tree_partition_manifest_ttl]; + manifest.max_threads = query_context->getSettingsRef()[Setting::max_threads]; + manifest.parallel_formatting = query_context->getSettingsRef()[Setting::output_format_parallel_formatting]; + manifest.parquet_parallel_encoding = query_context->getSettingsRef()[Setting::output_format_parquet_parallel_encoding]; ops.emplace_back(zkutil::makeCreateRequest( fs::path(partition_exports_path) / "metadata.json", From 7a12b3a2ff48c629fbbc9bc6a9623e5b6a9181dc Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 7 Nov 2025 17:41:08 +0100 Subject: [PATCH 37/58] make the zk structure a bit more flat --- ...portReplicatedMergeTreePartitionManifest.h | 78 +++++++++ .../ExportPartitionTaskScheduler.cpp | 150 +++++++++++------- .../MergeTree/ExportPartitionTaskScheduler.h | 14 ++ .../MergeTree/ExportPartitionUtils.cpp | 71 +++++---- src/Storages/StorageReplicatedMergeTree.cpp | 17 +- 5 files changed, 232 insertions(+), 98 deletions(-) diff --git a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h index a8ea4dc474b3..c715a95dd31b 100644 --- a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h +++ b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h @@ -9,6 +9,84 @@ namespace DB { +struct ExportReplicatedMergeTreePartitionProcessingPartEntry +{ + String part_name; + String status; + size_t retry_count; + String finished_by; + + std::string toJsonString() const + { + Poco::JSON::Object json; + + json.set("part_name", part_name); + json.set("status", status); + json.set("retry_count", retry_count); + json.set("finished_by", finished_by); + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + + return oss.str(); + } + + static ExportReplicatedMergeTreePartitionProcessingPartEntry fromJsonString(const std::string & json_string) + { + Poco::JSON::Parser parser; + auto json = parser.parse(json_string).extract(); + chassert(json); + + ExportReplicatedMergeTreePartitionProcessingPartEntry entry; + + entry.part_name = json->getValue("part_name"); + entry.status = json->getValue("status"); + entry.retry_count = json->getValue("retry_count"); + if (json->has("finished_by")) + { + entry.finished_by = json->getValue("finished_by"); + } + return entry; + } +}; + +struct ExportReplicatedMergeTreePartitionProcessedPartEntry +{ + String part_name; + String path_in_destination; + String status; + String finished_by; + + std::string toJsonString() const + { + Poco::JSON::Object json; + json.set("part_name", part_name); + json.set("path_in_destination", path_in_destination); + json.set("status", status); + json.set("finished_by", finished_by); + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + return oss.str(); + } + + static ExportReplicatedMergeTreePartitionProcessedPartEntry fromJsonString(const std::string & json_string) + { + Poco::JSON::Parser parser; + auto json = parser.parse(json_string).extract(); + chassert(json); + + ExportReplicatedMergeTreePartitionProcessedPartEntry entry; + + entry.part_name = json->getValue("part_name"); + entry.path_in_destination = json->getValue("path_in_destination"); + entry.status = json->getValue("status"); + entry.finished_by = json->getValue("finished_by"); + + return entry; + } +}; + struct ExportReplicatedMergeTreePartitionManifest { String transaction_id; diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp index dcda713f5b39..1df76b100e17 100644 --- a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -169,64 +169,16 @@ void ExportPartitionTaskScheduler::handlePartExportSuccess( { LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} exported successfully", relative_path_in_destination_storage); - Coordination::Stat locked_by_stat; - std::string locked_by; - - if (!zk->tryGet(export_path / "locks" / part_name, locked_by, &locked_by_stat)) - { - LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is not locked by any replica, will not commit or set it as completed", part_name); - return; - } - - /// Is this a good idea? what if the file we just pushed to s3 ends up triggering an exception in the replica that actually locks the part and it does not commit? - /// I guess we should not throw if file already exists for export partition, hard coded. - if (locked_by != storage.replica_name) - { - LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is locked by another replica, will not commit or set it as completed", part_name); - return; - } - - Coordination::Requests requests; - - if (zk->isFeatureEnabled(DB::KeeperFeatureFlag::REMOVE_RECURSIVE)) - { - requests.emplace_back(zkutil::makeRemoveRecursiveRequest(*zk, processing_parts_path / part_name, -1)); - } - else - { - // Remove children before parent (order matters for multi operations) - // Maybe a ls + multi rm.. - requests.emplace_back(zkutil::makeRemoveRequest(processing_parts_path / part_name / "retry_count", -1)); - requests.emplace_back(zkutil::makeRemoveRequest(processing_parts_path / part_name / "status", -1)); - requests.emplace_back(zkutil::makeRemoveRequest(processing_parts_path / part_name, -1)); - } - - requests.emplace_back(zkutil::makeCreateRequest(processed_part_path, "", zkutil::CreateMode::Persistent)); - requests.emplace_back(zkutil::makeCreateRequest(processed_part_path / "path", relative_path_in_destination_storage, zkutil::CreateMode::Persistent)); - requests.emplace_back(zkutil::makeCreateRequest(processed_part_path / "status", "COMPLETED", zkutil::CreateMode::Persistent)); - requests.emplace_back(zkutil::makeCreateRequest(processed_part_path / "finished_by", storage.replica_name, zkutil::CreateMode::Persistent)); - requests.emplace_back(zkutil::makeRemoveRequest(export_path / "locks" / part_name, locked_by_stat.version)); - - Coordination::Responses responses; - if (Coordination::Error::ZOK != zk->tryMulti(requests, responses)) + if (!tryToMovePartToProcessed(export_path, processing_parts_path, processed_part_path, part_name, relative_path_in_destination_storage, zk)) { - /// todo arthur remember what to do here - LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to update export path, skipping"); + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to move part to processed, will not commit export partition"); return; } LOG_INFO(storage.log, "ExportPartition scheduler task: Marked part export {} as completed", part_name); - - Strings parts_in_processing_or_pending; - if (Coordination::Error::ZOK != zk->tryGetChildren(export_path / "processing", parts_in_processing_or_pending)) - { - LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get parts in processing or pending, will not try to commit export partition"); - return; - } - if (!parts_in_processing_or_pending.empty()) + if (!areAllPartsProcessed(export_path, zk)) { - LOG_INFO(storage.log, "ExportPartition scheduler task: There are still parts in processing or pending, will not try to commit export partition"); return; } @@ -262,21 +214,32 @@ void ExportPartitionTaskScheduler::handlePartExportFailure( Coordination::Requests ops; const auto processing_part_path = processing_parts_path / part_name; - std::string retry_count_string; - if (zk->tryGet(processing_part_path / "retry_count", retry_count_string)) + + std::string processing_part_string; + + if (!zk->tryGet(processing_part_path, processing_part_string)) { - std::size_t retry_count = std::stoull(retry_count_string.c_str()) + 1; + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get processing part, will not increment error counts"); + return; + } - ops.emplace_back(zkutil::makeSetRequest(processing_part_path / "retry_count", std::to_string(retry_count), -1)); + /// todo arthur could this have been cached? + auto processing_part_entry = ExportReplicatedMergeTreePartitionProcessingPartEntry::fromJsonString(processing_part_string); + + processing_part_entry.retry_count++; + + if (processing_part_entry.retry_count) + { ops.emplace_back(zkutil::makeRemoveRequest(export_path / "locks" / part_name, locked_by_stat.version)); + ops.emplace_back(zkutil::makeSetRequest(processing_part_path, processing_part_entry.toJsonString(), -1)); - if (retry_count >= max_retries) + if (processing_part_entry.retry_count >= max_retries) { /// just set status in processing_part_path and finished_by - ops.emplace_back(zkutil::makeSetRequest(processing_part_path / "status", "FAILED", -1)); - ops.emplace_back(zkutil::makeCreateRequest(processing_part_path / "finished_by", storage.replica_name, zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeSetRequest(export_path / "status", "FAILED", -1)); + processing_part_entry.status = "FAILED"; + processing_part_entry.finished_by = storage.replica_name; + ops.emplace_back(zkutil::makeSetRequest(export_path / "status", "FAILED", -1)); LOG_INFO(storage.log, "ExportPartition scheduler task: Retry count limit exceeded for part {}, will try to fail the entire task", part_name); } @@ -316,4 +279,73 @@ void ExportPartitionTaskScheduler::handlePartExportFailure( } } +bool ExportPartitionTaskScheduler::tryToMovePartToProcessed( + const std::filesystem::path & export_path, + const std::filesystem::path & processing_parts_path, + const std::filesystem::path & processed_part_path, + const std::string & part_name, + const String & relative_path_in_destination_storage, + const zkutil::ZooKeeperPtr & zk +) +{ + Coordination::Stat locked_by_stat; + std::string locked_by; + + if (!zk->tryGet(export_path / "locks" / part_name, locked_by, &locked_by_stat)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is not locked by any replica, will not commit or set it as completed", part_name); + return false; + } + + /// Is this a good idea? what if the file we just pushed to s3 ends up triggering an exception in the replica that actually locks the part and it does not commit? + /// I guess we should not throw if file already exists for export partition, hard coded. + if (locked_by != storage.replica_name) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is locked by another replica, will not commit or set it as completed", part_name); + return false; + } + + Coordination::Requests requests; + + ExportReplicatedMergeTreePartitionProcessedPartEntry processed_part_entry; + processed_part_entry.part_name = part_name; + processed_part_entry.path_in_destination = relative_path_in_destination_storage; + processed_part_entry.status = "SUCCESS"; + processed_part_entry.finished_by = storage.replica_name; + + requests.emplace_back(zkutil::makeRemoveRequest(processing_parts_path / part_name, -1)); + requests.emplace_back(zkutil::makeCreateRequest(processed_part_path, processed_part_entry.toJsonString(), zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeRemoveRequest(export_path / "locks" / part_name, locked_by_stat.version)); + + Coordination::Responses responses; + if (Coordination::Error::ZOK != zk->tryMulti(requests, responses)) + { + /// todo arthur remember what to do here + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to update export path, skipping"); + return false; + } + + return true; +} + +bool ExportPartitionTaskScheduler::areAllPartsProcessed( + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk) +{ + Strings parts_in_processing_or_pending; + if (Coordination::Error::ZOK != zk->tryGetChildren(export_path / "processing", parts_in_processing_or_pending)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get parts in processing or pending, will not try to commit export partition"); + return false; + } + + if (!parts_in_processing_or_pending.empty()) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: There are still parts in processing or pending, will not try to commit export partition"); + return false; + } + + return true; +} + } diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.h b/src/Storages/MergeTree/ExportPartitionTaskScheduler.h index ee5d03dd5bed..bd420ed7bc1b 100644 --- a/src/Storages/MergeTree/ExportPartitionTaskScheduler.h +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.h @@ -46,6 +46,20 @@ class ExportPartitionTaskScheduler const zkutil::ZooKeeperPtr & zk, const String & exception, size_t max_retries); + + bool tryToMovePartToProcessed( + const std::filesystem::path & export_path, + const std::filesystem::path & processing_parts_path, + const std::filesystem::path & processed_part_path, + const std::string & part_name, + const String & relative_path_in_destination_storage, + const zkutil::ZooKeeperPtr & zk + ); + + bool areAllPartsProcessed( + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk + ); }; } diff --git a/src/Storages/MergeTree/ExportPartitionUtils.cpp b/src/Storages/MergeTree/ExportPartitionUtils.cpp index 6af6ad05e711..c6fa5097266e 100644 --- a/src/Storages/MergeTree/ExportPartitionUtils.cpp +++ b/src/Storages/MergeTree/ExportPartitionUtils.cpp @@ -2,7 +2,6 @@ #include #include #include "Storages/ExportReplicatedMergeTreePartitionManifest.h" -#include "Storages/StorageReplicatedMergeTree.h" #include namespace DB @@ -12,31 +11,50 @@ namespace fs = std::filesystem; namespace ExportPartitionUtils { + /// Collect all the exported paths from the processed parts + /// If multiRead is supported by the keeper implementation, it is done in a single request + /// Otherwise, multiple async requests are sent std::vector getExportedPaths(const LoggerPtr & log, const zkutil::ZooKeeperPtr & zk, const std::string & export_path) { std::vector exported_paths; LOG_INFO(log, "ExportPartition: Getting exported paths for {}", export_path); - std::vector parts_children; - if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(export_path) / "processed", parts_children)) + const auto processed_parts_path = fs::path(export_path) / "processed"; + + std::vector processed_parts; + if (Coordination::Error::ZOK != zk->tryGetChildren(processed_parts_path, processed_parts)) { /// todo arthur do something here LOG_INFO(log, "ExportPartition: Failed to get parts children, exiting"); - return exported_paths; + return {}; } - for (const auto & part_child : parts_children) + std::vector get_paths; + + for (const auto & processed_part : processed_parts) { - std::string path_in_destination_storage; + get_paths.emplace_back(processed_parts_path / processed_part); + } + + auto responses = zk->tryGet(get_paths); - if (zk->tryGet(fs::path(export_path) / "parts" / part_child / "path", path_in_destination_storage)) + responses.waitForResponses(); + + for (size_t i = 0; i < responses.size(); ++i) + { + if (responses[i].error != Coordination::Error::ZOK) { - LOG_INFO(log, "ExportPartition: Failed to get path in destination storage for part {}, skipping", part_child); - continue; + /// todo arthur what to do in this case? + /// It could be that zk is corrupt, in that case we should fail the task + /// but it can also be some temporary network issue? not sure + LOG_INFO(log, "ExportPartition: Failed to get exported path, exiting"); + return {}; } - exported_paths.push_back(path_in_destination_storage); + const auto processed_part_entry = ExportReplicatedMergeTreePartitionProcessedPartEntry::fromJsonString(responses[i].data); + + exported_paths.emplace_back(processed_part_entry.path_in_destination); } return exported_paths; @@ -48,32 +66,29 @@ namespace ExportPartitionUtils const zkutil::ZooKeeperPtr & zk, const LoggerPtr & log, const std::string & entry_path, - const ContextPtr & context - ) + const ContextPtr & context) { const auto exported_paths = ExportPartitionUtils::getExportedPaths(log, zk, entry_path); - - if (exported_paths.size() == manifest.parts.size()) + + if (exported_paths.size() != manifest.parts.size()) { - LOG_INFO(log, "ExportPartition: Exported paths size matches parts size, commit the export"); - destination_storage->commitExportPartitionTransaction(manifest.transaction_id, manifest.partition_id, exported_paths, context); - - LOG_INFO(log, "ExportPartition: Committed export, mark as completed"); - if (Coordination::Error::ZOK == zk->trySet(fs::path(entry_path) / "status", "COMPLETED", -1)) - { - LOG_INFO(log, "ExportPartition: Marked export as completed"); - } - else - { - LOG_INFO(log, "ExportPartition: Failed to mark export as completed, will not try to fix it"); - } + LOG_INFO(log, "ExportPartition: Skipping {}: exported paths size does not match parts size, this is a BUG", entry_path); + return; + } + + LOG_INFO(log, "ExportPartition: Exported paths size matches parts size, commit the export"); + destination_storage->commitExportPartitionTransaction(manifest.transaction_id, manifest.partition_id, exported_paths, context); + + LOG_INFO(log, "ExportPartition: Committed export, mark as completed"); + if (Coordination::Error::ZOK == zk->trySet(fs::path(entry_path) / "status", "COMPLETED", -1)) + { + LOG_INFO(log, "ExportPartition: Marked export as completed"); } else { - LOG_INFO(log, "ExportPartition: Skipping {}: exported paths size does not match parts size, this is a BUG", entry_path); + LOG_INFO(log, "ExportPartition: Failed to mark export as completed, will not try to fix it"); } } - } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d396c4a594c6..4ed3dbcb9eb3 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8176,19 +8176,14 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & for (const auto & part : part_names) { - ops.emplace_back(zkutil::makeCreateRequest( - fs::path(partition_exports_path) / "processing" / part, - "", - zkutil::CreateMode::Persistent)); - - ops.emplace_back(zkutil::makeCreateRequest( - fs::path(partition_exports_path) / "processing" / part / "status", - "PENDING", - zkutil::CreateMode::Persistent)); + ExportReplicatedMergeTreePartitionProcessingPartEntry entry; + entry.status = "PENDING"; + entry.part_name = part; + entry.retry_count = 0; ops.emplace_back(zkutil::makeCreateRequest( - fs::path(partition_exports_path) / "processing" / part / "retry_count", - "0", + fs::path(partition_exports_path) / "processing" / part, + entry.toJsonString(), zkutil::CreateMode::Persistent)); } From de9deb2cf84222f4bd0eab85410f62bf6f92ad96 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 10 Nov 2025 15:30:13 +0100 Subject: [PATCH 38/58] introduce file already exists policy setting --- src/Core/Settings.cpp | 6 + ...portReplicatedMergeTreePartitionManifest.h | 15 +++ src/Storages/MergeTree/ExportList.h | 2 +- src/Storages/MergeTree/ExportPartTask.cpp | 59 +++++++--- .../ExportPartitionTaskScheduler.cpp | 2 + src/Storages/MergeTree/MergeTreeData.cpp | 13 ++- .../MergeTree/MergeTreePartExportManifest.h | 13 ++- src/Storages/StorageReplicatedMergeTree.cpp | 14 +++ .../test.py | 104 +++++++++++++++++- 9 files changed, 204 insertions(+), 24 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 39548489a35e..82bd5748003a 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -6883,6 +6883,12 @@ Maximum number of retries for exporting a merge tree part in an export partition DECLARE(UInt64, export_merge_tree_partition_manifest_ttl, 180, R"( Determines how long the manifest will live in ZooKeeper. It prevents the same partition from being exported twice to the same destination. This setting does not affect / delete in progress tasks. It'll only cleanup the completed ones. +)", 0) \ + DECLARE(String, export_merge_tree_part_file_already_exists_policy, "NO_OP", R"( +Possible values: +- NO_OP - No-op if the file already exists - Default. +- ERROR - Throw an error if the file already exists. +- OVERWRITE - Overwrite the file )", 0) \ \ /* ####################################################### */ \ diff --git a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h index c715a95dd31b..b9e80fcfdec4 100644 --- a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h +++ b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -102,6 +103,7 @@ struct ExportReplicatedMergeTreePartitionManifest size_t max_threads; bool parallel_formatting; bool parquet_parallel_encoding; + MergeTreePartExportManifest::FileAlreadyExistsPolicy file_already_exists_policy; std::string toJsonString() const { @@ -120,6 +122,7 @@ struct ExportReplicatedMergeTreePartitionManifest json.set("parallel_formatting", parallel_formatting); json.set("max_threads", max_threads); json.set("parquet_parallel_encoding", parquet_parallel_encoding); + json.set("file_already_exists_policy", String(magic_enum::enum_name(file_already_exists_policy))); json.set("create_time", create_time); json.set("max_retries", max_retries); json.set("ttl_seconds", ttl_seconds); @@ -152,6 +155,18 @@ struct ExportReplicatedMergeTreePartitionManifest manifest.max_threads = json->getValue("max_threads"); manifest.parallel_formatting = json->getValue("parallel_formatting"); manifest.parquet_parallel_encoding = json->getValue("parquet_parallel_encoding"); + + if (json->has("file_already_exists_policy")) + { + const auto file_already_exists_policy = magic_enum::enum_cast(json->getValue("file_already_exists_policy")); + if (file_already_exists_policy) + { + manifest.file_already_exists_policy = file_already_exists_policy.value(); + } + + /// what to do if it's not a valid value? + } + return manifest; } }; diff --git a/src/Storages/MergeTree/ExportList.h b/src/Storages/MergeTree/ExportList.h index ade18b69480c..3c4daa07737b 100644 --- a/src/Storages/MergeTree/ExportList.h +++ b/src/Storages/MergeTree/ExportList.h @@ -41,7 +41,7 @@ struct ExportsListElement : private boost::noncopyable const StorageID destination_table_id; const UInt64 part_size; const String part_name; - const String destination_file_path; + String destination_file_path; UInt64 rows_read {0}; UInt64 total_rows_to_read {0}; UInt64 total_size_bytes_compressed {0}; diff --git a/src/Storages/MergeTree/ExportPartTask.cpp b/src/Storages/MergeTree/ExportPartTask.cpp index 36c9dfe95072..0fe2680ae848 100644 --- a/src/Storages/MergeTree/ExportPartTask.cpp +++ b/src/Storages/MergeTree/ExportPartTask.cpp @@ -12,6 +12,7 @@ #include #include "Common/Exception.h" #include +#include "Storages/MergeTree/ExportList.h" namespace ProfileEvents { @@ -68,8 +69,19 @@ bool ExportPartTask::executeStep() throw Exception(ErrorCodes::UNKNOWN_TABLE, "Failed to reconstruct destination storage: {}", destination_storage_id_name); } + auto exports_list_entry = storage.getContext()->getExportsList().insert( + getStorageID(), + manifest.destination_storage_id, + manifest.data_part->getBytesOnDisk(), + manifest.data_part->name, + "not_computed_yet", + manifest.data_part->rows_count, + manifest.data_part->getBytesOnDisk(), + manifest.data_part->getBytesUncompressedOnDisk(), + manifest.create_time, + local_context); + SinkToStoragePtr sink; - std::string destination_file_path; try { @@ -81,18 +93,44 @@ bool ExportPartTask::executeStep() sink = destination_storage->import( manifest.data_part->name + "_" + manifest.data_part->checksums.getTotalChecksumHex(), block_with_partition_values, - destination_file_path, - manifest.overwrite_file_if_exists, + (*exports_list_entry)->destination_file_path, + manifest.file_already_exists_policy == MergeTreePartExportManifest::FileAlreadyExistsPolicy::OVERWRITE, context_copy); } catch (const Exception & e) { - tryLogCurrentException(__PRETTY_FUNCTION__); if (e.code() == ErrorCodes::FILE_ALREADY_EXISTS) { ProfileEvents::increment(ProfileEvents::PartsExportDuplicated); + + /// File already exists and the policy is NO_OP, treat it as success. + if (manifest.file_already_exists_policy == MergeTreePartExportManifest::FileAlreadyExistsPolicy::NO_OP) + { + storage.writePartLog( + PartLogElement::Type::EXPORT_PART, + {}, + static_cast((*exports_list_entry)->elapsed * 1000000000), + manifest.data_part->name, + manifest.data_part, + {manifest.data_part}, + nullptr, + nullptr, + exports_list_entry.get()); + + std::lock_guard inner_lock(storage.export_manifests_mutex); + storage.export_manifests.erase(manifest); + + ProfileEvents::increment(ProfileEvents::PartsExports); + ProfileEvents::increment(ProfileEvents::PartsExportTotalMilliseconds, static_cast((*exports_list_entry)->elapsed * 1000)); + + if (manifest.completion_callback) + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createSuccess((*exports_list_entry)->destination_file_path)); + return false; + } } + tryLogCurrentException(__PRETTY_FUNCTION__); + ProfileEvents::increment(ProfileEvents::PartsExportFailures); std::lock_guard inner_lock(storage.export_manifests_mutex); @@ -139,17 +177,6 @@ bool ExportPartTask::executeStep() local_context, getLogger("ExportPartition")); - auto exports_list_entry = storage.getContext()->getExportsList().insert( - getStorageID(), - manifest.destination_storage_id, - manifest.data_part->getBytesOnDisk(), - manifest.data_part->name, - destination_file_path, - manifest.data_part->rows_count, - manifest.data_part->getBytesOnDisk(), - manifest.data_part->getBytesUncompressedOnDisk(), - manifest.create_time, - local_context); ThreadGroupSwitcher switcher((*exports_list_entry)->thread_group, ""); @@ -198,7 +225,7 @@ bool ExportPartTask::executeStep() ProfileEvents::increment(ProfileEvents::PartsExportTotalMilliseconds, static_cast((*exports_list_entry)->elapsed * 1000)); if (manifest.completion_callback) - manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createSuccess(destination_file_path)); + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createSuccess((*exports_list_entry)->destination_file_path)); } catch (const Exception & e) { diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp index 1df76b100e17..df151762c2f4 100644 --- a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -4,6 +4,7 @@ #include #include "Common/ZooKeeper/Types.h" #include "Storages/MergeTree/ExportPartitionUtils.h" +#include "Storages/MergeTree/MergeTreePartExportManifest.h" namespace DB @@ -17,6 +18,7 @@ namespace context_copy->setSetting("output_format_parallel_formatting", manifest.parallel_formatting); context_copy->setSetting("output_format_parquet_parallel_encoding", manifest.parquet_parallel_encoding); context_copy->setSetting("max_threads", manifest.max_threads); + context_copy->setSetting("export_merge_tree_part_file_already_exists_policy", String(magic_enum::enum_name(manifest.file_already_exists_policy))); return context_copy; } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9a7a4561e6e2..6af5661f3e15 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -212,7 +212,7 @@ namespace Setting extern const SettingsBool apply_patch_parts; extern const SettingsBool allow_experimental_export_merge_tree_part; extern const SettingsUInt64 min_bytes_to_use_direct_io; - extern const SettingsBool export_merge_tree_part_overwrite_file_if_exists; + extern const SettingsString export_merge_tree_part_file_already_exists_policy; extern const SettingsBool output_format_parallel_formatting; extern const SettingsBool output_format_parquet_parallel_encoding; } @@ -6253,11 +6253,20 @@ void MergeTreeData::exportPartToTable( part_name, getStorageID().getFullTableName()); { + const auto file_already_exists_policy = magic_enum::enum_cast(Poco::toUpper(query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value)); + if (!file_already_exists_policy) + { + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Invalid value for setting 'export_merge_tree_part_file_already_exists_policy': {}", + query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value); + } + MergeTreePartExportManifest manifest( dest_storage->getStorageID(), part, transaction_id, - query_context->getSettingsRef()[Setting::export_merge_tree_part_overwrite_file_if_exists], + *file_already_exists_policy, query_context->getSettingsRef()[Setting::output_format_parallel_formatting], query_context->getSettingsRef()[Setting::output_format_parquet_parallel_encoding], query_context->getSettingsRef()[Setting::max_threads], diff --git a/src/Storages/MergeTree/MergeTreePartExportManifest.h b/src/Storages/MergeTree/MergeTreePartExportManifest.h index fa7be1c2b863..77a4446c270e 100644 --- a/src/Storages/MergeTree/MergeTreePartExportManifest.h +++ b/src/Storages/MergeTree/MergeTreePartExportManifest.h @@ -11,6 +11,13 @@ class ExportPartTask; struct MergeTreePartExportManifest { + enum class FileAlreadyExistsPolicy + { + NO_OP, + ERROR, + OVERWRITE, + }; + using DataPartPtr = std::shared_ptr; struct CompletionCallbackResult @@ -39,7 +46,7 @@ struct MergeTreePartExportManifest const StorageID & destination_storage_id_, const DataPartPtr & data_part_, const String & query_id_, - bool overwrite_file_if_exists_, + FileAlreadyExistsPolicy file_already_exists_policy_, bool parallel_formatting_, bool parquet_parallel_encoding_, std::size_t max_threads_, @@ -47,7 +54,7 @@ struct MergeTreePartExportManifest : destination_storage_id(destination_storage_id_), data_part(data_part_), query_id(query_id_), - overwrite_file_if_exists(overwrite_file_if_exists_), + file_already_exists_policy(file_already_exists_policy_), parallel_formatting(parallel_formatting_), parquet_parallel_encoding(parquet_parallel_encoding_), max_threads(max_threads_), @@ -58,7 +65,7 @@ struct MergeTreePartExportManifest DataPartPtr data_part; /// Used for killing the export. String query_id; - bool overwrite_file_if_exists; + FileAlreadyExistsPolicy file_already_exists_policy; bool parallel_formatting; /// parquet has a different setting for parallel formatting bool parquet_parallel_encoding; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 4ed3dbcb9eb3..9b1bc7bde4c3 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -198,6 +198,7 @@ namespace Setting extern const SettingsBool output_format_parallel_formatting; extern const SettingsBool output_format_parquet_parallel_encoding; extern const SettingsMaxThreads max_threads; + extern const SettingsString export_merge_tree_part_file_already_exists_policy; } namespace MergeTreeSetting @@ -300,6 +301,7 @@ namespace ErrorCodes extern const int FAULT_INJECTED; extern const int CANNOT_FORGET_PARTITION; extern const int TIMEOUT_EXCEEDED; + extern const int INVALID_SETTING_VALUE; } namespace ActionLocks @@ -8159,6 +8161,18 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & manifest.parallel_formatting = query_context->getSettingsRef()[Setting::output_format_parallel_formatting]; manifest.parquet_parallel_encoding = query_context->getSettingsRef()[Setting::output_format_parquet_parallel_encoding]; + const auto file_already_exists_policy = magic_enum::enum_cast(query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value); + + if (!file_already_exists_policy) + { + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Invalid value for setting 'export_merge_tree_part_file_already_exists_policy': {}", + query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value); + } + + manifest.file_already_exists_policy = file_already_exists_policy.value(); + ops.emplace_back(zkutil::makeCreateRequest( fs::path(partition_exports_path) / "metadata.json", manifest.toJsonString(), diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index 97056dd1dfd3..e4b4435443f8 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -4,6 +4,7 @@ import string import time from typing import Optional +import uuid from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager @@ -231,7 +232,7 @@ def test_drop_source_table_during_export(cluster): node.query(f"DROP TABLE {mt_table}") # Sleep some time to let the export finish (assuming it was not properly cancelled) - time.sleep(5) + time.sleep(10) assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_*', format=LineAsString)") == '0\n', "Background operations completed even with the table dropped" @@ -284,7 +285,7 @@ def test_drop_destination_table_during_export(cluster): node.query(f"DROP TABLE {s3_table}") # give some time for the export to finish - time.sleep(5) + time.sleep(10) # not sure this is the expected behavior, but adding until we make a decision assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_*', format=LineAsString)") != '0\n', "Background operations did not complete after dropping the destination table" @@ -555,6 +556,105 @@ def test_export_ttl(cluster): assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '1\n', "Export did not succeed" +# export an individual part with alter table export part +# and then try to export the partition. It should not fail because export partition is idempotent. +def test_export_part_and_partition(cluster): + node = cluster.instances["replica1"] + + mt_table = "export_part_and_partition_mt_table" + s3_table = "export_part_and_partition_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # stop merges so part names remain stable. it is important for the test. + node.query(f"SYSTEM STOP MERGES {mt_table}") + + query_id_1 = uuid.uuid4().hex + query_id_2 = uuid.uuid4().hex + query_id_3 = uuid.uuid4().hex + query_id_4 = uuid.uuid4().hex + + # Export all parts + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1", + query_id=query_id_1, + ) + + # check system.replicated_partition_exports for the export + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + AND transaction_id = '{query_id_1}' + """ + ) == "COMPLETED\n", "Export should be marked as COMPLETED" + + # wait for the exports to finish + time.sleep(3) + + # try to export the partition + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_force_export=1", + query_id=query_id_2, + ) + + time.sleep(3) + + # check system.replicated_partition_exports for the export + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + AND transaction_id = '{query_id_2}' + """ + ) == "COMPLETED\n", "Export should be marked as COMPLETED" + + # now let's try with a file exists policy that is not NO_OP + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_force_export=1, export_merge_tree_part_file_already_exists_policy='OVERWRITE'", + query_id=query_id_3, + ) + + # wait for the export to finish + time.sleep(3) + + # check system.replicated_partition_exports for the export + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + AND transaction_id = '{query_id_3}' + """ + ) == "COMPLETED\n", "Export should be marked as COMPLETED" + + # last but not least, let's try with the error policy + # max retries = 1 so it fails fast + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_force_export=1, export_merge_tree_part_file_already_exists_policy='ERROR', export_merge_tree_partition_max_retries=1", + query_id=query_id_4, + ) + + # wait for the export to finish + time.sleep(3) + + # check system.replicated_partition_exports for the export + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + AND transaction_id = '{query_id_4}' + """ + ) == "FAILED\n", "Export should be marked as FAILED" + + # def test_source_mutations_during_export_snapshot(cluster): # node = cluster.instances["replica1"] From cf13ec26228f5e939164560a164c2bff71027374 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 10 Nov 2025 17:39:19 +0100 Subject: [PATCH 39/58] use enum for statuses, remove status from completed part export, try to make tests a bit more stable, cache status --- src/Core/SettingsChangesHistory.cpp | 1 + ...portReplicatedMergeTreePartitionManifest.h | 17 ++++++---- ...ortReplicatedMergeTreePartitionTaskEntry.h | 10 ++++++ .../ExportPartitionManifestUpdatingTask.cpp | 2 +- .../ExportPartitionTaskScheduler.cpp | 31 ++++++++++++++----- .../MergeTree/ExportPartitionUtils.cpp | 3 +- .../tests/gtest_export_partition_ordering.cpp | 6 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- .../test.py | 11 +++---- 9 files changed, 57 insertions(+), 26 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 38e399f7e2ae..992827d97f6a 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -50,6 +50,7 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"export_merge_tree_partition_force_export", false, false, "New setting."}, {"export_merge_tree_partition_max_retries", 3, 3, "New setting."}, {"export_merge_tree_partition_manifest_ttl", 180, 180, "New setting."}, + {"export_merge_tree_part_file_already_exists_policy", "NO_OP", "NO_OP", "New setting."}, }); addSettingsChanges(settings_changes_history, "25.8", { diff --git a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h index b9e80fcfdec4..81f61b5b9f12 100644 --- a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h +++ b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h @@ -12,8 +12,16 @@ namespace DB struct ExportReplicatedMergeTreePartitionProcessingPartEntry { + + enum class Status + { + PENDING, + COMPLETED, + FAILED + }; + String part_name; - String status; + Status status; size_t retry_count; String finished_by; @@ -22,7 +30,7 @@ struct ExportReplicatedMergeTreePartitionProcessingPartEntry Poco::JSON::Object json; json.set("part_name", part_name); - json.set("status", status); + json.set("status", String(magic_enum::enum_name(status))); json.set("retry_count", retry_count); json.set("finished_by", finished_by); std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM @@ -41,7 +49,7 @@ struct ExportReplicatedMergeTreePartitionProcessingPartEntry ExportReplicatedMergeTreePartitionProcessingPartEntry entry; entry.part_name = json->getValue("part_name"); - entry.status = json->getValue("status"); + entry.status = magic_enum::enum_cast(json->getValue("status")).value(); entry.retry_count = json->getValue("retry_count"); if (json->has("finished_by")) { @@ -55,7 +63,6 @@ struct ExportReplicatedMergeTreePartitionProcessedPartEntry { String part_name; String path_in_destination; - String status; String finished_by; std::string toJsonString() const @@ -63,7 +70,6 @@ struct ExportReplicatedMergeTreePartitionProcessedPartEntry Poco::JSON::Object json; json.set("part_name", part_name); json.set("path_in_destination", path_in_destination); - json.set("status", status); json.set("finished_by", finished_by); std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); @@ -81,7 +87,6 @@ struct ExportReplicatedMergeTreePartitionProcessedPartEntry entry.part_name = json->getValue("part_name"); entry.path_in_destination = json->getValue("path_in_destination"); - entry.status = json->getValue("status"); entry.finished_by = json->getValue("finished_by"); return entry; diff --git a/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h index 07aad0aacb8a..6c5a01b92ebd 100644 --- a/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h +++ b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h @@ -15,6 +15,16 @@ struct ExportReplicatedMergeTreePartitionTaskEntry using DataPartPtr = std::shared_ptr; ExportReplicatedMergeTreePartitionManifest manifest; + enum class Status + { + PENDING, + COMPLETED, + FAILED + }; + + /// Allows us to skip completed / failed entries during scheduling + mutable Status status; + /// References to the parts that should be exported /// This is used to prevent the parts from being deleted before finishing the export operation /// It does not mean this replica will export all the parts diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp index e932fd77fd25..9ba0243f4ddc 100644 --- a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp @@ -265,7 +265,7 @@ void ExportPartitionManifestUpdatingTask::addTask( } /// Insert or update entry. The multi_index container automatically maintains both indexes. - auto entry = ExportReplicatedMergeTreePartitionTaskEntry {metadata, std::move(part_references)}; + auto entry = ExportReplicatedMergeTreePartitionTaskEntry {metadata, ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING, std::move(part_references)}; auto it = entries_by_key.find(key); if (it != entries_by_key.end()) entries_by_key.replace(it, entry); diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp index df151762c2f4..7cfdf380a0f3 100644 --- a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -35,13 +35,20 @@ void ExportPartitionTaskScheduler::run() auto zk = storage.getZooKeeper(); // Iterate sorted by create_time - for (const auto & entry : storage.export_merge_tree_partition_task_entries_by_create_time) + for (auto & entry : storage.export_merge_tree_partition_task_entries_by_create_time) { const auto & manifest = entry.manifest; const auto key = entry.getCompositeKey(); const auto & database = storage.getContext()->resolveDatabase(manifest.destination_database); const auto & table = manifest.destination_table; + /// No need to query zk for status if the local one is not PENDING + if (entry.status != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Skipping... Local status is {}", magic_enum::enum_name(entry.status).data()); + continue; + } + const auto destination_storage_id = StorageID(QualifiedTableName {database, table}); const auto destination_storage = DatabaseCatalog::instance().tryGetTable(destination_storage_id, storage.getContext()); @@ -52,16 +59,25 @@ void ExportPartitionTaskScheduler::run() continue; } - std::string status; - if (!zk->tryGet(fs::path(storage.zookeeper_path) / "exports" / key / "status", status)) + std::string status_in_zk_string; + if (!zk->tryGet(fs::path(storage.zookeeper_path) / "exports" / key / "status", status_in_zk_string)) { LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get status, skipping"); continue; } - if (status != "PENDING") + const auto status_in_zk = magic_enum::enum_cast(status_in_zk_string); + + if (!status_in_zk) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get status from zk, skipping"); + continue; + } + + if (status_in_zk.value() != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) { - LOG_INFO(storage.log, "ExportPartition scheduler task: Skipping... Status is not PENDING"); + entry.status = status_in_zk.value(); + LOG_INFO(storage.log, "ExportPartition scheduler task: Skipping... Status from zk is {}", entry.status); continue; } @@ -238,10 +254,10 @@ void ExportPartitionTaskScheduler::handlePartExportFailure( if (processing_part_entry.retry_count >= max_retries) { /// just set status in processing_part_path and finished_by - processing_part_entry.status = "FAILED"; + processing_part_entry.status = ExportReplicatedMergeTreePartitionProcessingPartEntry::Status::FAILED; processing_part_entry.finished_by = storage.replica_name; - ops.emplace_back(zkutil::makeSetRequest(export_path / "status", "FAILED", -1)); + ops.emplace_back(zkutil::makeSetRequest(export_path / "status", String(magic_enum::enum_name(ExportReplicatedMergeTreePartitionTaskEntry::Status::FAILED)).data(), -1)); LOG_INFO(storage.log, "ExportPartition scheduler task: Retry count limit exceeded for part {}, will try to fail the entire task", part_name); } @@ -312,7 +328,6 @@ bool ExportPartitionTaskScheduler::tryToMovePartToProcessed( ExportReplicatedMergeTreePartitionProcessedPartEntry processed_part_entry; processed_part_entry.part_name = part_name; processed_part_entry.path_in_destination = relative_path_in_destination_storage; - processed_part_entry.status = "SUCCESS"; processed_part_entry.finished_by = storage.replica_name; requests.emplace_back(zkutil::makeRemoveRequest(processing_parts_path / part_name, -1)); diff --git a/src/Storages/MergeTree/ExportPartitionUtils.cpp b/src/Storages/MergeTree/ExportPartitionUtils.cpp index c6fa5097266e..466eb79e8367 100644 --- a/src/Storages/MergeTree/ExportPartitionUtils.cpp +++ b/src/Storages/MergeTree/ExportPartitionUtils.cpp @@ -2,6 +2,7 @@ #include #include #include "Storages/ExportReplicatedMergeTreePartitionManifest.h" +#include "Storages/ExportReplicatedMergeTreePartitionTaskEntry.h" #include namespace DB @@ -80,7 +81,7 @@ namespace ExportPartitionUtils destination_storage->commitExportPartitionTransaction(manifest.transaction_id, manifest.partition_id, exported_paths, context); LOG_INFO(log, "ExportPartition: Committed export, mark as completed"); - if (Coordination::Error::ZOK == zk->trySet(fs::path(entry_path) / "status", "COMPLETED", -1)) + if (Coordination::Error::ZOK == zk->trySet(fs::path(entry_path) / "status", String(magic_enum::enum_name(ExportReplicatedMergeTreePartitionTaskEntry::Status::COMPLETED)).data(), -1)) { LOG_INFO(log, "ExportPartition: Marked export as completed"); } diff --git a/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp b/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp index 480f76db4d99..a3b74a2f5e6e 100644 --- a/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp +++ b/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp @@ -40,9 +40,9 @@ TEST_F(ExportPartitionOrderingTest, IterationOrderMatchesCreateTime) manifest3.destination_table = "table1"; manifest3.create_time = base_time; // Oldest - ExportReplicatedMergeTreePartitionTaskEntry entry1{manifest1, {}}; - ExportReplicatedMergeTreePartitionTaskEntry entry2{manifest2, {}}; - ExportReplicatedMergeTreePartitionTaskEntry entry3{manifest3, {}}; + ExportReplicatedMergeTreePartitionTaskEntry entry1{manifest1, ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING, {}}; + ExportReplicatedMergeTreePartitionTaskEntry entry2{manifest2, ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING, {}}; + ExportReplicatedMergeTreePartitionTaskEntry entry3{manifest3, ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING, {}}; // Insert in reverse order by_key.insert(entry1); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 9b1bc7bde4c3..7f7e3d61314b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8191,7 +8191,7 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & for (const auto & part : part_names) { ExportReplicatedMergeTreePartitionProcessingPartEntry entry; - entry.status = "PENDING"; + entry.status = ExportReplicatedMergeTreePartitionProcessingPartEntry::Status::PENDING; entry.part_name = part; entry.retry_count = 0; diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index e4b4435443f8..f3ebf3c83b36 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -553,16 +553,15 @@ def test_export_ttl(cluster): time.sleep(expiration_time) # assert that the export succeeded, check the commit file - assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '1\n', "Export did not succeed" + # there should be two commit files now, one for the first export and one for the second export + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '2\n', "Export did not succeed" -# export an individual part with alter table export part -# and then try to export the partition. It should not fail because export partition is idempotent. -def test_export_part_and_partition(cluster): +def test_export_partition_file_already_exists_policy(cluster): node = cluster.instances["replica1"] - mt_table = "export_part_and_partition_mt_table" - s3_table = "export_part_and_partition_s3_table" + mt_table = "export_partition_file_already_exists_policy_mt_table" + s3_table = "export_partition_file_already_exists_policy_s3_table" create_tables_and_insert_data(node, mt_table, s3_table, "replica1") From 0c0b85fcb34c2b673fe8a144ab7f67072116078a Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 12 Nov 2025 21:06:39 +0100 Subject: [PATCH 40/58] refactor kill so that it sets status instead of removing entry, adds some logs --- ...ortReplicatedMergeTreePartitionTaskEntry.h | 13 +- src/Storages/MergeTree/ExportPartTask.cpp | 15 ++- src/Storages/MergeTree/ExportPartTask.h | 3 + .../ExportPartitionManifestUpdatingTask.cpp | 72 ++++++++++- .../ExportPartitionManifestUpdatingTask.h | 11 +- .../ReplicatedMergeTreeRestartingThread.cpp | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 119 +++++++++++++----- src/Storages/StorageReplicatedMergeTree.h | 7 ++ .../test.py | 72 +---------- 9 files changed, 211 insertions(+), 102 deletions(-) diff --git a/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h index 6c5a01b92ebd..76674bfc4a92 100644 --- a/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h +++ b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h @@ -19,7 +19,8 @@ struct ExportReplicatedMergeTreePartitionTaskEntry { PENDING, COMPLETED, - FAILED + FAILED, + KILLED }; /// Allows us to skip completed / failed entries during scheduling @@ -37,6 +38,11 @@ struct ExportReplicatedMergeTreePartitionTaskEntry return manifest.partition_id + "_" + qualified_table_name.getFullName(); } + std::string getTransactionId() const + { + return manifest.transaction_id; + } + /// Get create_time for sorted iteration time_t getCreateTime() const { @@ -46,6 +52,7 @@ struct ExportReplicatedMergeTreePartitionTaskEntry struct ExportPartitionTaskEntryTagByCompositeKey {}; struct ExportPartitionTaskEntryTagByCreateTime {}; +struct ExportPartitionTaskEntryTagByTransactionId {}; // Multi-index container for export partition task entries // - Index 0 (TagByCompositeKey): hashed_unique on composite key for O(1) lookup @@ -60,6 +67,10 @@ using ExportPartitionTaskEntriesContainer = boost::multi_index_container< boost::multi_index::ordered_non_unique< boost::multi_index::tag, boost::multi_index::const_mem_fun + >, + boost::multi_index::hashed_unique< + boost::multi_index::tag, + boost::multi_index::const_mem_fun > > >; diff --git a/src/Storages/MergeTree/ExportPartTask.cpp b/src/Storages/MergeTree/ExportPartTask.cpp index 0fe2680ae848..870c9d9373cb 100644 --- a/src/Storages/MergeTree/ExportPartTask.cpp +++ b/src/Storages/MergeTree/ExportPartTask.cpp @@ -30,6 +30,7 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; extern const int FILE_ALREADY_EXISTS; extern const int LOGICAL_ERROR; + extern const int QUERY_WAS_CANCELLED; } namespace Setting @@ -201,12 +202,18 @@ bool ExportPartTask::executeStep() auto is_cancelled_callback = [this]() { - return storage.parts_mover.moves_blocker.isCancelled(); + return isCancelled(); }; exec.setCancelCallback(is_cancelled_callback, 100); exec.execute(); + + if (isCancelled()) + { + throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Export part was cancelled"); + } + std::lock_guard inner_lock(storage.export_manifests_mutex); storage.writePartLog( PartLogElement::Type::EXPORT_PART, @@ -257,9 +264,15 @@ bool ExportPartTask::executeStep() void ExportPartTask::cancel() noexcept { + cancel_requested.store(true); pipeline.cancel(); } +bool ExportPartTask::isCancelled() const +{ + return cancel_requested.load() || storage.parts_mover.moves_blocker.isCancelled(); +} + void ExportPartTask::onCompleted() { } diff --git a/src/Storages/MergeTree/ExportPartTask.h b/src/Storages/MergeTree/ExportPartTask.h index 790ef9f6ecba..bcec68b2b737 100644 --- a/src/Storages/MergeTree/ExportPartTask.h +++ b/src/Storages/MergeTree/ExportPartTask.h @@ -27,6 +27,9 @@ class ExportPartTask : public IExecutableTask MergeTreePartExportManifest manifest; ContextPtr local_context; QueryPipeline pipeline; + std::atomic cancel_requested = false; + + bool isCancelled() const; }; } diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp index 9ba0243f4ddc..b933a7c5ef9d 100644 --- a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp @@ -2,6 +2,7 @@ #include #include #include "Storages/MergeTree/ExportPartitionUtils.h" +#include "Common/logger_useful.h" #include #include #include @@ -154,10 +155,12 @@ ExportPartitionManifestUpdatingTask::ExportPartitionManifestUpdatingTask(Storage { } -void ExportPartitionManifestUpdatingTask::run() +void ExportPartitionManifestUpdatingTask::poll() { std::lock_guard lock(storage.export_merge_tree_partition_mutex); + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Polling for new entries for table {}. Current number of entries: {}", storage.getStorageID().getNameForLogs(), storage.export_merge_tree_partition_task_entries_by_key.size()); + auto zk = storage.getZooKeeper(); const std::string exports_path = fs::path(storage.zookeeper_path) / "exports"; @@ -200,8 +203,13 @@ void ExportPartitionManifestUpdatingTask::run() if (!cleanup_lock.is_locked && has_local_entry_and_is_up_to_date) continue; + auto status_watch_callback = std::make_shared([this, key](const Coordination::WatchResponse &) { + storage.export_merge_tree_partition_manifest_updater->addStatusChange(key); + storage.export_merge_tree_partition_status_handling_task->schedule(); + }); + std::string status; - if (!zk->tryGet(fs::path(entry_path) / "status", status)) + if (!zk->tryGetWatch(fs::path(entry_path) / "status", status, nullptr, status_watch_callback)) { LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: missing status", key); continue; @@ -245,6 +253,8 @@ void ExportPartitionManifestUpdatingTask::run() /// Remove entries that were deleted by someone else removeStaleEntries(zk_children, entries_by_key); + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: finished polling for new entries. Number of entries: {}", entries_by_key.size()); + storage.export_merge_tree_partition_select_task->schedule(); } @@ -303,4 +313,62 @@ void ExportPartitionManifestUpdatingTask::removeStaleEntries( } } +void ExportPartitionManifestUpdatingTask::addStatusChange(const std::string & key) +{ + std::lock_guard lock(status_changes_mutex); + status_changes.emplace(key); +} + +void ExportPartitionManifestUpdatingTask::handleStatusChanges() +{ + std::lock_guard lock(status_changes_mutex); + std::lock_guard task_entries_lock(storage.export_merge_tree_partition_mutex); + auto zk = storage.getZooKeeper(); + + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: handling status changes. Number of status changes: {}", status_changes.size()); + + while (!status_changes.empty()) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: handling status change for task {}", status_changes.front()); + const auto key = status_changes.front(); + status_changes.pop(); + + auto it = storage.export_merge_tree_partition_task_entries_by_key.find(key); + if (it == storage.export_merge_tree_partition_task_entries_by_key.end()) + continue; + + /// get new status from zk + std::string new_status_string; + if (!zk->tryGet(fs::path(storage.zookeeper_path) / "exports" / key / "status", new_status_string)) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Failed to get new status for task {}, skipping", key); + continue; + } + + const auto new_status = magic_enum::enum_cast(new_status_string); + if (!new_status) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Invalid status {} for task {}, skipping", new_status_string, key); + continue; + } + + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: status changed for task {}. New status: {}", key, magic_enum::enum_name(*new_status).data()); + + /// If status changed to KILLED, cancel local export operations + if (*new_status == ExportReplicatedMergeTreePartitionTaskEntry::Status::KILLED) + { + try + { + storage.killExportPart(it->manifest.transaction_id); + } + catch (...) + { + tryLogCurrentException(storage.log, __PRETTY_FUNCTION__); + } + } + + it->status = *new_status; + } +} + } diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h index fa694299fc8a..ea52f679d654 100644 --- a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h @@ -1,5 +1,7 @@ #pragma once +#include +#include #include #include namespace DB @@ -13,7 +15,11 @@ class ExportPartitionManifestUpdatingTask public: ExportPartitionManifestUpdatingTask(StorageReplicatedMergeTree & storage); - void run(); + void poll(); + + void handleStatusChanges(); + + void addStatusChange(const std::string & key); private: StorageReplicatedMergeTree & storage; @@ -28,6 +34,9 @@ class ExportPartitionManifestUpdatingTask const std::unordered_set & zk_children, auto & entries_by_key ); + + std::mutex status_changes_mutex; + std::queue status_changes; }; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index c7b183ea0895..802afa69ea44 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -176,6 +176,7 @@ bool ReplicatedMergeTreeRestartingThread::runImpl() storage.part_check_thread.start(); storage.export_merge_tree_partition_updating_task->activateAndSchedule(); storage.export_merge_tree_partition_select_task->activateAndSchedule(); + storage.export_merge_tree_partition_status_handling_task->activateAndSchedule(); LOG_DEBUG(log, "Table started successfully"); return true; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7f7e3d61314b..d67d104194d4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -432,6 +432,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , queue(*this, merge_strategy_picker) , fetcher(*this) , export_merge_tree_partition_task_entries_by_key(export_merge_tree_partition_task_entries.get()) + , export_merge_tree_partition_task_entries_by_transaction_id(export_merge_tree_partition_task_entries.get()) , export_merge_tree_partition_task_entries_by_create_time(export_merge_tree_partition_task_entries.get()) , cleanup_thread(*this) , async_block_ids_cache(*this) @@ -488,6 +489,11 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( export_merge_tree_partition_updating_task->deactivate(); + export_merge_tree_partition_status_handling_task = getContext()->getSchedulePool().createTask( + getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_status_handling_task)", [this] { exportMergeTreePartitionStatusHandlingTask(); }); + + export_merge_tree_partition_status_handling_task->deactivate(); + export_merge_tree_partition_watch_callback = std::make_shared(export_merge_tree_partition_updating_task->getWatchCallback()); export_merge_tree_partition_select_task = getContext()->getSchedulePool().createTask( @@ -4395,7 +4401,7 @@ void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() { try { - export_merge_tree_partition_manifest_updater->run(); + export_merge_tree_partition_manifest_updater->poll(); } catch (...) { @@ -4420,6 +4426,18 @@ void StorageReplicatedMergeTree::selectPartsToExport() export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); } +void StorageReplicatedMergeTree::exportMergeTreePartitionStatusHandlingTask() +{ + try + { + export_merge_tree_partition_manifest_updater->handleStatusChanges(); + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } +} + std::vector StorageReplicatedMergeTree::getPartitionExportsInfo() const { std::vector infos; @@ -5901,6 +5919,7 @@ void StorageReplicatedMergeTree::partialShutdown() mutations_finalizing_task->deactivate(); export_merge_tree_partition_updating_task->deactivate(); export_merge_tree_partition_select_task->deactivate(); + export_merge_tree_partition_status_handling_task->deactivate(); cleanup_thread.stop(); async_block_ids_cache.stop(); @@ -9633,45 +9652,85 @@ CancellationCode StorageReplicatedMergeTree::killPartMoveToShard(const UUID & ta CancellationCode StorageReplicatedMergeTree::killExportPartition(const String & transaction_id) { - auto zk = getZooKeeper(); - const auto exports_path = fs::path(zookeeper_path) / "exports"; - - const auto export_keys = zk->getChildren(exports_path); - String export_key_to_be_cancelled; - for (const auto & export_key : export_keys) + auto try_set_status_to_killed = [this](const zkutil::ZooKeeperPtr & zk, const std::string & status_path) { - std::string status; - if (!zk->tryGet(fs::path(exports_path) / export_key / "status", status)) - continue; - if (status != "PENDING") - continue; + Coordination::Stat stat; + std::string status_from_zk_string; - std::string metadata_json; - if (!zk->tryGet(fs::path(exports_path) / export_key / "metadata.json", metadata_json)) - continue; - const auto manifest = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); - if (manifest.transaction_id == transaction_id) + if (!zk->tryGet(status_path, status_from_zk_string, &stat)) { - export_key_to_be_cancelled = export_key; - break; + /// found entry locally, but not in zk. It might have been deleted by another replica and we did not have time to update the local entry. + LOG_INFO(log, "Export partition task not found in zk, can not cancel it"); + return CancellationCode::CancelCannotBeSent; } - } - if (export_key_to_be_cancelled.empty()) - return CancellationCode::NotFound; + const auto status_from_zk = magic_enum::enum_cast(status_from_zk_string); + + if (!status_from_zk) + { + LOG_INFO(log, "Export partition task status is invalid, can not cancel it"); + return CancellationCode::CancelCannotBeSent; + } + + if (status_from_zk.value() != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + LOG_INFO(log, "Export partition task is {}, can not cancel it", String(magic_enum::enum_name(status_from_zk.value()))); + return CancellationCode::CancelCannotBeSent; + } + + if (zk->trySet(status_path, String(magic_enum::enum_name(ExportReplicatedMergeTreePartitionTaskEntry::Status::KILLED)), stat.version) != Coordination::Error::ZOK) + { + LOG_INFO(log, "Status has been updated while trying to kill the export partition task, can not cancel it"); + return CancellationCode::CancelCannotBeSent; + } - try - { - /// Once the entry is removed from zk, the update task will be triggered in all replicas - /// The logic for cancelling individual part exports will be triggered in the update task. - zk->removeRecursive(fs::path(exports_path) / export_key_to_be_cancelled); return CancellationCode::CancelSent; + }; + + std::lock_guard lock(export_merge_tree_partition_mutex); + + const auto zk = getZooKeeper(); + + /// if we have the entry locally, no need to list from zk. we can save some requests. + const auto & entry = export_merge_tree_partition_task_entries_by_transaction_id.find(transaction_id); + if (entry != export_merge_tree_partition_task_entries_by_transaction_id.end()) + { + LOG_INFO(log, "Export partition task found locally, trying to cancel it"); + /// found locally, no need to get children on zk + if (entry->status != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + LOG_INFO(log, "Export partition task is not pending, can not cancel it"); + return CancellationCode::CancelCannotBeSent; + } + + return try_set_status_to_killed(zk, fs::path(zookeeper_path) / "exports" / entry->getCompositeKey() / "status"); } - catch (...) + else { - tryLogCurrentException(log, __PRETTY_FUNCTION__); - return CancellationCode::CancelCannotBeSent; + LOG_INFO(log, "Export partition task not found locally, trying to find it on zk"); + /// for some reason, we don't have the entry locally. ls on zk to find the entry + const auto exports_path = fs::path(zookeeper_path) / "exports"; + + const auto export_keys = zk->getChildren(exports_path); + String export_key_to_be_cancelled; + + for (const auto & export_key : export_keys) + { + std::string metadata_json; + if (!zk->tryGet(fs::path(exports_path) / export_key / "metadata.json", metadata_json)) + continue; + const auto manifest = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + if (manifest.transaction_id == transaction_id) + { + LOG_INFO(log, "Export partition task found on zk, trying to cancel it"); + return try_set_status_to_killed(zk, fs::path(exports_path) / export_key / "status"); + } + } } + + LOG_INFO(log, "Export partition task not found, can not cancel it"); + + return CancellationCode::NotFound; } void StorageReplicatedMergeTree::getCommitPartOps( diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index ab7a49398ace..712ba0ba4183 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -521,6 +521,9 @@ class StorageReplicatedMergeTree final : public MergeTreeData BackgroundSchedulePoolTaskHolder mutations_finalizing_task; BackgroundSchedulePoolTaskHolder export_merge_tree_partition_updating_task; + + /// mostly handle kill operations + BackgroundSchedulePoolTaskHolder export_merge_tree_partition_status_handling_task; std::shared_ptr export_merge_tree_partition_manifest_updater; std::shared_ptr export_merge_tree_partition_task_scheduler; @@ -535,6 +538,7 @@ class StorageReplicatedMergeTree final : public MergeTreeData // Convenience references to indexes ExportPartitionTaskEntriesContainer::index::type & export_merge_tree_partition_task_entries_by_key; + ExportPartitionTaskEntriesContainer::index::type & export_merge_tree_partition_task_entries_by_transaction_id; ExportPartitionTaskEntriesContainer::index::type & export_merge_tree_partition_task_entries_by_create_time; /// A thread that removes old parts, log entries, and blocks. ReplicatedMergeTreeCleanupThread cleanup_thread; @@ -768,6 +772,9 @@ class StorageReplicatedMergeTree final : public MergeTreeData /// update in-memory list of partition exports void exportMergeTreePartitionUpdatingTask(); + /// handle status changes for export partition tasks + void exportMergeTreePartitionStatusHandlingTask(); + /** Write the selected parts to merge into the log, * Call when merge_selecting_mutex is locked. * Returns false if any part is not in ZK. diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index f3ebf3c83b36..d5c248e9f97c 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -173,7 +173,7 @@ def test_kill_export(cluster): # Kill only 2020 while S3 is blocked - retry mechanism keeps exports alive # ZooKeeper operations (KILL) proceed quickly since only S3 is blocked - node.query(f"KILL EXPORT PARTITION WHERE partition_id = '2020'") + node.query(f"KILL EXPORT PARTITION WHERE partition_id = '2020' and source_table = '{mt_table}' and destination_table = '{s3_table}'") # wait for 2021 to finish time.sleep(5) @@ -182,6 +182,10 @@ def test_kill_export(cluster): assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '0\n', "Partition 2020 was written to S3, it was not killed as expected" assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2021_*', format=LineAsString)") != f'0\n', "Partition 2021 was not written to S3, but it should have been" + # check system.replicated_partition_exports for the export, status should be KILLED + assert node.query(f"SELECT status FROM system.replicated_partition_exports WHERE partition_id = '2020' and source_table = '{mt_table}' and destination_table = '{s3_table}'") == 'KILLED\n', "Partition 2020 was not killed as expected" + assert node.query(f"SELECT status FROM system.replicated_partition_exports WHERE partition_id = '2021' and source_table = '{mt_table}' and destination_table = '{s3_table}'") == 'COMPLETED\n', "Partition 2021 was not completed, this is unexpected" + def test_drop_source_table_during_export(cluster): node = cluster.instances["replica1"] @@ -291,72 +295,6 @@ def test_drop_destination_table_during_export(cluster): assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_*', format=LineAsString)") != '0\n', "Background operations did not complete after dropping the destination table" -def test_kill_export_by_table(cluster): - node = cluster.instances["replica1"] - - mt_table = "kill_granularity_by_table_mt" - s3_table = "kill_granularity_by_table_s3" - alt_mt_table = "kill_granularity_by_table_alt_mt" - alt_s3_table = "kill_granularity_by_table_alt_s3" - - create_tables_and_insert_data(node, mt_table, s3_table, "replica1") - create_tables_and_insert_data(node, alt_mt_table, alt_s3_table, "replica1") - - # Block S3/MinIO requests to keep exports alive via retry mechanism - # This allows ZooKeeper operations (KILL) to proceed quickly - minio_ip = cluster.minio_ip - minio_port = cluster.minio_port - - with PartitionManager() as pm: - # Block responses from MinIO (source_port matches MinIO service) - pm_rule_reject_responses = { - "destination": node.ip_address, - "source_port": minio_port, - "action": "REJECT --reject-with tcp-reset", - } - pm._add_rule(pm_rule_reject_responses) - - # Block requests to MinIO (destination: MinIO, destination_port: minio_port) - pm_rule_reject_requests = { - "destination": minio_ip, - "destination_port": minio_port, - "action": "REJECT --reject-with tcp-reset", - } - pm._add_rule(pm_rule_reject_requests) - - # Start two exports for the same table and one export for another table concurrently - node.query( - f""" - ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_max_retries = 50; - ALTER TABLE {mt_table} EXPORT PARTITION ID '2021' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_max_retries = 50; - ALTER TABLE {alt_mt_table} EXPORT PARTITION ID '2020' TO TABLE {alt_s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_max_retries = 50; - """ - ) - - # Kill all exports for the first table only while S3 is blocked - # Retry mechanism keeps exports alive, ZooKeeper operations proceed quickly - node.query(f"KILL EXPORT PARTITION WHERE source_table = '{mt_table}'") - - # Give some time for effects to propagate - time.sleep(5) - - # The killed table should have no commit for either partition - assert ( - node.query( - f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_*', format=LineAsString)" - ) - == '0\n' - ), "Partition 2020 was written to S3, but KILL by table should have stopped it" - - # The alternate table (not killed) should complete - assert ( - node.query( - f"SELECT count() FROM s3(s3_conn, filename='{alt_s3_table}/commit_*', format=LineAsString)" - ) - != '0\n' - ), "Alternate table export was affected by KILL on a different table" - - def test_concurrent_exports_to_different_targets(cluster): node = cluster.instances["replica1"] From 7e90b70b7678ef4b8ea7676f2a2b343edb017a2f Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 13 Nov 2025 13:35:21 +0100 Subject: [PATCH 41/58] some logs --- src/Storages/StorageReplicatedMergeTree.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d67d104194d4..ebaaef491ec4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8123,7 +8123,13 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & if (zookeeper->tryGet(fs::path(partition_exports_path) / "metadata.json", metadata_json)) { const auto manifest = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); - if (static_cast(manifest.create_time + manifest.ttl_seconds) < time(nullptr)) + + const auto now = time(nullptr); + const auto expiration_time = manifest.create_time + manifest.ttl_seconds; + + LOG_INFO(log, "Export with key {} has expiration time {}, now is {}", export_key, expiration_time, now); + + if (static_cast(expiration_time) < now) { has_expired = true; } From 0c67f052ea6bc92815f17069a494dcdbf7235253 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 13 Nov 2025 14:07:44 +0100 Subject: [PATCH 42/58] small changes --- .../ExportPartitionManifestUpdatingTask.cpp | 46 ------------------- src/Storages/StorageReplicatedMergeTree.cpp | 7 ++- 2 files changed, 6 insertions(+), 47 deletions(-) diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp index b933a7c5ef9d..6c44dfc891ef 100644 --- a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp @@ -10,52 +10,6 @@ namespace DB { -/// v2 of my initial design -/* -table_path/ - exports/ - / - metadata.json -> {tid, partition_id, destination_id, create_time, ttl} - parts/ - processing/ <-- not started, in progress - part_1.json -> {retry_count, max_retry_count, path_in_destination} - ... - part_n.json - processed/ - part_1.json -> {retry_count, max_retry_count, path_in_destination} - ... - part_n.json - locks - part_1 -> r1 - part_n -> rN - cleanup_lock <--- ephemeral - - One of the ideas behind this design is to reduce the number of required CAS loops. - It should work as follows: - - upon request, the structure should be created in zk in case it does not exist. - - once the task is published in zk, replicas are notified there is a new task and will fetch it. - - once they have it loaded locally, eventually the scheduler thread will run and try to lock individual parts in that task to export. - - the lock process is kind of the following: - - try to create an ephemeral node with the aprt name under the `locks` path. If it succeeded, the part is locked and the task will be scheduled within that replica. - - if it fails, it means the part is already locked by another replica. Try the next part. - - Once it completes, moves the part structure that lives under processing to processed with status either of failed or succeeded. If it failed, it'll also fail the entire task. - - Also, once it completes a local part, after moving it to processed (a transaction). It tries to read `processing` to check if it is empty. - - If it is empty, it means all parts have been exported and it is time to commit the export. Note that this is not transactional with the previous operation of moving the part to processed. - - So it means there is a chance the last part will be exported, but the server might die right before checking processing path and will never commit. For this, the cleanup thread also helps - - This is the overall idea, but please read the code to get a better understanding -*/ - struct CleanupLockRAII { CleanupLockRAII(const zkutil::ZooKeeperPtr & zk_, const std::string & cleanup_lock_path_, const std::string & replica_name_, const LoggerPtr & log_) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ebaaef491ec4..05d954664269 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4444,7 +4444,12 @@ std::vector StorageReplicatedMergeTree::getPartit const auto zk = getZooKeeper(); const auto exports_path = fs::path(zookeeper_path) / "exports"; - const auto children = zk->getChildren(exports_path); + std::vector children; + if (Coordination::Error::ZOK != zk->tryGetChildren(exports_path, children)) + { + LOG_INFO(log, "Failed to get children from exports path, returning empty export info list"); + return infos; + } for (const auto & child : children) { From 059467c35db707d02a057d2bba27114c22314831 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 14 Nov 2025 14:15:07 +0100 Subject: [PATCH 43/58] address a few minor comments --- src/Interpreters/InterpreterKillQueryQuery.cpp | 15 +++++++-------- src/Storages/MergeTree/ExportPartTask.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 1 + 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index bd91cf613acc..e7cbe29c3299 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -259,16 +259,15 @@ BlockIO InterpreterKillQueryQuery::execute() return res_io; const ColumnString & src_db_col = typeid_cast(*exports_block.getByName("source_database").column); - const ColumnString & src_tbl_col = typeid_cast(*exports_block.getByName("source_table").column); + const ColumnString & src_table_col = typeid_cast(*exports_block.getByName("source_table").column); const ColumnString & dst_db_col = typeid_cast(*exports_block.getByName("destination_database").column); - const ColumnString & dst_tbl_col = typeid_cast(*exports_block.getByName("destination_table").column); + const ColumnString & dst_table_col = typeid_cast(*exports_block.getByName("destination_table").column); const ColumnString & tx_col = typeid_cast(*exports_block.getByName("transaction_id").column); auto header = exports_block.cloneEmpty(); header.insert(0, {ColumnString::create(), std::make_shared(), "kill_status"}); MutableColumns res_columns = header.cloneEmptyColumns(); - auto table_id = StorageID::createEmpty(); AccessRightsElements required_access_rights; auto access = getContext()->getAccess(); bool access_denied = false; @@ -276,12 +275,12 @@ BlockIO InterpreterKillQueryQuery::execute() for (size_t i = 0; i < exports_block.rows(); ++i) { const auto src_database = src_db_col.getDataAt(i).toString(); - const auto src_table = src_tbl_col.getDataAt(i).toString(); - const auto dst_database = dst_db_col.getDataAt(i).toString(); - const auto dst_table = dst_tbl_col.getDataAt(i).toString(); + const auto src_table = src_table_col.getDataAt(i).toString(); + const auto dst_database = dst_db_col.getDataAt(i).toView(); + const auto dst_table = dst_table_col.getDataAt(i).toView(); - table_id = StorageID{src_database, src_table}; - auto transaction_id = tx_col.getDataAt(i).toString(); + const auto table_id = StorageID{src_database, src_table}; + const auto transaction_id = tx_col.getDataAt(i).toString(); CancellationCode code = CancellationCode::Unknown; if (!query.test) diff --git a/src/Storages/MergeTree/ExportPartTask.cpp b/src/Storages/MergeTree/ExportPartTask.cpp index 870c9d9373cb..71cf70f935a8 100644 --- a/src/Storages/MergeTree/ExportPartTask.cpp +++ b/src/Storages/MergeTree/ExportPartTask.cpp @@ -10,9 +10,9 @@ #include #include #include -#include "Common/Exception.h" +#include #include -#include "Storages/MergeTree/ExportList.h" +#include namespace ProfileEvents { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 05d954664269..8e87ddd03dfc 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -925,6 +925,7 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodesAttempt() const futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/quorum/last_part", String(), zkutil::CreateMode::Persistent)); futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/quorum/failed_parts", String(), zkutil::CreateMode::Persistent)); futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/mutations", String(), zkutil::CreateMode::Persistent)); + /// todo arthur do I need to put exports here as well? futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/quorum/parallel", String(), zkutil::CreateMode::Persistent)); From a8b9ce4bc80a10083ba1d93c035d7f643c859aba Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 14 Nov 2025 14:22:06 +0100 Subject: [PATCH 44/58] use existing raii construct for ephemeral nodes in zk --- .../ExportPartitionManifestUpdatingTask.cpp | 40 ++++--------------- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 2 files changed, 8 insertions(+), 34 deletions(-) diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp index 6c44dfc891ef..7886f3fee02d 100644 --- a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp @@ -9,36 +9,6 @@ namespace DB { - -struct CleanupLockRAII -{ - CleanupLockRAII(const zkutil::ZooKeeperPtr & zk_, const std::string & cleanup_lock_path_, const std::string & replica_name_, const LoggerPtr & log_) - : cleanup_lock_path(cleanup_lock_path_), zk(zk_), replica_name(replica_name_), log(log_) - { - is_locked = zk->tryCreate(cleanup_lock_path, replica_name, ::zkutil::CreateMode::Ephemeral) == Coordination::Error::ZOK; - - if (is_locked) - { - LOG_INFO(log, "ExportPartition Manifest Updating Task: Cleanup lock acquired, will remove stale entries"); - } - } - - ~CleanupLockRAII() - { - if (is_locked) - { - LOG_INFO(log, "ExportPartition Manifest Updating Task: Releasing cleanup lock"); - zk->tryRemove(cleanup_lock_path); - } - } - - bool is_locked; - std::string cleanup_lock_path; - zkutil::ZooKeeperPtr zk; - std::string replica_name; - LoggerPtr log; -}; - namespace { /* @@ -120,7 +90,11 @@ void ExportPartitionManifestUpdatingTask::poll() const std::string exports_path = fs::path(storage.zookeeper_path) / "exports"; const std::string cleanup_lock_path = fs::path(storage.zookeeper_path) / "exports_cleanup_lock"; - CleanupLockRAII cleanup_lock(zk, cleanup_lock_path, storage.replica_name, storage.log.load()); + auto cleanup_lock = zkutil::EphemeralNodeHolder::tryCreate(cleanup_lock_path, *zk, storage.replica_name); + if (cleanup_lock) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Cleanup lock acquired, will remove stale entries"); + } Coordination::Stat stat; const auto children = zk->getChildrenWatch(exports_path, &stat, storage.export_merge_tree_partition_watch_callback); @@ -154,7 +128,7 @@ void ExportPartitionManifestUpdatingTask::poll() && local_entry->manifest.transaction_id == metadata.transaction_id; /// If the entry is up to date and we don't have the cleanup lock, early exit, nothing to be done. - if (!cleanup_lock.is_locked && has_local_entry_and_is_up_to_date) + if (!cleanup_lock && has_local_entry_and_is_up_to_date) continue; auto status_watch_callback = std::make_shared([this, key](const Coordination::WatchResponse &) { @@ -173,7 +147,7 @@ void ExportPartitionManifestUpdatingTask::poll() /// if we have the cleanup lock, try to cleanup /// if we successfully cleaned it up, early exit - if (cleanup_lock.is_locked) + if (cleanup_lock) { bool cleanup_successful = tryCleanup( zk, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 8e87ddd03dfc..d4524fa34411 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8119,7 +8119,7 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & /// check if entry already exists if (zookeeper->exists(partition_exports_path)) { - LOG_INFO(log, "Export with key {} is already exported or it is being exported. Checking if it has expired so that we can overwrite it.", export_key); + LOG_INFO(log, "Export with key {} is already exported or it is being exported. Checking if it has expired so that we can overwrite it", export_key); bool has_expired = false; From 460f2f4189a2723706ff0f625a8ac166df5e066e Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 14 Nov 2025 14:36:13 +0100 Subject: [PATCH 45/58] add a missing piece (I hope it was a missing piece) --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d4524fa34411..fcb12b4ea881 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -925,7 +925,7 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodesAttempt() const futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/quorum/last_part", String(), zkutil::CreateMode::Persistent)); futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/quorum/failed_parts", String(), zkutil::CreateMode::Persistent)); futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/mutations", String(), zkutil::CreateMode::Persistent)); - /// todo arthur do I need to put exports here as well? + futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/exports", String(), zkutil::CreateMode::Persistent)); futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/quorum/parallel", String(), zkutil::CreateMode::Persistent)); From 522cfdb4e9e19075c219bd154a98b4d381f196a5 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 14 Nov 2025 15:20:50 +0100 Subject: [PATCH 46/58] do not mark part export as failed in case of cancelled export --- src/Storages/MergeTree/ExportPartTask.cpp | 4 +-- .../ExportPartitionTaskScheduler.cpp | 27 ++++++++++++++++--- .../MergeTree/ExportPartitionTaskScheduler.h | 3 ++- .../MergeTree/MergeTreePartExportManifest.h | 15 ++++++----- 4 files changed, 36 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/ExportPartTask.cpp b/src/Storages/MergeTree/ExportPartTask.cpp index 71cf70f935a8..dda4b5ae7b63 100644 --- a/src/Storages/MergeTree/ExportPartTask.cpp +++ b/src/Storages/MergeTree/ExportPartTask.cpp @@ -138,7 +138,7 @@ bool ExportPartTask::executeStep() storage.export_manifests.erase(manifest); if (manifest.completion_callback) - manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createFailure(e.message())); + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createFailure(e)); return false; } @@ -255,7 +255,7 @@ bool ExportPartTask::executeStep() storage.export_manifests.erase(manifest); if (manifest.completion_callback) - manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createFailure(e.message())); + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createFailure(e)); throw; } diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp index 7cfdf380a0f3..6f7e4625823d 100644 --- a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -2,7 +2,8 @@ #include #include #include -#include "Common/ZooKeeper/Types.h" +#include +#include #include "Storages/MergeTree/ExportPartitionUtils.h" #include "Storages/MergeTree/MergeTreePartExportManifest.h" @@ -10,6 +11,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int QUERY_WAS_CANCELLED; + extern const int LOGICAL_ERROR; +} + namespace { ContextPtr getContextCopyWithTaskSettings(const ContextPtr & context, const ExportReplicatedMergeTreePartitionManifest & manifest) @@ -210,10 +217,22 @@ void ExportPartitionTaskScheduler::handlePartExportFailure( const std::string & part_name, const std::filesystem::path & export_path, const zkutil::ZooKeeperPtr & zk, - const String & exception, + const std::optional & exception, size_t max_retries ) { + if (!exception) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "ExportPartition scheduler task: No exception provided for error handling. Sounds like a bug"); + } + + /// Early exit if the query was cancelled - no need to increment error counts + if (exception->code() == ErrorCodes::QUERY_WAS_CANCELLED) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} export was cancelled, skipping error handling", part_name); + return; + } + Coordination::Stat locked_by_stat; std::string locked_by; @@ -274,7 +293,7 @@ void ExportPartitionTaskScheduler::handlePartExportFailure( num_exceptions = std::stoull(num_exceptions_string.c_str()); ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "part", part_name, -1)); - ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "exception", exception, -1)); + ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "exception", exception->message(), -1)); } else { @@ -282,7 +301,7 @@ void ExportPartitionTaskScheduler::handlePartExportFailure( ops.emplace_back(zkutil::makeCreateRequest(count_path, "0", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(last_exception_path, "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "part", part_name, zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "exception", exception, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "exception", exception->message(), zkutil::CreateMode::Persistent)); } num_exceptions++; diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.h b/src/Storages/MergeTree/ExportPartitionTaskScheduler.h index bd420ed7bc1b..0045019a4ec7 100644 --- a/src/Storages/MergeTree/ExportPartitionTaskScheduler.h +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.h @@ -6,6 +6,7 @@ namespace DB { +class Exception; class StorageReplicatedMergeTree; struct ExportReplicatedMergeTreePartitionManifest; @@ -44,7 +45,7 @@ class ExportPartitionTaskScheduler const std::string & part_name, const std::filesystem::path & export_path, const zkutil::ZooKeeperPtr & zk, - const String & exception, + const std::optional & exception, size_t max_retries); bool tryToMovePartToProcessed( diff --git a/src/Storages/MergeTree/MergeTreePartExportManifest.h b/src/Storages/MergeTree/MergeTreePartExportManifest.h index 77a4446c270e..7fc195daac91 100644 --- a/src/Storages/MergeTree/MergeTreePartExportManifest.h +++ b/src/Storages/MergeTree/MergeTreePartExportManifest.h @@ -3,10 +3,13 @@ #include #include #include +#include namespace DB { +class Exception; + class ExportPartTask; struct MergeTreePartExportManifest @@ -23,23 +26,23 @@ struct MergeTreePartExportManifest struct CompletionCallbackResult { private: - CompletionCallbackResult(bool success_, const String & relative_path_in_destination_storage_, const String & exception_) - : success(success_), relative_path_in_destination_storage(relative_path_in_destination_storage_), exception(exception_) {} + CompletionCallbackResult(bool success_, const String & relative_path_in_destination_storage_, std::optional exception_) + : success(success_), relative_path_in_destination_storage(relative_path_in_destination_storage_), exception(std::move(exception_)) {} public: static CompletionCallbackResult createSuccess(const String & relative_path_in_destination_storage_) { - return CompletionCallbackResult(true, relative_path_in_destination_storage_, ""); + return CompletionCallbackResult(true, relative_path_in_destination_storage_, std::nullopt); } - static CompletionCallbackResult createFailure(const String & exception_) + static CompletionCallbackResult createFailure(Exception exception_) { - return CompletionCallbackResult(false, "", exception_); + return CompletionCallbackResult(false, "", std::move(exception_)); } bool success = false; String relative_path_in_destination_storage; - String exception; + std::optional exception; }; MergeTreePartExportManifest( From 147d21c367fd69f4cc9521290fc2c66ed57a5f77 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 14 Nov 2025 15:38:50 +0100 Subject: [PATCH 47/58] rmv super flaky test --- .../test.py | 54 ------------------- 1 file changed, 54 deletions(-) diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index d5c248e9f97c..ad25f06e70bb 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -241,60 +241,6 @@ def test_drop_source_table_during_export(cluster): assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_*', format=LineAsString)") == '0\n', "Background operations completed even with the table dropped" -def test_drop_destination_table_during_export(cluster): - node = cluster.instances["replica1"] - # node2 = cluster.instances["replica2"] - watcher_node = cluster.instances["watcher_node"] - - mt_table = "drop_destination_table_during_export_mt_table" - s3_table = "drop_destination_table_during_export_s3_table" - - create_tables_and_insert_data(node, mt_table, s3_table, "replica1") - create_s3_table(watcher_node, s3_table) - - # Block S3/MinIO requests to keep exports alive via retry mechanism - # This allows ZooKeeper operations (KILL) to proceed quickly - minio_ip = cluster.minio_ip - minio_port = cluster.minio_port - - with PartitionManager() as pm: - # Block responses from MinIO (source_port matches MinIO service) - pm_rule_reject_responses = { - "destination": node.ip_address, - "source_port": minio_port, - "action": "REJECT --reject-with tcp-reset", - } - pm._add_rule(pm_rule_reject_responses) - - # Block requests to MinIO (destination: MinIO, destination_port: minio_port) - pm_rule_reject_requests = { - "destination": minio_ip, - "destination_port": minio_port, - "action": "REJECT --reject-with tcp-reset", - } - pm._add_rule(pm_rule_reject_requests) - - export_queries = f""" - ALTER TABLE {mt_table} - EXPORT PARTITION ID '2020' TO TABLE {s3_table} - SETTINGS allow_experimental_export_merge_tree_part=1; - ALTER TABLE {mt_table} - EXPORT PARTITION ID '2021' TO TABLE {s3_table} - SETTINGS allow_experimental_export_merge_tree_part=1; - """ - - node.query(export_queries) - - # The pointer to the destination table is still valid, so the write will continue - node.query(f"DROP TABLE {s3_table}") - - # give some time for the export to finish - time.sleep(10) - - # not sure this is the expected behavior, but adding until we make a decision - assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_*', format=LineAsString)") != '0\n', "Background operations did not complete after dropping the destination table" - - def test_concurrent_exports_to_different_targets(cluster): node = cluster.instances["replica1"] From bada40f648404d73b0fa75c0b36ad6502324ac56 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Sat, 15 Nov 2025 22:47:49 +0100 Subject: [PATCH 48/58] put the background threads, kill operation and system table behind server experimental setting --- src/Core/ServerSettings.cpp | 1 + .../InterpreterKillQueryQuery.cpp | 13 +++++ .../ReplicatedMergeTreeRestartingThread.cpp | 18 +++++-- src/Storages/StorageReplicatedMergeTree.cpp | 49 ++++++++++++------- src/Storages/System/attachSystemTables.cpp | 12 ++++- ...perimental_export_merge_tree_partition.xml | 3 ++ tests/config/install.sh | 1 + .../allow_experimental_export_partition.xml | 3 ++ .../disable_experimental_export_partition.xml | 3 ++ .../test.py | 27 +++++++++- 10 files changed, 105 insertions(+), 25 deletions(-) create mode 100644 tests/config/config.d/enable_experimental_export_merge_tree_partition.xml create mode 100644 tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/allow_experimental_export_partition.xml create mode 100644 tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/disable_experimental_export_partition.xml diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 8d0a31538dc3..28a660647541 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -1144,6 +1144,7 @@ The policy on how to perform a scheduling of CPU slots specified by `concurrent_ DECLARE(UInt64, object_storage_list_objects_cache_max_entries, 1000, "Maximum size of ObjectStorage list objects cache in entries. Zero means disabled.", 0) \ DECLARE(UInt64, object_storage_list_objects_cache_ttl, 3600, "Time to live of records in ObjectStorage list objects cache in seconds. Zero means unlimited", 0) \ DECLARE(UInt64, input_format_parquet_metadata_cache_max_size, 500000000, "Maximum size of parquet file metadata cache", 0) \ + DECLARE(Bool, enable_experimental_export_merge_tree_partition_feature, false, "Enable export replicated merge tree partition feature. It is experimental and not yet ready for production use.", 0) \ // clang-format on /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in dumpToSystemServerSettingsColumns below diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index e7cbe29c3299..4054b1e7211a 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -37,11 +38,17 @@ namespace Setting extern const SettingsUInt64 max_parser_depth; } +namespace ServerSetting +{ + extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; +} + namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int ACCESS_DENIED; extern const int NOT_IMPLEMENTED; + extern const int SUPPORT_IS_DISABLED; } @@ -252,6 +259,12 @@ BlockIO InterpreterKillQueryQuery::execute() } case ASTKillQueryQuery::Type::ExportPartition: { + if (!getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Exporting merge tree partition is experimental. Set the server setting `enable_experimental_export_merge_tree_partition_feature` to enable it"); + } + Block exports_block = getSelectResult( "source_database, source_table, transaction_id, destination_database, destination_table, partition_id", "system.replicated_partition_exports"); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 802afa69ea44..aca997b1e443 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -11,6 +11,7 @@ #include #include #include +#include namespace CurrentMetrics @@ -27,6 +28,11 @@ namespace MergeTreeSetting extern const MergeTreeSettingsSeconds zookeeper_session_expiration_check_period; } +namespace ServerSetting +{ + extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; +} + namespace ErrorCodes { extern const int REPLICA_IS_ALREADY_ACTIVE; @@ -171,12 +177,18 @@ bool ReplicatedMergeTreeRestartingThread::runImpl() storage.mutations_updating_task->activateAndSchedule(); storage.mutations_finalizing_task->activateAndSchedule(); storage.merge_selecting_task->activateAndSchedule(); + + if (storage.getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + storage.export_merge_tree_partition_updating_task->activateAndSchedule(); + storage.export_merge_tree_partition_select_task->activateAndSchedule(); + storage.export_merge_tree_partition_status_handling_task->activateAndSchedule(); + } + storage.cleanup_thread.start(); storage.async_block_ids_cache.start(); storage.part_check_thread.start(); - storage.export_merge_tree_partition_updating_task->activateAndSchedule(); - storage.export_merge_tree_partition_select_task->activateAndSchedule(); - storage.export_merge_tree_partition_status_handling_task->activateAndSchedule(); + LOG_DEBUG(log, "Table started successfully"); return true; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index fcb12b4ea881..81673b82d49f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -304,6 +304,11 @@ namespace ErrorCodes extern const int INVALID_SETTING_VALUE; } +namespace ServerSetting +{ + extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; +} + namespace ActionLocks { extern const StorageActionBlockType PartsMerge; @@ -480,26 +485,30 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( /// Will be activated by restarting thread. mutations_finalizing_task->deactivate(); - export_merge_tree_partition_manifest_updater = std::make_shared(*this); + if (getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + export_merge_tree_partition_manifest_updater = std::make_shared(*this); + + export_merge_tree_partition_task_scheduler = std::make_shared(*this); - export_merge_tree_partition_task_scheduler = std::make_shared(*this); + export_merge_tree_partition_updating_task = getContext()->getSchedulePool().createTask( + getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_updating_task)", [this] { exportMergeTreePartitionUpdatingTask(); }); - export_merge_tree_partition_updating_task = getContext()->getSchedulePool().createTask( - getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_updating_task)", [this] { exportMergeTreePartitionUpdatingTask(); }); + export_merge_tree_partition_updating_task->deactivate(); - export_merge_tree_partition_updating_task->deactivate(); + export_merge_tree_partition_status_handling_task = getContext()->getSchedulePool().createTask( + getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_status_handling_task)", [this] { exportMergeTreePartitionStatusHandlingTask(); }); - export_merge_tree_partition_status_handling_task = getContext()->getSchedulePool().createTask( - getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_status_handling_task)", [this] { exportMergeTreePartitionStatusHandlingTask(); }); + export_merge_tree_partition_status_handling_task->deactivate(); - export_merge_tree_partition_status_handling_task->deactivate(); + export_merge_tree_partition_watch_callback = std::make_shared(export_merge_tree_partition_updating_task->getWatchCallback()); - export_merge_tree_partition_watch_callback = std::make_shared(export_merge_tree_partition_updating_task->getWatchCallback()); - - export_merge_tree_partition_select_task = getContext()->getSchedulePool().createTask( - getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_select_task)", [this] { selectPartsToExport(); }); + export_merge_tree_partition_select_task = getContext()->getSchedulePool().createTask( + getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_select_task)", [this] { selectPartsToExport(); }); + + export_merge_tree_partition_select_task->deactivate(); + } - export_merge_tree_partition_select_task->deactivate(); bool has_zookeeper = getContext()->hasZooKeeper() || getContext()->hasAuxiliaryZooKeeper(zookeeper_info.zookeeper_name); if (has_zookeeper) @@ -5923,9 +5932,13 @@ void StorageReplicatedMergeTree::partialShutdown() queue_updating_task->deactivate(); mutations_updating_task->deactivate(); mutations_finalizing_task->deactivate(); - export_merge_tree_partition_updating_task->deactivate(); - export_merge_tree_partition_select_task->deactivate(); - export_merge_tree_partition_status_handling_task->deactivate(); + + if (getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + export_merge_tree_partition_updating_task->deactivate(); + export_merge_tree_partition_select_task->deactivate(); + export_merge_tree_partition_status_handling_task->deactivate(); + } cleanup_thread.stop(); async_block_ids_cache.stop(); @@ -8073,10 +8086,10 @@ void StorageReplicatedMergeTree::fetchPartition( void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & command, ContextPtr query_context) { - if (!query_context->getSettingsRef()[Setting::allow_experimental_export_merge_tree_part]) + if (!query_context->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Exporting merge tree part is experimental. Set `allow_experimental_export_merge_tree_part` to enable it"); + "Exporting merge tree partition is experimental. Set the server setting `enable_experimental_export_merge_tree_partition_feature` to enable it"); } const auto dest_database = query_context->resolveDatabase(command.to_database); diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 358ac3d78438..d6ce54003aea 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -5,7 +5,7 @@ #include #include #include - +#include #include #include #include @@ -130,6 +130,11 @@ namespace DB { +namespace ServerSetting +{ + extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; +} + void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, bool has_zookeeper) { attachNoDescription(context, system_database, "one", "This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`."); @@ -212,7 +217,10 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "merges", "Contains a list of merges currently executing merges of MergeTree tables and their progress. Each merge operation is represented by a single row."); attach(context, system_database, "moves", "Contains information about in-progress data part moves of MergeTree tables. Each data part movement is represented by a single row."); attach(context, system_database, "exports", "Contains a list of exports currently executing exports of MergeTree tables and their progress. Each export operation is represented by a single row."); - attach(context, system_database, "replicated_partition_exports", "Contains a list of partition exports of ReplicatedMergeTree tables and their progress. Each export operation is represented by a single row."); + if (context->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + attach(context, system_database, "replicated_partition_exports", "Contains a list of partition exports of ReplicatedMergeTree tables and their progress. Each export operation is represented by a single row."); + } attach(context, system_database, "mutations", "Contains a list of mutations and their progress. Each mutation command is represented by a single row."); attachNoDescription(context, system_database, "replicas", "Contains information and status of all table replicas on current server. Each replica is represented by a single row."); attach(context, system_database, "replication_queue", "Contains information about tasks from replication queues stored in ClickHouse Keeper, or ZooKeeper, for each table replica."); diff --git a/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml b/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml new file mode 100644 index 000000000000..14c6dfbfa7c8 --- /dev/null +++ b/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml @@ -0,0 +1,3 @@ + + 0 + \ No newline at end of file diff --git a/tests/config/install.sh b/tests/config/install.sh index b8874097c940..90b70f3ce5cc 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -85,6 +85,7 @@ ln -sf $SRC_PATH/config.d/blob_storage_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/custom_settings_prefixes.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/database_catalog_drop_table_concurrency.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_access_control_improvements.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/enable_experimental_export_merge_tree_partition.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/macros.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/secure_ports.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/clusters.xml $DEST_SERVER_PATH/config.d/ diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/allow_experimental_export_partition.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/allow_experimental_export_partition.xml new file mode 100644 index 000000000000..f8c5fab1a3be --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/allow_experimental_export_partition.xml @@ -0,0 +1,3 @@ + + 1 + \ No newline at end of file diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/disable_experimental_export_partition.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/disable_experimental_export_partition.xml new file mode 100644 index 000000000000..ba6508ebd660 --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/disable_experimental_export_partition.xml @@ -0,0 +1,3 @@ + + 0 + \ No newline at end of file diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index ad25f06e70bb..425c361d44ab 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -15,7 +15,7 @@ def cluster(): cluster = ClickHouseCluster(__file__) cluster.add_instance( "replica1", - main_configs=["configs/named_collections.xml"], + main_configs=["configs/named_collections.xml", "configs/allow_experimental_export_partition.xml"], user_configs=["configs/users.d/profile.xml"], with_minio=True, stay_alive=True, @@ -23,7 +23,7 @@ def cluster(): ) cluster.add_instance( "replica2", - main_configs=["configs/named_collections.xml"], + main_configs=["configs/named_collections.xml", "configs/allow_experimental_export_partition.xml"], user_configs=["configs/users.d/profile.xml"], with_minio=True, stay_alive=True, @@ -36,6 +36,14 @@ def cluster(): user_configs=[], with_minio=True, ) + cluster.add_instance( + "replica_with_export_disabled", + main_configs=["configs/named_collections.xml", "configs/disable_experimental_export_partition.xml"], + user_configs=["configs/users.d/profile.xml"], + with_minio=True, + stay_alive=True, + with_zookeeper=True, + ) logging.info("Starting cluster...") cluster.start() yield cluster @@ -538,6 +546,21 @@ def test_export_partition_file_already_exists_policy(cluster): ) == "FAILED\n", "Export should be marked as FAILED" +def test_export_partition_feature_is_disabled(cluster): + replica_with_export_disabled = cluster.instances["replica_with_export_disabled"] + + mt_table = "export_partition_feature_is_disabled_mt_table" + s3_table = "export_partition_feature_is_disabled_s3_table" + + create_tables_and_insert_data(replica_with_export_disabled, mt_table, s3_table, "replica1") + + error = replica_with_export_disabled.query_and_get_error(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1;") + assert "experimental" in error, "Expected error about disabled feature" + + # make sure kill operation also throws + error = replica_with_export_disabled.query_and_get_error(f"KILL EXPORT PARTITION WHERE partition_id = '2020' and source_table = '{mt_table}' and destination_table = '{s3_table}'") + assert "experimental" in error, "Expected error about disabled feature" + # def test_source_mutations_during_export_snapshot(cluster): # node = cluster.instances["replica1"] From 0486df28a7fd64677ec7edc9e09fb38c893eb2e7 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Sun, 16 Nov 2025 00:34:23 +0100 Subject: [PATCH 49/58] try to prevent dangling reference access --- .../ExportPartitionManifestUpdatingTask.cpp | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp index 7886f3fee02d..79b92663b7bf 100644 --- a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp @@ -131,9 +131,17 @@ void ExportPartitionManifestUpdatingTask::poll() if (!cleanup_lock && has_local_entry_and_is_up_to_date) continue; - auto status_watch_callback = std::make_shared([this, key](const Coordination::WatchResponse &) { - storage.export_merge_tree_partition_manifest_updater->addStatusChange(key); - storage.export_merge_tree_partition_status_handling_task->schedule(); + std::weak_ptr weak_manifest_updater = storage.export_merge_tree_partition_manifest_updater; + + auto status_watch_callback = std::make_shared([weak_manifest_updater, key](const Coordination::WatchResponse &) + { + /// If the table is dropped but the watch is not removed, we need to prevent use after free + /// below code assumes that if manifest updater is still alive, the status handling task is also alive + if (auto manifest_updater = weak_manifest_updater.lock()) + { + manifest_updater->addStatusChange(key); + manifest_updater->storage.export_merge_tree_partition_status_handling_task->schedule(); + } }); std::string status; From 6a5e2a26b355b8a190ae4e7c7d677b9bc05adb6a Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Sun, 16 Nov 2025 14:39:15 +0100 Subject: [PATCH 50/58] rmv unrelated change --- tests/config/config.d/clusters.xml | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/tests/config/config.d/clusters.xml b/tests/config/config.d/clusters.xml index de2293cf458f..1a507a76fd85 100644 --- a/tests/config/config.d/clusters.xml +++ b/tests/config/config.d/clusters.xml @@ -162,19 +162,6 @@ - - - false - - 127.0.0.1 - 9000 - - - 127.0.0.1 - 9100 - - - false From 8602587f24bb8013f6a7522ef032f698bf162aaf Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Sun, 16 Nov 2025 14:47:58 +0100 Subject: [PATCH 51/58] use the right config name --- .../enable_experimental_export_merge_tree_partition.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml b/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml index 14c6dfbfa7c8..0ff84f2cd7ae 100644 --- a/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml +++ b/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml @@ -1,3 +1,3 @@ - 0 + 0 \ No newline at end of file From 276dbf14ee47472fee0ef5d0c89a5e6787e9f7df Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Sun, 16 Nov 2025 12:11:44 -0300 Subject: [PATCH 52/58] use settingsenum --- src/Core/Settings.cpp | 8 ++++---- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 2 +- src/Core/SettingsEnums.cpp | 2 ++ src/Core/SettingsEnums.h | 10 ++++++++++ src/Storages/MergeTree/ExportPartTask.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 13 ++----------- .../MergeTree/MergeTreePartExportManifest.h | 7 +------ src/Storages/StorageReplicatedMergeTree.cpp | 14 ++------------ .../test.py | 4 ++-- 10 files changed, 27 insertions(+), 38 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 168cf5c9ef59..2c1b5498ba56 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -6884,11 +6884,11 @@ Maximum number of retries for exporting a merge tree part in an export partition Determines how long the manifest will live in ZooKeeper. It prevents the same partition from being exported twice to the same destination. This setting does not affect / delete in progress tasks. It'll only cleanup the completed ones. )", 0) \ - DECLARE(String, export_merge_tree_part_file_already_exists_policy, "NO_OP", R"( + DECLARE(MergeTreePartExportFileAlreadyExistsPolicy, export_merge_tree_part_file_already_exists_policy, MergeTreePartExportFileAlreadyExistsPolicy::skip, R"( Possible values: -- NO_OP - No-op if the file already exists - Default. -- ERROR - Throw an error if the file already exists. -- OVERWRITE - Overwrite the file +- skip - Skip the file if it already exists. +- error - Throw an error if the file already exists. +- overwrite - Overwrite the file. )", 0) \ DECLARE(Timezone, iceberg_timezone_for_timestamptz, "UTC", R"( Timezone for Iceberg timestamptz field. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 546c687e445a..c70767e21aca 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -81,6 +81,7 @@ class WriteBuffer; M(CLASS_NAME, LogsLevel) \ M(CLASS_NAME, Map) \ M(CLASS_NAME, MaxThreads) \ + M(CLASS_NAME, MergeTreePartExportFileAlreadyExistsPolicy) \ M(CLASS_NAME, Milliseconds) \ M(CLASS_NAME, MsgPackUUIDRepresentation) \ M(CLASS_NAME, MySQLDataTypesSupport) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 04eb38a8c8e1..0d3105849722 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -50,7 +50,7 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"export_merge_tree_partition_force_export", false, false, "New setting."}, {"export_merge_tree_partition_max_retries", 3, 3, "New setting."}, {"export_merge_tree_partition_manifest_ttl", 180, 180, "New setting."}, - {"export_merge_tree_part_file_already_exists_policy", "NO_OP", "NO_OP", "New setting."}, + {"export_merge_tree_part_file_already_exists_policy", "skip", "skip", "New setting."}, {"iceberg_timezone_for_timestamptz", "UTC", "UTC", "New setting."} }); addSettingsChanges(settings_changes_history, "25.8", diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 8e34fc296b1f..2fb4f1668ed4 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -370,4 +370,6 @@ IMPLEMENT_SETTING_ENUM( {"manifest_list_entry", IcebergMetadataLogLevel::ManifestListEntry}, {"manifest_file_metadata", IcebergMetadataLogLevel::ManifestFileMetadata}, {"manifest_file_entry", IcebergMetadataLogLevel::ManifestFileEntry}}) + +IMPLEMENT_SETTING_AUTO_ENUM(MergeTreePartExportFileAlreadyExistsPolicy, ErrorCodes::BAD_ARGUMENTS); } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index d4472e339edf..bb058974944f 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -480,4 +480,14 @@ enum class IcebergMetadataLogLevel : uint8_t }; DECLARE_SETTING_ENUM(IcebergMetadataLogLevel) + +enum class MergeTreePartExportFileAlreadyExistsPolicy : uint8_t +{ + skip, + error, + overwrite, +}; + +DECLARE_SETTING_ENUM(MergeTreePartExportFileAlreadyExistsPolicy) + } diff --git a/src/Storages/MergeTree/ExportPartTask.cpp b/src/Storages/MergeTree/ExportPartTask.cpp index dda4b5ae7b63..b34b4154b9a1 100644 --- a/src/Storages/MergeTree/ExportPartTask.cpp +++ b/src/Storages/MergeTree/ExportPartTask.cpp @@ -95,7 +95,7 @@ bool ExportPartTask::executeStep() manifest.data_part->name + "_" + manifest.data_part->checksums.getTotalChecksumHex(), block_with_partition_values, (*exports_list_entry)->destination_file_path, - manifest.file_already_exists_policy == MergeTreePartExportManifest::FileAlreadyExistsPolicy::OVERWRITE, + manifest.file_already_exists_policy == MergeTreePartExportManifest::FileAlreadyExistsPolicy::overwrite, context_copy); } catch (const Exception & e) @@ -105,7 +105,7 @@ bool ExportPartTask::executeStep() ProfileEvents::increment(ProfileEvents::PartsExportDuplicated); /// File already exists and the policy is NO_OP, treat it as success. - if (manifest.file_already_exists_policy == MergeTreePartExportManifest::FileAlreadyExistsPolicy::NO_OP) + if (manifest.file_already_exists_policy == MergeTreePartExportManifest::FileAlreadyExistsPolicy::skip) { storage.writePartLog( PartLogElement::Type::EXPORT_PART, diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6af5661f3e15..f1f39c4c64bf 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -212,7 +212,7 @@ namespace Setting extern const SettingsBool apply_patch_parts; extern const SettingsBool allow_experimental_export_merge_tree_part; extern const SettingsUInt64 min_bytes_to_use_direct_io; - extern const SettingsString export_merge_tree_part_file_already_exists_policy; + extern const SettingsMergeTreePartExportFileAlreadyExistsPolicy export_merge_tree_part_file_already_exists_policy; extern const SettingsBool output_format_parallel_formatting; extern const SettingsBool output_format_parquet_parallel_encoding; } @@ -6253,20 +6253,11 @@ void MergeTreeData::exportPartToTable( part_name, getStorageID().getFullTableName()); { - const auto file_already_exists_policy = magic_enum::enum_cast(Poco::toUpper(query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value)); - if (!file_already_exists_policy) - { - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Invalid value for setting 'export_merge_tree_part_file_already_exists_policy': {}", - query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value); - } - MergeTreePartExportManifest manifest( dest_storage->getStorageID(), part, transaction_id, - *file_already_exists_policy, + query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value, query_context->getSettingsRef()[Setting::output_format_parallel_formatting], query_context->getSettingsRef()[Setting::output_format_parquet_parallel_encoding], query_context->getSettingsRef()[Setting::max_threads], diff --git a/src/Storages/MergeTree/MergeTreePartExportManifest.h b/src/Storages/MergeTree/MergeTreePartExportManifest.h index 7fc195daac91..0259088c3ad5 100644 --- a/src/Storages/MergeTree/MergeTreePartExportManifest.h +++ b/src/Storages/MergeTree/MergeTreePartExportManifest.h @@ -14,12 +14,7 @@ class ExportPartTask; struct MergeTreePartExportManifest { - enum class FileAlreadyExistsPolicy - { - NO_OP, - ERROR, - OVERWRITE, - }; + using FileAlreadyExistsPolicy = MergeTreePartExportFileAlreadyExistsPolicy; using DataPartPtr = std::shared_ptr; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 81673b82d49f..f2ed0aa22aa1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -198,7 +198,7 @@ namespace Setting extern const SettingsBool output_format_parallel_formatting; extern const SettingsBool output_format_parquet_parallel_encoding; extern const SettingsMaxThreads max_threads; - extern const SettingsString export_merge_tree_part_file_already_exists_policy; + extern const SettingsMergeTreePartExportFileAlreadyExistsPolicy export_merge_tree_part_file_already_exists_policy; } namespace MergeTreeSetting @@ -8205,17 +8205,7 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & manifest.parallel_formatting = query_context->getSettingsRef()[Setting::output_format_parallel_formatting]; manifest.parquet_parallel_encoding = query_context->getSettingsRef()[Setting::output_format_parquet_parallel_encoding]; - const auto file_already_exists_policy = magic_enum::enum_cast(query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value); - - if (!file_already_exists_policy) - { - throw Exception( - ErrorCodes::INVALID_SETTING_VALUE, - "Invalid value for setting 'export_merge_tree_part_file_already_exists_policy': {}", - query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value); - } - - manifest.file_already_exists_policy = file_already_exists_policy.value(); + manifest.file_already_exists_policy = query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value; ops.emplace_back(zkutil::makeCreateRequest( fs::path(partition_exports_path) / "metadata.json", diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index 425c361d44ab..e878e7f59231 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -506,7 +506,7 @@ def test_export_partition_file_already_exists_policy(cluster): # now let's try with a file exists policy that is not NO_OP node.query( - f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_force_export=1, export_merge_tree_part_file_already_exists_policy='OVERWRITE'", + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_force_export=1, export_merge_tree_part_file_already_exists_policy='overwrite'", query_id=query_id_3, ) @@ -527,7 +527,7 @@ def test_export_partition_file_already_exists_policy(cluster): # last but not least, let's try with the error policy # max retries = 1 so it fails fast node.query( - f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_force_export=1, export_merge_tree_part_file_already_exists_policy='ERROR', export_merge_tree_partition_max_retries=1", + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_force_export=1, export_merge_tree_part_file_already_exists_policy='error', export_merge_tree_partition_max_retries=1", query_id=query_id_4, ) From 1ec94fa7b9da47fec01448fc121a50ee78ab82ee Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Sun, 16 Nov 2025 15:20:19 -0300 Subject: [PATCH 53/58] Enable experimental export merge tree partition --- .../enable_experimental_export_merge_tree_partition.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml b/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml index 0ff84f2cd7ae..f778420caa19 100644 --- a/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml +++ b/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml @@ -1,3 +1,3 @@ - 0 - \ No newline at end of file + 1 + From 4135ff74b1b20514f2215f640e339a824ef06f0e Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Sun, 16 Nov 2025 17:08:56 -0300 Subject: [PATCH 54/58] Update enable_experimental_export_merge_tree_partition.xml --- .../enable_experimental_export_merge_tree_partition.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml b/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml index f778420caa19..72014c9de4db 100644 --- a/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml +++ b/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml @@ -1,3 +1,3 @@ - 1 + 1 From a736a6c6673ac597981a3e35c5c8609682c40f9c Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 17 Nov 2025 09:22:47 -0300 Subject: [PATCH 55/58] fix privilege issue --- src/Interpreters/InterpreterAlterQuery.cpp | 2 +- .../test.py | 75 +++++++++++++++++++ 2 files changed, 76 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index aac89a671861..e280cd6318f9 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -547,7 +547,7 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS } case ASTAlterCommand::EXPORT_PARTITION: { - required_access.emplace_back(AccessType::ALTER_EXPORT_PARTITION, command.to_database, command.to_table); + required_access.emplace_back(AccessType::ALTER_EXPORT_PARTITION, database, table); required_access.emplace_back(AccessType::INSERT, command.to_database, command.to_table); break; } diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index e878e7f59231..f0103bee795b 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -561,6 +561,81 @@ def test_export_partition_feature_is_disabled(cluster): error = replica_with_export_disabled.query_and_get_error(f"KILL EXPORT PARTITION WHERE partition_id = '2020' and source_table = '{mt_table}' and destination_table = '{s3_table}'") assert "experimental" in error, "Expected error about disabled feature" + +def test_export_partition_permissions(cluster): + """Test that export partition validates permissions correctly: + - User needs ALTER permission on source table + - User needs INSERT permission on destination table + """ + node = cluster.instances["replica1"] + + mt_table = "permissions_mt_table" + s3_table = "permissions_s3_table" + + # Create tables as default user + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # Create test users with specific permissions + node.query("CREATE USER IF NOT EXISTS user_no_alter IDENTIFIED WITH no_password") + node.query("CREATE USER IF NOT EXISTS user_no_insert IDENTIFIED WITH no_password") + node.query("CREATE USER IF NOT EXISTS user_with_permissions IDENTIFIED WITH no_password") + + # Grant basic access to all users + node.query(f"GRANT SELECT ON {mt_table} TO user_no_alter") + node.query(f"GRANT SELECT ON {s3_table} TO user_no_alter") + + # user_no_insert has ALTER on source but no INSERT on destination + node.query(f"GRANT ALTER ON {mt_table} TO user_no_insert") + node.query(f"GRANT SELECT ON {s3_table} TO user_no_insert") + + # user_with_permissions has both ALTER and INSERT + node.query(f"GRANT ALTER ON {mt_table} TO user_with_permissions") + node.query(f"GRANT INSERT ON {s3_table} TO user_with_permissions") + + # Test 1: User without ALTER permission should fail + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " + f"SETTINGS allow_experimental_export_merge_tree_part=1", + user="user_no_alter" + ) + assert "ACCESS_DENIED" in error or "Not enough privileges" in error, \ + f"Expected ACCESS_DENIED error for user without ALTER, got: {error}" + + # Test 2: User with ALTER but without INSERT permission should fail + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " + f"SETTINGS allow_experimental_export_merge_tree_part=1", + user="user_no_insert" + ) + assert "ACCESS_DENIED" in error or "Not enough privileges" in error, \ + f"Expected ACCESS_DENIED error for user without INSERT, got: {error}" + + # Test 3: User with both ALTER and INSERT should succeed + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " + f"SETTINGS allow_experimental_export_merge_tree_part=1", + user="user_with_permissions" + ) + + # Wait for export to complete + time.sleep(5) + + # Verify the export succeeded + result = node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") + assert result.strip() == "3", f"Expected 3 rows exported, got: {result}" + + # Verify system table shows COMPLETED status + status = node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) + assert status.strip() == "COMPLETED", f"Expected COMPLETED status, got: {status}" + + # def test_source_mutations_during_export_snapshot(cluster): # node = cluster.instances["replica1"] From 37d50ece982b8983a46f9411e103deccc58103cd Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 17 Nov 2025 10:55:07 -0300 Subject: [PATCH 56/58] generate transaction id instead of reusing the queryid --- src/Functions/generateSnowflakeID.cpp | 5 + src/Functions/generateSnowflakeID.h | 2 + src/Storages/MergeTree/ExportPartTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 7 +- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/MergeTreePartExportManifest.h | 6 +- src/Storages/StorageReplicatedMergeTree.cpp | 3 +- .../test.py | 118 +++++++++++------- 8 files changed, 88 insertions(+), 57 deletions(-) diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index 5055f548a0ee..d3c8137e6611 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -154,6 +154,11 @@ uint64_t generateSnowflakeID() return fromSnowflakeId(snowflake_id); } +std::string generateSnowflakeIDString() +{ + return std::to_string(generateSnowflakeID()); +} + class FunctionGenerateSnowflakeID : public IFunction { public: diff --git a/src/Functions/generateSnowflakeID.h b/src/Functions/generateSnowflakeID.h index 38fa684a9b4b..4fc173dcf1be 100644 --- a/src/Functions/generateSnowflakeID.h +++ b/src/Functions/generateSnowflakeID.h @@ -7,4 +7,6 @@ namespace DB uint64_t generateSnowflakeID(); +std::string generateSnowflakeIDString(); + } diff --git a/src/Storages/MergeTree/ExportPartTask.cpp b/src/Storages/MergeTree/ExportPartTask.cpp index b34b4154b9a1..a22eca8397c2 100644 --- a/src/Storages/MergeTree/ExportPartTask.cpp +++ b/src/Storages/MergeTree/ExportPartTask.cpp @@ -289,7 +289,7 @@ Priority ExportPartTask::getPriority() const String ExportPartTask::getQueryId() const { - return manifest.query_id; + return manifest.transaction_id; } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f1f39c4c64bf..576340b9001d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -102,6 +102,7 @@ #include #include #include +#include #include @@ -6212,7 +6213,7 @@ void MergeTreeData::exportPartToTable(const PartitionCommand & command, ContextP const auto database_name = query_context->resolveDatabase(command.to_database); - exportPartToTable(part_name, StorageID{database_name, command.to_table}, query_context->getCurrentQueryId(), query_context); + exportPartToTable(part_name, StorageID{database_name, command.to_table}, generateSnowflakeIDString(), query_context); } void MergeTreeData::exportPartToTable( @@ -6275,13 +6276,13 @@ void MergeTreeData::exportPartToTable( background_moves_assignee.trigger(); } -void MergeTreeData::killExportPart(const String & query_id) +void MergeTreeData::killExportPart(const String & transaction_id) { std::lock_guard lock(export_manifests_mutex); std::erase_if(export_manifests, [&](const auto & manifest) { - if (manifest.query_id == query_id) + if (manifest.transaction_id == transaction_id) { if (manifest.task) manifest.task->cancel(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index abeda51be05e..521bc7e50279 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -991,7 +991,7 @@ class MergeTreeData : public IStorage, public WithMutableContext ContextPtr query_context, std::function completion_callback = {}); - void killExportPart(const String & query_id); + void killExportPart(const String & transaction_id); virtual void exportPartitionToTable(const PartitionCommand &, ContextPtr) { diff --git a/src/Storages/MergeTree/MergeTreePartExportManifest.h b/src/Storages/MergeTree/MergeTreePartExportManifest.h index 0259088c3ad5..ab5e067edd53 100644 --- a/src/Storages/MergeTree/MergeTreePartExportManifest.h +++ b/src/Storages/MergeTree/MergeTreePartExportManifest.h @@ -43,7 +43,7 @@ struct MergeTreePartExportManifest MergeTreePartExportManifest( const StorageID & destination_storage_id_, const DataPartPtr & data_part_, - const String & query_id_, + const String & transaction_id_, FileAlreadyExistsPolicy file_already_exists_policy_, bool parallel_formatting_, bool parquet_parallel_encoding_, @@ -51,7 +51,7 @@ struct MergeTreePartExportManifest std::function completion_callback_ = {}) : destination_storage_id(destination_storage_id_), data_part(data_part_), - query_id(query_id_), + transaction_id(transaction_id_), file_already_exists_policy(file_already_exists_policy_), parallel_formatting(parallel_formatting_), parquet_parallel_encoding(parquet_parallel_encoding_), @@ -62,7 +62,7 @@ struct MergeTreePartExportManifest StorageID destination_storage_id; DataPartPtr data_part; /// Used for killing the export. - String query_id; + String transaction_id; FileAlreadyExistsPolicy file_already_exists_policy; bool parallel_formatting; /// parquet has a different setting for parallel formatting diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f2ed0aa22aa1..c5ee08a45343 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -118,6 +118,7 @@ #include #include +#include "Functions/generateSnowflakeID.h" #include "Interpreters/StorageID.h" #include "QueryPipeline/QueryPlanResourceHolder.h" #include "Storages/ExportReplicatedMergeTreePartitionManifest.h" @@ -8191,7 +8192,7 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & ExportReplicatedMergeTreePartitionManifest manifest; - manifest.transaction_id = query_context->getCurrentQueryId(); + manifest.transaction_id = generateSnowflakeIDString(); manifest.partition_id = partition_id; manifest.destination_database = dest_database; manifest.destination_table = dest_table; diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index f0103bee795b..3bf14409e82c 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -106,10 +106,10 @@ def test_restart_nodes_during_export(cluster): export_queries = f""" ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} - SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_max_retries = 50; + SETTINGS export_merge_tree_partition_max_retries = 50; ALTER TABLE {mt_table} EXPORT PARTITION ID '2021' TO TABLE {s3_table} - SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_max_retries = 50; + SETTINGS export_merge_tree_partition_max_retries = 50; """ node.query(export_queries) @@ -171,10 +171,10 @@ def test_kill_export(cluster): export_queries = f""" ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} - SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_max_retries = 50; + SETTINGS export_merge_tree_partition_max_retries = 50; ALTER TABLE {mt_table} EXPORT PARTITION ID '2021' TO TABLE {s3_table} - SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_max_retries = 50; + SETTINGS export_merge_tree_partition_max_retries = 50; """ node.query(export_queries) @@ -231,11 +231,9 @@ def test_drop_source_table_during_export(cluster): export_queries = f""" ALTER TABLE {mt_table} - EXPORT PARTITION ID '2020' TO TABLE {s3_table} - SETTINGS allow_experimental_export_merge_tree_part=1; + EXPORT PARTITION ID '2020' TO TABLE {s3_table}; ALTER TABLE {mt_table} - EXPORT PARTITION ID '2021' TO TABLE {s3_table} - SETTINGS allow_experimental_export_merge_tree_part=1; + EXPORT PARTITION ID '2021' TO TABLE {s3_table}; """ node.query(export_queries) @@ -264,10 +262,10 @@ def test_concurrent_exports_to_different_targets(cluster): pm.add_network_delay(node, delay_ms=1000) node.query( - f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table_a} SETTINGS allow_experimental_export_merge_tree_part=1;" + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table_a}" ) node.query( - f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table_b} SETTINGS allow_experimental_export_merge_tree_part=1;" + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table_b}" ) time.sleep(5) @@ -315,7 +313,7 @@ def test_failure_is_logged_in_system_table(cluster): pm._add_rule(pm_rule_reject_requests) node.query( - f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_max_retries=1;" + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_max_retries=1;" ) # Wait so that the export fails @@ -381,7 +379,7 @@ def test_inject_short_living_failures(cluster): # set big max_retries so that the export does not fail completely node.query( - f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_max_retries=100;" + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_max_retries=100;" ) # wait only for a second to get at least one failure, but not enough to finish the export @@ -426,10 +424,10 @@ def test_export_ttl(cluster): create_tables_and_insert_data(node, mt_table, s3_table, "replica1") # start export - node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_manifest_ttl={expiration_time};") + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_manifest_ttl={expiration_time};") # assert that I get an error when trying to export the same partition again, query_and_get_error - error = node.query_and_get_error(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1;") + error = node.query_and_get_error(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table};") assert "Export with key" in error, "Expected error about expired export" # wait for the export to finish and for the manifest to expire @@ -439,7 +437,7 @@ def test_export_ttl(cluster): assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '1\n', "Export did not succeed" # start export again - node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1") + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}") # wait for the export to finish time.sleep(expiration_time) @@ -460,15 +458,9 @@ def test_export_partition_file_already_exists_policy(cluster): # stop merges so part names remain stable. it is important for the test. node.query(f"SYSTEM STOP MERGES {mt_table}") - query_id_1 = uuid.uuid4().hex - query_id_2 = uuid.uuid4().hex - query_id_3 = uuid.uuid4().hex - query_id_4 = uuid.uuid4().hex - # Export all parts node.query( - f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1", - query_id=query_id_1, + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}", ) # check system.replicated_partition_exports for the export @@ -478,7 +470,6 @@ def test_export_partition_file_already_exists_policy(cluster): WHERE source_table = '{mt_table}' AND destination_table = '{s3_table}' AND partition_id = '2020' - AND transaction_id = '{query_id_1}' """ ) == "COMPLETED\n", "Export should be marked as COMPLETED" @@ -487,48 +478,45 @@ def test_export_partition_file_already_exists_policy(cluster): # try to export the partition node.query( - f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_force_export=1", - query_id=query_id_2, + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_force_export=1" ) time.sleep(3) - # check system.replicated_partition_exports for the export assert node.query( f""" - SELECT status FROM system.replicated_partition_exports + SELECT count() FROM system.replicated_partition_exports WHERE source_table = '{mt_table}' AND destination_table = '{s3_table}' AND partition_id = '2020' - AND transaction_id = '{query_id_2}' + AND status = 'COMPLETED' """ - ) == "COMPLETED\n", "Export should be marked as COMPLETED" + ) == '1\n', "Expected the export to be marked as COMPLETED" - # now let's try with a file exists policy that is not NO_OP + # overwrite policy node.query( - f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_force_export=1, export_merge_tree_part_file_already_exists_policy='overwrite'", - query_id=query_id_3, + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_force_export=1, export_merge_tree_part_file_already_exists_policy='overwrite'" ) # wait for the export to finish time.sleep(3) # check system.replicated_partition_exports for the export + # ideally we would make sure the transaction id is different, but I do not have the time to do that now assert node.query( f""" - SELECT status FROM system.replicated_partition_exports + SELECT count() FROM system.replicated_partition_exports WHERE source_table = '{mt_table}' AND destination_table = '{s3_table}' AND partition_id = '2020' - AND transaction_id = '{query_id_3}' + AND status = 'COMPLETED' """ - ) == "COMPLETED\n", "Export should be marked as COMPLETED" + ) == '1\n', "Expected the export to be marked as COMPLETED" # last but not least, let's try with the error policy # max retries = 1 so it fails fast node.query( - f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1, export_merge_tree_partition_force_export=1, export_merge_tree_part_file_already_exists_policy='error', export_merge_tree_partition_max_retries=1", - query_id=query_id_4, + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_force_export=1, export_merge_tree_part_file_already_exists_policy='error', export_merge_tree_partition_max_retries=1", ) # wait for the export to finish @@ -537,13 +525,13 @@ def test_export_partition_file_already_exists_policy(cluster): # check system.replicated_partition_exports for the export assert node.query( f""" - SELECT status FROM system.replicated_partition_exports + SELECT count() FROM system.replicated_partition_exports WHERE source_table = '{mt_table}' AND destination_table = '{s3_table}' AND partition_id = '2020' - AND transaction_id = '{query_id_4}' + AND status = 'FAILED' """ - ) == "FAILED\n", "Export should be marked as FAILED" + ) == '1\n', "Expected the export to be marked as FAILED" def test_export_partition_feature_is_disabled(cluster): @@ -554,7 +542,7 @@ def test_export_partition_feature_is_disabled(cluster): create_tables_and_insert_data(replica_with_export_disabled, mt_table, s3_table, "replica1") - error = replica_with_export_disabled.query_and_get_error(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1;") + error = replica_with_export_disabled.query_and_get_error(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table};") assert "experimental" in error, "Expected error about disabled feature" # make sure kill operation also throws @@ -594,8 +582,7 @@ def test_export_partition_permissions(cluster): # Test 1: User without ALTER permission should fail error = node.query_and_get_error( - f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " - f"SETTINGS allow_experimental_export_merge_tree_part=1", + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}", user="user_no_alter" ) assert "ACCESS_DENIED" in error or "Not enough privileges" in error, \ @@ -603,8 +590,7 @@ def test_export_partition_permissions(cluster): # Test 2: User with ALTER but without INSERT permission should fail error = node.query_and_get_error( - f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " - f"SETTINGS allow_experimental_export_merge_tree_part=1", + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}", user="user_no_insert" ) assert "ACCESS_DENIED" in error or "Not enough privileges" in error, \ @@ -612,8 +598,7 @@ def test_export_partition_permissions(cluster): # Test 3: User with both ALTER and INSERT should succeed node.query( - f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " - f"SETTINGS allow_experimental_export_merge_tree_part=1", + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}", user="user_with_permissions" ) @@ -636,6 +621,43 @@ def test_export_partition_permissions(cluster): assert status.strip() == "COMPLETED", f"Expected COMPLETED status, got: {status}" +# assert multiple exports within a single query are executed. They all share the same query id +# and previously the transaction id was the query id, which would cause problems +def test_multiple_exports_within_a_single_query(cluster): + node = cluster.instances["replica1"] + + mt_table = "multiple_exports_within_a_single_query_mt_table" + s3_table = "multiple_exports_within_a_single_query_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}, EXPORT PARTITION ID '2021' TO TABLE {s3_table};") + + time.sleep(5) + + # assert the exports have been executed + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") == '3\n', "Export did not succeed" + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2021") == '1\n', "Export did not succeed" + + # check system.replicated_partition_exports for the exports + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) == "COMPLETED\n", "Export should be marked as COMPLETED" + + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2021' + """ + ) == "COMPLETED\n", "Export should be marked as COMPLETED" + # def test_source_mutations_during_export_snapshot(cluster): # node = cluster.instances["replica1"] @@ -650,7 +672,7 @@ def test_export_partition_permissions(cluster): # # Start export of 2020 # node.query( -# f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS allow_experimental_export_merge_tree_part=1;" +# f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table};" # ) # # Mutate the source after export started (delete the same partition) From 3314f21e723e872226bd217a47dffb73647bb7fb Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 17 Nov 2025 11:14:07 -0300 Subject: [PATCH 57/58] fix possible dangling ref --- src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp index 6f7e4625823d..528841e21188 100644 --- a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -46,7 +46,7 @@ void ExportPartitionTaskScheduler::run() { const auto & manifest = entry.manifest; const auto key = entry.getCompositeKey(); - const auto & database = storage.getContext()->resolveDatabase(manifest.destination_database); + const auto database = storage.getContext()->resolveDatabase(manifest.destination_database); const auto & table = manifest.destination_table; /// No need to query zk for status if the local one is not PENDING From 59cd727cd8633e68557b1c03ce8b05a4fc252ebd Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 17 Nov 2025 11:21:10 -0300 Subject: [PATCH 58/58] use yet another tryget --- src/Storages/StorageReplicatedMergeTree.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c5ee08a45343..0b7c3c600e5f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4497,7 +4497,13 @@ std::vector StorageReplicatedMergeTree::getPartit const auto exceptions_per_replica_path = export_partition_path / "exceptions_per_replica"; - const auto exception_replicas = zk->getChildren(exceptions_per_replica_path); + Strings exception_replicas; + if (Coordination::Error::ZOK != zk->tryGetChildren(exceptions_per_replica_path, exception_replicas)) + { + LOG_INFO(log, "Skipping {}: missing exceptions_per_replica", export_partition_path); + continue; + } + for (const auto & replica : exception_replicas) { std::string exception_count_string;