From 9c0be2e76f99a35b99e5e4aaa886e8ae993f4545 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 28 Jul 2025 16:04:43 -0300 Subject: [PATCH 01/48] squash export mt part to obj storage --- src/Common/CurrentMetrics.cpp | 1 + src/Databases/DatabaseS3.cpp | 1 + src/Disks/ObjectStorages/IObjectStorage.h | 1 + src/Interpreters/Context.cpp | 3 + src/Interpreters/Context.h | 5 + src/Interpreters/InterpreterAlterQuery.cpp | 6 + src/Interpreters/PartLog.cpp | 4 +- src/Interpreters/PartLog.h | 1 + src/Parsers/ASTAlterQuery.cpp | 23 ++ src/Parsers/ASTAlterQuery.h | 1 + src/Parsers/CommonParsers.h | 1 + src/Parsers/ParserAlterQuery.cpp | 18 ++ src/Storages/ExportsList.cpp | 34 +++ src/Storages/ExportsList.h | 62 +++++ src/Storages/IPartitionStrategy.cpp | 17 +- src/Storages/IPartitionStrategy.h | 14 +- src/Storages/IStorage.h | 15 + src/Storages/MergeTree/MergeTreeData.cpp | 5 + src/Storages/MergeTree/MergeTreeData.h | 2 + src/Storages/MergeTree/MergeTreePartition.cpp | 16 ++ src/Storages/MergeTree/MergeTreePartition.h | 2 + .../DataLakes/DeltaLakeMetadataDeltaKernel.h | 2 + .../DataLakes/Iceberg/SchemaProcessor.h | 1 + .../ObjectStorage/FilePathGenerator.h | 60 ++++ ...geObjectStorageMergeTreePartImportSink.cpp | 96 +++++++ ...rageObjectStorageMergeTreePartImportSink.h | 38 +++ ...ObjectStorageSinkMTPartImportDecorator.cpp | 0 ...geObjectStorageSinkMTPartImportDecorator.h | 102 +++++++ .../ObjectStorageFilenameGenerator.h | 48 ++++ .../ObjectStorage/S3/Configuration.cpp | 3 + .../ObjectStorage/StorageObjectStorage.cpp | 168 ++++++++++- .../ObjectStorage/StorageObjectStorage.h | 11 +- .../StorageObjectStorageSink.cpp | 13 +- .../ObjectStorage/StorageObjectStorageSink.h | 8 +- src/Storages/PartitionCommands.cpp | 11 + src/Storages/PartitionCommands.h | 1 + src/Storages/PartitionStrategy.cpp | 261 ++++++++++++++++++ src/Storages/PartitionStrategy.h | 95 +++++++ src/Storages/PartitionedSink.cpp | 72 ++++- src/Storages/PartitionedSink.h | 32 ++- src/Storages/StorageFile.cpp | 21 +- src/Storages/StorageMergeTree.cpp | 78 ++++++ src/Storages/StorageMergeTree.h | 1 + src/Storages/StorageURL.cpp | 17 +- src/Storages/System/StorageSystemExports.cpp | 50 ++++ src/Storages/System/StorageSystemExports.h | 25 ++ src/Storages/System/StorageSystemMoves.cpp | 3 +- src/Storages/System/attachSystemTables.cpp | 3 + src/TableFunctions/ITableFunction.h | 1 + 49 files changed, 1397 insertions(+), 56 deletions(-) create mode 100644 src/Storages/ExportsList.cpp create mode 100644 src/Storages/ExportsList.h create mode 100644 src/Storages/ObjectStorage/FilePathGenerator.h create mode 100644 src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImportSink.cpp create mode 100644 src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImportSink.h create mode 100644 src/Storages/ObjectStorage/MergeTree/StorageObjectStorageSinkMTPartImportDecorator.cpp create mode 100644 src/Storages/ObjectStorage/MergeTree/StorageObjectStorageSinkMTPartImportDecorator.h create mode 100644 src/Storages/ObjectStorage/ObjectStorageFilenameGenerator.h create mode 100644 src/Storages/PartitionStrategy.cpp create mode 100644 src/Storages/PartitionStrategy.h create mode 100644 src/Storages/System/StorageSystemExports.cpp create mode 100644 src/Storages/System/StorageSystemExports.h diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index ea3bd714930f..97bc7d309ac5 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -10,6 +10,7 @@ M(Merge, "Number of executing background merges") \ M(MergeParts, "Number of source parts participating in current background merges") \ M(Move, "Number of currently executing moves") \ + M(Export, "Number of currently executing exports") \ M(PartMutation, "Number of mutations (ALTER DELETE/UPDATE)") \ M(ReplicatedFetch, "Number of data parts being fetched from replica") \ M(ReplicatedSend, "Number of data parts being sent to replicas") \ diff --git a/src/Databases/DatabaseS3.cpp b/src/Databases/DatabaseS3.cpp index 2a42aa744909..2b657c75da96 100644 --- a/src/Databases/DatabaseS3.cpp +++ b/src/Databases/DatabaseS3.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 0d9464b1ad7e..824bc0a52f25 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -96,6 +96,7 @@ struct RelativePathWithMetadata virtual ~RelativePathWithMetadata() = default; virtual std::string getFileName() const { return std::filesystem::path(relative_path).filename(); } + virtual std::string getFileNameWithoutExtension() const { return std::filesystem::path(relative_path).stem(); } virtual std::string getPath() const { return relative_path; } virtual bool isArchive() const { return false; } virtual std::string getPathToArchive() const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not an archive"); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index db58e2352267..efec76e59ee5 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -463,6 +463,7 @@ struct ContextSharedPart : boost::noncopyable GlobalOvercommitTracker global_overcommit_tracker; MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) MovesList moves_list; /// The list of executing moves (for (Replicated)?MergeTree) + ExportsList exports_list; /// The list of executing exports MergeTree -> Object storage ReplicatedFetchList replicated_fetch_list; RefreshSet refresh_set; /// The list of active refreshes (for MaterializedView) ConfigurationPtr users_config TSA_GUARDED_BY(mutex); /// Config with the users, profiles and quotas sections. @@ -1148,6 +1149,8 @@ MergeList & Context::getMergeList() { return shared->merge_list; } const MergeList & Context::getMergeList() const { return shared->merge_list; } MovesList & Context::getMovesList() { return shared->moves_list; } const MovesList & Context::getMovesList() const { return shared->moves_list; } +ExportsList & Context::getExportsList() { return shared->exports_list; } +const ExportsList & Context::getExportsList() const { return shared->exports_list; } ReplicatedFetchList & Context::getReplicatedFetchList() { return shared->replicated_fetch_list; } const ReplicatedFetchList & Context::getReplicatedFetchList() const { return shared->replicated_fetch_list; } RefreshSet & Context::getRefreshSet() { return shared->refresh_set; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f26899ea3c72..5f5ed675112c 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -32,6 +32,8 @@ #include #include +#include "Storages/ExportsList.h" + namespace Poco::Net { @@ -1141,6 +1143,9 @@ class Context: public ContextData, public std::enable_shared_from_this MovesList & getMovesList(); const MovesList & getMovesList() const; + ExportsList & getExportsList(); + const ExportsList & getExportsList() const; + ReplicatedFetchList & getReplicatedFetchList(); const ReplicatedFetchList & getReplicatedFetchList() const; diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 262b4ce13cfd..f407a17aeedf 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -502,6 +502,12 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS required_access.emplace_back(AccessType::ALTER_DELETE | AccessType::INSERT, database, table); break; } + case ASTAlterCommand::EXPORT_PART: + { + required_access.emplace_back(AccessType::ALTER_MOVE_PARTITION, database, 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/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index 02c6f6e573b0..adec9baf659e 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -69,6 +69,7 @@ ColumnsDescription PartLogElement::getColumnsDescription() {"MovePart", static_cast(MOVE_PART)}, {"MergePartsStart", static_cast(MERGE_PARTS_START)}, {"MutatePartStart", static_cast(MUTATE_PART_START)}, + {"ExportPart", static_cast(EXPORT_PART)}, } ); @@ -109,7 +110,8 @@ ColumnsDescription PartLogElement::getColumnsDescription() "RemovePart — Removing or detaching a data part using [DETACH PARTITION](/sql-reference/statements/alter/partition#detach-partitionpart)." "MutatePartStart — Mutating of a data part has started, " "MutatePart — Mutating of a data part has finished, " - "MovePart — Moving the data part from the one disk to another one."}, + "MovePart — Moving the data part from the one disk to another one." + "ExportPart — Exporting the data part from a merge tree table to one (e.g, object storage)."}, {"merge_reason", std::move(merge_reason_datatype), "The reason for the event with type MERGE_PARTS. Can have one of the following values: " "NotAMerge — The current event has the type other than MERGE_PARTS, " diff --git a/src/Interpreters/PartLog.h b/src/Interpreters/PartLog.h index 44d2fb413c5f..4f58069dae55 100644 --- a/src/Interpreters/PartLog.h +++ b/src/Interpreters/PartLog.h @@ -30,6 +30,7 @@ struct PartLogElement MOVE_PART = 6, MERGE_PARTS_START = 7, MUTATE_PART_START = 8, + EXPORT_PART = 9, }; /// Copy of MergeAlgorithm since values are written to disk. diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index cdf8b558fd61..ab9d8cd1bf33 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -355,6 +355,29 @@ void ASTAlterCommand::formatImpl(WriteBuffer & ostr, const FormatSettings & sett ostr << quoteString(move_destination_name); } } + else if (type == ASTAlterCommand::EXPORT_PART) + { + ostr << (settings.hilite ? hilite_keyword : "") << "EXPORT " << (part ? "PART " : "PARTITION ") + << (settings.hilite ? hilite_none : ""); + partition->format(ostr, settings, state, frame); + ostr << " TO "; + switch (move_destination_type) + { + case DataDestinationType::TABLE: + ostr << "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 : ""); + return; + default: + break; + } + + } else if (type == ASTAlterCommand::REPLACE_PARTITION) { ostr << (settings.hilite ? hilite_keyword : "") << (replace ? "REPLACE" : "ATTACH") << " PARTITION " diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 3867a86cf797..e6f8a37997ba 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -71,6 +71,7 @@ class ASTAlterCommand : public IAST FREEZE_ALL, UNFREEZE_PARTITION, UNFREEZE_ALL, + EXPORT_PART, DELETE, UPDATE, diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index e4db7beb9d4e..a634df26603a 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -331,6 +331,7 @@ namespace DB MR_MACROS(MONTHS, "MONTHS") \ MR_MACROS(MOVE_PART, "MOVE PART") \ MR_MACROS(MOVE_PARTITION, "MOVE PARTITION") \ + 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 2c127e6ff1e1..ace9bbbc566e 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -82,6 +82,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_forget_partition(Keyword::FORGET_PARTITION); ParserKeyword s_move_partition(Keyword::MOVE_PARTITION); ParserKeyword s_move_part(Keyword::MOVE_PART); + ParserKeyword s_export_part(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); @@ -564,6 +565,23 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->move_destination_name = ast_space_name->as().value.safeGet(); } } + else if (s_export_part.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command_partition, expected)) + return false; + + command->type = ASTAlterCommand::EXPORT_PART; + // command->part = true; + + 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_add_constraint.ignore(pos, expected)) { if (s_if_not_exists.ignore(pos, expected)) diff --git a/src/Storages/ExportsList.cpp b/src/Storages/ExportsList.cpp new file mode 100644 index 000000000000..b1c5afebbb7b --- /dev/null +++ b/src/Storages/ExportsList.cpp @@ -0,0 +1,34 @@ +#include +#include "base/getThreadId.h" + +namespace DB +{ + +ExportsListElement::ExportsListElement( + const StorageID & source_table_id_, + const StorageID & destination_table_id_, + const std::string & part_name_, + const std::string & destination_path_) +: source_table_id(source_table_id_), + destination_table_id(destination_table_id_), + part_name(part_name_), + destination_path(destination_path_), + thread_id(getThreadId()) +{ +} + +ExportInfo ExportsListElement::getInfo() const +{ + ExportInfo res; + res.source_database = source_table_id.database_name; + res.source_table = source_table_id.table_name; + res.destination_database = destination_table_id.database_name; + res.destination_table = destination_table_id.table_name; + res.part_name = part_name; + res.destination_path = destination_path; + res.elapsed = watch.elapsedSeconds(); + res.thread_id = thread_id; + return res; +} + +} diff --git a/src/Storages/ExportsList.h b/src/Storages/ExportsList.h new file mode 100644 index 000000000000..053b89aeba2c --- /dev/null +++ b/src/Storages/ExportsList.h @@ -0,0 +1,62 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace CurrentMetrics +{ +extern const Metric Export; +} + +namespace DB +{ + +struct ExportInfo +{ + std::string source_database; + std::string destination_database; + std::string source_table; + std::string destination_table; + std::string part_name; + std::string destination_path; + + Float64 elapsed; + UInt64 thread_id; +}; + +struct ExportsListElement : private boost::noncopyable +{ + const StorageID source_table_id; + const StorageID destination_table_id; + const std::string part_name; + const std::string destination_path; + + Stopwatch watch; + const UInt64 thread_id; + + ExportsListElement( + const StorageID & source_table_id_, + const StorageID & destination_table_id_, + const std::string & part_name_, + const std::string & destination_path_); + + ExportInfo getInfo() const; +}; + + +/// List of currently processing moves +class ExportsList final : public BackgroundProcessList +{ +private: + using Parent = BackgroundProcessList; + +public: + ExportsList() + : Parent(CurrentMetrics::Export) + {} +}; + +} diff --git a/src/Storages/IPartitionStrategy.cpp b/src/Storages/IPartitionStrategy.cpp index 5ebdb249e661..676434869ce5 100644 --- a/src/Storages/IPartitionStrategy.cpp +++ b/src/Storages/IPartitionStrategy.cpp @@ -332,32 +332,27 @@ ColumnPtr HiveStylePartitionStrategy::computePartitionKey(const Chunk & chunk) return block_with_partition_by_expr.getByName(actions_with_column_name.column_name).column; } -ColumnRawPtrs HiveStylePartitionStrategy::getFormatChunkColumns(const Chunk & chunk) +Chunk HiveStylePartitionStrategy::getFormatChunk(const Chunk & chunk) { - ColumnRawPtrs result; + Chunk result; + if (partition_columns_in_data_file) { for (const auto & column : chunk.getColumns()) { - result.emplace_back(column.get()); + result.addColumn(column); } return result; } - if (chunk.getNumColumns() != sample_block.columns()) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Incorrect number of columns in chunk. Expected {}, found {}", - sample_block.columns(), chunk.getNumColumns()); - } + chassert(chunk.getColumns().size() == sample_block.columns()); for (size_t i = 0; i < sample_block.columns(); i++) { if (!partition_columns_name_set.contains(sample_block.getByPosition(i).name)) { - result.emplace_back(chunk.getColumns()[i].get()); + result.addColumn(chunk.getColumns()[i]); } } diff --git a/src/Storages/IPartitionStrategy.h b/src/Storages/IPartitionStrategy.h index bc90d7f03461..2f053013a516 100644 --- a/src/Storages/IPartitionStrategy.h +++ b/src/Storages/IPartitionStrategy.h @@ -32,17 +32,7 @@ struct IPartitionStrategy virtual std::string getPathForRead(const std::string & prefix) = 0; virtual std::string getPathForWrite(const std::string & prefix, const std::string & partition_key) = 0; - virtual ColumnRawPtrs getFormatChunkColumns(const Chunk & chunk) - { - ColumnRawPtrs result_columns; - - for (const auto & column : chunk.getColumns()) - { - result_columns.emplace_back(column.get()); - } - - return result_columns; - } + virtual Chunk getFormatChunk(const Chunk & chunk) { return chunk.clone(); } virtual Block getFormatHeader() { return sample_block; } @@ -114,7 +104,7 @@ struct HiveStylePartitionStrategy : IPartitionStrategy std::string getPathForRead(const std::string & prefix) override; std::string getPathForWrite(const std::string & prefix, const std::string & partition_key) override; - ColumnRawPtrs getFormatChunkColumns(const Chunk & chunk) override; + Chunk getFormatChunk(const Chunk & chunk) override; Block getFormatHeader() override; private: diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 006b9e377f4b..47cb9eeace1b 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -18,9 +18,12 @@ #include #include #include +#include #include +#include "MergeTree/RangesInDataPart.h" + namespace DB { @@ -67,6 +70,12 @@ class ConditionSelectivityEstimator; class ActionsDAG; +class MergeTreeData; + +class IMergeTreeDataPart; + +struct MergeTreePartImportStats; + /** Storage. Describes the table. Responsible for * - storage of the table data; * - the definition in which files (or not in files) the data is stored; @@ -436,6 +445,12 @@ class IStorage : public std::enable_shared_from_this, public TypePromo ContextPtr /*context*/, bool /*async_insert*/); + virtual void importMergeTreePartition( + const MergeTreeData &, + const std::vector &, + ContextPtr /*context*/, + std::function) {} + /** Writes the data to a table in distributed manner. * It is supposed that implementation looks into SELECT part of the query and executes distributed * INSERT SELECT if it is possible with current storage as a receiver and query SELECT part as a producer. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index aa3ac075a5a7..305b828371f1 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5940,6 +5940,11 @@ Pipe MergeTreeData::alterPartition( } } break; + case PartitionCommand::EXPORT_PART: + { + exportPartitionToTable(command, query_context); + break; + } case PartitionCommand::DROP_DETACHED_PARTITION: dropDetached(command.partition, command.part, query_context); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 89d3507266e0..5c524a2f4007 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -901,6 +901,8 @@ class MergeTreeData : public IStorage, public WithMutableContext /// Moves partition to specified Table void movePartitionToTable(const PartitionCommand & command, ContextPtr query_context); + virtual void exportPartitionToTable(const PartitionCommand &, ContextPtr) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "export not implemented");} + /// 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/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 445fc8846da3..19c33cbca5ed 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -479,6 +479,22 @@ void MergeTreePartition::create(const StorageMetadataPtr & metadata_snapshot, Bl } } +Block MergeTreePartition::getBlockWithPartitionValues(const NamesAndTypesList & partition_columns) const +{ + chassert(partition_columns.size() == value.size()); + + Block result; + + std::size_t i = 0; + for (const auto & partition_column : partition_columns) + { + auto column = partition_column.type->createColumnConst(1, value[i++]); + result.insert({column, partition_column.type, partition_column.name}); + } + + return result; +} + NamesAndTypesList MergeTreePartition::executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context) { auto adjusted_partition_key = adjustPartitionKey(metadata_snapshot, context); diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index 4338b216cdb8..811cfdc2a90c 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -60,6 +60,8 @@ struct MergeTreePartition void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row, ContextPtr context); + Block getBlockWithPartitionValues(const NamesAndTypesList & partition_columns) const; + /// Adjust partition key and execute its expression on block. Return sample block according to used expression. static NamesAndTypesList executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context); diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h index 679214d5c489..bc10960fa64d 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h @@ -29,6 +29,8 @@ class DeltaLakeMetadataDeltaKernel final : public IDataLakeMetadata ObjectStoragePtr object_storage_, ConfigurationObserverPtr configuration_); + DeltaLakeMetadataDeltaKernel(const DeltaLakeMetadataDeltaKernel & other) : log(other.log), table_snapshot(other.table_snapshot) {} + bool supportsUpdate() const override { return true; } bool update(const ContextPtr & context) override; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h index 41c54feb0ee7..107d6c950864 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h @@ -78,6 +78,7 @@ class IcebergSchemaProcessor using Node = ActionsDAG::Node; public: + void addIcebergTableSchema(Poco::JSON::Object::Ptr schema_ptr); std::shared_ptr getClickhouseTableSchemaById(Int32 id); std::shared_ptr getSchemaTransformationDagByIds(Int32 old_id, Int32 new_id); diff --git a/src/Storages/ObjectStorage/FilePathGenerator.h b/src/Storages/ObjectStorage/FilePathGenerator.h new file mode 100644 index 000000000000..bc1d5d929829 --- /dev/null +++ b/src/Storages/ObjectStorage/FilePathGenerator.h @@ -0,0 +1,60 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + struct ObjectStorageFilePathGenerator + { + virtual ~ObjectStorageFilePathGenerator() = default; + virtual std::string getWritingPath(const std::string & partition_id, std::optional filename_override = {}) const = 0; + virtual std::string getReadingPath() const = 0; + }; + + struct ObjectStorageWildcardFilePathGenerator : ObjectStorageFilePathGenerator + { + explicit ObjectStorageWildcardFilePathGenerator(const std::string & raw_path_) : raw_path(raw_path_) {} + + std::string getWritingPath(const std::string & partition_id, std::optional /**/ = {}) const override + { + return PartitionedSink::replaceWildcards(raw_path, partition_id); + } + + std::string getReadingPath() const override + { + return raw_path; + } + + private: + std::string raw_path; + + }; + + struct ObjectStorageAppendFilePathGenerator : ObjectStorageFilePathGenerator + { + explicit ObjectStorageAppendFilePathGenerator( + const std::string & raw_path_, + const std::string & file_format_, + const std::shared_ptr & filename_generator_) + : raw_path(raw_path_), file_format(file_format_), filename_generator(filename_generator_){} + + std::string getWritingPath(const std::string & partition_id, std::optional filename_override) const override + { + return raw_path + "/" + partition_id + "/" + (filename_override ? *filename_override : filename_generator->generate()) + "." + file_format; + } + + std::string getReadingPath() const override + { + return raw_path + "**." + file_format; + } + + private: + std::string raw_path; + std::string file_format; + + std::shared_ptr filename_generator; + }; + +} diff --git a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImportSink.cpp b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImportSink.cpp new file mode 100644 index 000000000000..0c729daf76f8 --- /dev/null +++ b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImportSink.cpp @@ -0,0 +1,96 @@ +#include +#include +#include "Core/Settings.h" +#include "Disks/ObjectStorages/IObjectStorage.h" +#include "Disks/ObjectStorages/StoredObject.h" +#include "Formats/FormatFactory.h" +#include "IO/CompressionMethod.h" +#include "Processors/Formats/IOutputFormat.h" + +namespace DB +{ + +namespace Setting +{ +extern const SettingsUInt64 output_format_compression_level; +extern const SettingsUInt64 output_format_compression_zstd_window_log; +} + + +StorageObjectStorageMergeTreePartSink::StorageObjectStorageMergeTreePartSink( + const DataPartPtr & part, + ObjectStoragePtr object_storage, + const std::string & format, + const std::string & compression_method, + const std::optional & format_settings_, + const Block & sample_block_, + ContextPtr context) +: SinkToStorage(sample_block_) +{ + const auto & settings = context->getSettingsRef(); + const auto chosen_compression_method = chooseCompressionMethod(path, compression_method); + + auto buffer = object_storage->writeObject( + StoredObject(path), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, context->getWriteSettings()); + + write_buf = wrapWriteBufferWithCompressionMethod( + std::move(buffer), + chosen_compression_method, + static_cast(settings[Setting::output_format_compression_level]), + static_cast(settings[Setting::output_format_compression_zstd_window_log])); + + writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, format_settings_); +} + +void StorageObjectStorageMergeTreePartSink::consume(Chunk & chunk) +{ + if (isCancelled()) + return; + writer->write(getHeader().cloneWithColumns(chunk.getColumns())); +} + +void StorageObjectStorageMergeTreePartSink::onFinish() +{ + if (isCancelled()) + return; + + finalizeBuffers(); + releaseBuffers(); +} + +void StorageObjectStorageMergeTreePartSink::finalizeBuffers() +{ + if (!writer) + return; + + try + { + writer->flush(); + writer->finalize(); + } + catch (...) + { + /// Stop ParallelFormattingOutputFormat correctly. + cancelBuffers(); + releaseBuffers(); + throw; + } + + write_buf->finalize(); +} + +void StorageObjectStorageMergeTreePartSink::releaseBuffers() +{ + writer.reset(); + write_buf.reset(); +} + +void StorageObjectStorageMergeTreePartSink::cancelBuffers() +{ + if (writer) + writer->cancel(); + if (write_buf) + write_buf->cancel(); +} + +} diff --git a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImportSink.h b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImportSink.h new file mode 100644 index 000000000000..80aa6c19dea1 --- /dev/null +++ b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImportSink.h @@ -0,0 +1,38 @@ +#pragma once + +#include +#include +#include "Storages/MergeTree/RangesInDataPart.h" + +namespace DB +{ +class StorageObjectStorageMergeTreePartSink : public SinkToStorage +{ +public: + StorageObjectStorageMergeTreePartSink( + const DataPartPtr & part, + ObjectStoragePtr object_storage, + const std::string & format, + const std::string & compression_method, + const std::optional & format_settings_, + const Block & sample_block_, + ContextPtr context); + + String getName() const override { return "StorageObjectStorageMergeTreePartSink"; } + + void consume(Chunk & chunk) override; + + void onFinish() override; + +private: + const String path; + const Block sample_block; + std::unique_ptr write_buf; + OutputFormatPtr writer; + + void finalizeBuffers(); + void releaseBuffers(); + void cancelBuffers(); +}; + +} diff --git a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageSinkMTPartImportDecorator.cpp b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageSinkMTPartImportDecorator.cpp new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageSinkMTPartImportDecorator.h b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageSinkMTPartImportDecorator.h new file mode 100644 index 000000000000..5d0f3a0e45f7 --- /dev/null +++ b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageSinkMTPartImportDecorator.h @@ -0,0 +1,102 @@ +#pragma once + +#include +#include +#include "Core/Settings.h" +#include "Disks/ObjectStorages/IObjectStorage.h" +#include "Disks/ObjectStorages/StoredObject.h" +#include "Formats/FormatFactory.h" +#include "IO/CompressionMethod.h" +#include "Processors/Formats/IOutputFormat.h" +#include "Storages/MergeTree/IMergeTreeDataPart.h" + +namespace DB +{ + +struct MergeTreePartImportStats +{ + ExecutionStatus status; + std::size_t bytes_on_disk = 0; + std::size_t read_rows = 0; + std::size_t read_bytes = 0; + std::string file_path = ""; + DataPartPtr part = nullptr; +}; + +class StorageObjectStorageSinkMTPartImportDecorator : public SinkToStorage +{ +public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + + StorageObjectStorageSinkMTPartImportDecorator( + const DataPartPtr & part_, + const std::string & path_, + const ObjectStoragePtr & object_storage_, + const ConfigurationPtr & configuration_, + const std::optional & format_settings_, + const Block & sample_block_, + const std::function & part_log_, + const ContextPtr & context_) + : SinkToStorage(sample_block_) + , object_storage(object_storage_) + , configuration(configuration_) + , format_settings(format_settings_) + , sample_block(sample_block_) + , context(context_) + , part_log(part_log_) + { + stats.part = part_; + stats.file_path = path_; + } + + String getName() const override { return "StorageObjectStorageSinkMTPartImportDecorator"; } + + void consume(Chunk & chunk) override + { + if (!sink) + { + sink = std::make_shared( + stats.file_path, + object_storage, + configuration, + format_settings, + sample_block, + context + ); + } + + stats.read_bytes += chunk.bytes(); + stats.read_rows += chunk.getNumRows(); + + sink->consume(chunk); + } + + void onFinish() override + { + sink->onFinish(); + if (const auto object_metadata = object_storage->tryGetObjectMetadata(stats.file_path)) + { + stats.bytes_on_disk = object_metadata->size_bytes; + } + part_log(stats); + } + + void onException(std::exception_ptr exception) override + { + SinkToStorage::onException(exception); + part_log(stats); + } + +private: + std::shared_ptr sink; + ObjectStoragePtr object_storage; + ConfigurationPtr configuration; + std::optional format_settings; + Block sample_block; + ContextPtr context; + std::function part_log; + + MergeTreePartImportStats stats; +}; + +} diff --git a/src/Storages/ObjectStorage/ObjectStorageFilenameGenerator.h b/src/Storages/ObjectStorage/ObjectStorageFilenameGenerator.h new file mode 100644 index 000000000000..0dcdb26b719f --- /dev/null +++ b/src/Storages/ObjectStorage/ObjectStorageFilenameGenerator.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include + +namespace DB +{ + +struct ObjectStorageFilenameGenerator +{ + virtual ~ObjectStorageFilenameGenerator() = default; + virtual std::string generate() const = 0; +}; + +struct NoOpObjectStorageFilenameGenerator : ObjectStorageFilenameGenerator +{ + std::string generate() const override + { + return ""; + } +}; + +struct SnowflakeObjectStorageFilenameGenerator : ObjectStorageFilenameGenerator +{ + std::string generate() const override + { + return std::to_string(generateSnowflakeID()); + } + +private: + std::string file_format; +}; + +struct PredefinedObjectStorageFilenameGenerator : public ObjectStorageFilenameGenerator +{ + explicit PredefinedObjectStorageFilenameGenerator(const std::string & filename_) + : file_name(filename_) {} + + std::string generate() const override + { + return file_name; + } + +private: + std::string file_name; +}; + +} diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index dc552fad8518..fb7d62d054f8 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -168,6 +168,9 @@ void StorageS3Configuration::fromNamedCollection(const NamedCollection & collect else url = S3::URI(collection.get("url"), settings[Setting::allow_archive_path_syntax]); + partition_strategy_name = collection.getOrDefault("partition_strategy", "wildcard"); + partition_columns_in_data_file = collection.getOrDefault("partition_columns_in_data_file", partition_strategy_name != "hive"); + auth_settings[S3AuthSetting::access_key_id] = collection.getOrDefault("access_key_id", ""); auth_settings[S3AuthSetting::secret_access_key] = collection.getOrDefault("secret_access_key", ""); auth_settings[S3AuthSetting::use_environment_credentials] = collection.getOrDefault("use_environment_credentials", 1); diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index f910691767aa..001a75b8fc49 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -1,5 +1,7 @@ #include +#include #include +#include "MergeTree/StorageObjectStorageSinkMTPartImportDecorator.h" #include #include @@ -29,8 +31,13 @@ #include #include #include +#include #include +#include +#include +#include +#include namespace DB { @@ -538,6 +545,152 @@ void StorageObjectStorage::read( query_plan.addStep(std::move(read_step)); } +void StorageObjectStorage::importMergeTreePartition( + const MergeTreeData & merge_tree_data, + const std::vector & data_parts, + ContextPtr local_context, + std::function part_log) +{ + if (data_parts.empty()) + return; + + RelativePathsWithMetadata relative_paths_with_metadata; + object_storage->listObjects("", relative_paths_with_metadata, 1000); + + std::vector part_plans; + part_plans.reserve(data_parts.size()); + + auto metadata_snapshot = merge_tree_data.getInMemoryMetadataPtr(); + Names columns_to_read = metadata_snapshot->getColumns().getNamesOfPhysical(); + StorageSnapshotPtr storage_snapshot = merge_tree_data.getStorageSnapshot(metadata_snapshot, local_context); + + QueryPlan plan; + + // todo arthur + MergeTreeSequentialSourceType read_type = MergeTreeSequentialSourceType::Merge; + + bool apply_deleted_mask = true; + bool read_with_direct_io = false; + bool prefetch = false; + + QueryPipeline root_pipeline; + + std::vector export_list_entries; + + std::vector files_to_be_deleted; + for (const auto & data_part : data_parts) + { + bool upload_part = true; + for (const auto & object_with_metadata : relative_paths_with_metadata) + { + const auto remote_object_filename = object_with_metadata->getFileNameWithoutExtension(); + if (remote_object_filename == data_part->name) + { + upload_part = false; + break; + } + + const auto remote_fake_part = MergeTreePartInfo::tryParsePartName(remote_object_filename, merge_tree_data.format_version); + + if (!remote_fake_part) + { + continue; + } + + /// If the part does not intersect, proceed to the next file + if (data_part->info.isDisjoint(remote_fake_part.value())) + { + continue; + } + + files_to_be_deleted.emplace_back(object_with_metadata->relative_path); + } + + if (!upload_part) + { + continue; + } + + const auto partition_columns = configuration->partition_strategy->getPartitionColumns(); + + auto block_with_partition_values = data_part->partition.getBlockWithPartitionValues(partition_columns); + + const auto column_with_partition_key = configuration->partition_strategy->computePartitionKey(block_with_partition_values); + + const auto file_path = configuration->file_path_generator->getWritingPath(column_with_partition_key->getDataAt(0).toString(), data_part->name); + + export_list_entries.emplace_back(local_context->getGlobalContext()->getExportsList().insert( + merge_tree_data.getStorageID(), + getStorageID(), + data_part->name, + file_path + )); + + MergeTreeData::IMutationsSnapshot::Params params + { + .metadata_version = metadata_snapshot->getMetadataVersion(), + .min_part_metadata_version = data_part->getMetadataVersion(), + }; + + auto mutations_snapshot = merge_tree_data.getMutationsSnapshot(params); + + auto alter_conversions = MergeTreeData::getAlterConversionsForPart( + data_part, + mutations_snapshot, + local_context); + + QueryPlan plan_for_part; + + createReadFromPartStep( + read_type, + plan_for_part, + merge_tree_data, + storage_snapshot, + RangesInDataPart(data_part), + alter_conversions, + nullptr, + columns_to_read, + nullptr, + apply_deleted_mask, + std::nullopt, + read_with_direct_io, + prefetch, + local_context, + getLogger("ExportPartition")); + + QueryPlanOptimizationSettings optimization_settings(local_context); + auto pipeline_settings = BuildQueryPipelineSettings(local_context); + auto builder = plan_for_part.buildQueryPipeline(optimization_settings, pipeline_settings); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + + auto sink = std::make_shared( + data_part, + file_path, + object_storage, + configuration, + format_settings, + metadata_snapshot->getSampleBlock(), + part_log, + local_context + ); + + pipeline.complete(sink); + + root_pipeline.addCompletedPipeline(std::move(pipeline)); + } + + if (root_pipeline.completed()) + { + root_pipeline.setNumThreads(local_context->getSettingsRef()[Setting::max_threads]); + + /// shouldn't this be part of the sink and or pipeline? + object_storage->removeObjectsIfExist(files_to_be_deleted); + + CompletedPipelineExecutor exec(root_pipeline); + exec.execute(); + } +} + SinkToStoragePtr StorageObjectStorage::write( const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, @@ -577,7 +730,8 @@ SinkToStoragePtr StorageObjectStorage::write( if (configuration->partition_strategy) { - return std::make_shared(object_storage, configuration, format_settings, sample_block, local_context); + auto sink_creator = std::make_shared(object_storage, configuration, configuration->file_path_generator, format_settings, sample_block, local_context); + return std::make_shared(configuration->partition_strategy, sink_creator, local_context, sample_block); } auto paths = configuration->getPaths(); @@ -757,6 +911,18 @@ void StorageObjectStorage::Configuration::initialize( configuration_to_initialize.partition_strategy_type = PartitionStrategyFactory::StrategyType::WILDCARD; } + if (configuration_to_initialize.partition_strategy_type == PartitionStrategyFactory::StrategyType::HIVE) + { + configuration_to_initialize.file_path_generator = std::make_shared( + configuration_to_initialize.getRawPath().path, + configuration_to_initialize.format, + std::make_shared()); + } + else + { + configuration_to_initialize.file_path_generator = std::make_shared(configuration_to_initialize.getRawPath().path); + } + if (configuration_to_initialize.format == "auto") { if (configuration_to_initialize.isDataLakeConfiguration()) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 0c75431cd1c1..ff7aa493e742 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -16,7 +16,7 @@ #include -#include +#include namespace DB { @@ -98,6 +98,12 @@ class StorageObjectStorage : public IStorage ContextPtr context, bool async_insert) override; + void importMergeTreePartition( + const MergeTreeData & merge_tree_data, + const std::vector & data_parts, + ContextPtr /*context*/, + std::function part_log) override; + void truncate( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, @@ -311,6 +317,9 @@ class StorageObjectStorage::Configuration bool if_not_updated_before, bool check_consistent_with_previous_metadata); + void initPartitionStrategy(ASTPtr partition_by, const ColumnsDescription & columns, ContextPtr context); + + const StorageObjectStorageSettings & getSettingsRef() const; virtual const DataLakeStorageSettings & getDataLakeSettings() const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getDataLakeSettings() is not implemented for configuration type {}", getTypeName()); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index e23787b5aa6b..7eb24fa85d6c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -130,16 +130,19 @@ void StorageObjectStorageSink::cancelBuffers() PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( ObjectStoragePtr object_storage_, ConfigurationPtr configuration_, + const std::shared_ptr & file_path_generator_, std::optional format_settings_, const Block & sample_block_, - ContextPtr context_) - : PartitionedSink(configuration_->partition_strategy, context_, sample_block_) - , object_storage(object_storage_) + ContextPtr context_, + std::optional filename_override_) + : object_storage(object_storage_) , configuration(configuration_) + , file_path_generator(file_path_generator_) , query_settings(configuration_->getQuerySettings(context_)) , format_settings(format_settings_) , sample_block(sample_block_) , context(context_) + , filename_override(filename_override_) { } @@ -151,7 +154,7 @@ StorageObjectStorageSink::~StorageObjectStorageSink() SinkPtr PartitionedStorageObjectStorageSink::createSinkForPartition(const String & partition_id) { - auto file_path = configuration->getPathForWrite(partition_id).path; + auto file_path = file_path_generator->getWritingPath(partition_id, filename_override); validateNamespace(configuration->getNamespace(), configuration); validateKey(file_path); @@ -167,7 +170,7 @@ SinkPtr PartitionedStorageObjectStorageSink::createSinkForPartition(const String object_storage, configuration, format_settings, - partition_strategy->getFormatHeader(), + configuration->partition_strategy->getFormatHeader(), context ); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h index ebfee5ab96e6..d515d9c6137a 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -38,7 +38,7 @@ class StorageObjectStorageSink : public SinkToStorage void cancelBuffers(); }; -class PartitionedStorageObjectStorageSink : public PartitionedSink +class PartitionedStorageObjectStorageSink : public PartitionedSink::SinkCreator { public: using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; @@ -46,20 +46,24 @@ class PartitionedStorageObjectStorageSink : public PartitionedSink PartitionedStorageObjectStorageSink( ObjectStoragePtr object_storage_, ConfigurationPtr configuration_, + const std::shared_ptr & file_path_generator_, std::optional format_settings_, const Block & sample_block_, - ContextPtr context_); + ContextPtr context_, + std::optional filename_override_ = std::nullopt); SinkPtr createSinkForPartition(const String & partition_id) override; private: ObjectStoragePtr object_storage; ConfigurationPtr configuration; + std::shared_ptr file_path_generator; const StorageObjectStorage::QuerySettings query_settings; const std::optional format_settings; const Block sample_block; const ContextPtr context; + std::optional filename_override; }; } diff --git a/src/Storages/PartitionCommands.cpp b/src/Storages/PartitionCommands.cpp index 41cc03552458..0d86bdc5802f 100644 --- a/src/Storages/PartitionCommands.cpp +++ b/src/Storages/PartitionCommands.cpp @@ -52,6 +52,17 @@ std::optional PartitionCommand::parse(const ASTAlterCommand * res.part = command_ast->part; return res; } + if (command_ast->type == ASTAlterCommand::EXPORT_PART) + { + PartitionCommand res; + res.type = EXPORT_PART; + res.partition = command_ast->partition->clone(); + res.part = command_ast->part; + res.move_destination_type = PartitionCommand::MoveDestinationType::TABLE; + res.to_database = command_ast->to_database; + res.to_table = command_ast->to_table; + return res; + } if (command_ast->type == ASTAlterCommand::MOVE_PARTITION) { PartitionCommand res; diff --git a/src/Storages/PartitionCommands.h b/src/Storages/PartitionCommands.h index 917e510f24b4..15d2a7fb869f 100644 --- a/src/Storages/PartitionCommands.h +++ b/src/Storages/PartitionCommands.h @@ -33,6 +33,7 @@ struct PartitionCommand UNFREEZE_ALL_PARTITIONS, UNFREEZE_PARTITION, REPLACE_PARTITION, + EXPORT_PART, }; Type type = UNKNOWN; diff --git a/src/Storages/PartitionStrategy.cpp b/src/Storages/PartitionStrategy.cpp new file mode 100644 index 000000000000..d3bcb6c0991f --- /dev/null +++ b/src/Storages/PartitionStrategy.cpp @@ -0,0 +1,261 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +extern const int BAD_ARGUMENTS; +} + +namespace +{ + HiveStylePartitionStrategy::PartitionExpressionActionsAndColumnName buildExpressionHive( + ASTPtr partition_by, + const NamesAndTypesList & partition_columns, + const Block & sample_block, + ContextPtr context) + { + HiveStylePartitionStrategy::PartitionExpressionActionsAndColumnName actions_with_column_name; + ASTs concat_args; + + if (const auto * tuple_function = partition_by->as(); + tuple_function && tuple_function->name == "tuple") + { + chassert(tuple_function->arguments->children.size() == partition_columns.size()); + + std::size_t index = 0; + + for (const auto & partition_column : partition_columns) + { + const auto & child = tuple_function->arguments->children[index++]; + + concat_args.push_back(std::make_shared(partition_column.name + "=")); + + concat_args.push_back(makeASTFunction("toString", child)); + + concat_args.push_back(std::make_shared("/")); + } + } + else + { + chassert(partition_columns.size() == 1); + + ASTs to_string_args = {1, partition_by}; + concat_args.push_back(std::make_shared(partition_columns.front().name + "=")); + concat_args.push_back(makeASTFunction("toString", std::move(to_string_args))); + concat_args.push_back(std::make_shared("/")); + } + + ASTPtr hive_expr = makeASTFunction("concat", std::move(concat_args)); + auto hive_syntax_result = TreeRewriter(context).analyze(hive_expr, sample_block.getNamesAndTypesList()); + actions_with_column_name.actions = ExpressionAnalyzer(hive_expr, hive_syntax_result, context).getActions(false); + actions_with_column_name.column_name = hive_expr->getColumnName(); + + return actions_with_column_name; + } + + Block buildBlockWithoutPartitionColumns( + const Block & sample_block, + const std::unordered_set & partition_expression_required_columns_set) + { + Block result; + for (size_t i = 0; i < sample_block.columns(); i++) + { + if (!partition_expression_required_columns_set.contains(sample_block.getByPosition(i).name)) + { + result.insert(sample_block.getByPosition(i)); + } + } + + return result; + } +} + +std::unordered_map PartitionStrategy::partition_strategy_to_wildcard_acceptance = +{ + {"wildcard", true}, + {"hive", false} +}; + +PartitionStrategy::PartitionStrategy(ASTPtr partition_by_, const Block & sample_block_, ContextPtr context_) +: partition_by(partition_by_), sample_block(sample_block_), context(context_) +{ + auto key_description = KeyDescription::getKeyFromAST(partition_by, ColumnsDescription::fromNamesAndTypes(sample_block.getNamesAndTypes()), context); + partition_columns = key_description.sample_block.getNamesAndTypesList(); +} + +const NamesAndTypesList & PartitionStrategy::getPartitionColumns() const +{ + return partition_columns; +} + +struct HivePartitionStrategyFactory +{ + static std::shared_ptr get( + ASTPtr partition_by, + const Block & sample_block, + ContextPtr context, + bool globbed_path, + bool partition_columns_in_data_file) + { + if (!partition_by) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition strategy hive can not be used without a PARTITION BY expression"); + } + + if (globbed_path) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition strategy {} can not be used with a globbed path", "hive"); + } + + return std::make_shared( + partition_by, + sample_block, + context, + partition_columns_in_data_file); + } +}; + +struct WildcardPartitionStrategyFactory +{ + static std::shared_ptr get( + ASTPtr partition_by, + const Block & sample_block, + ContextPtr context, + bool partition_columns_in_data_file) + { + if (!partition_by) + { + return nullptr; + } + + if (!partition_columns_in_data_file) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition strategy {} can not be used with partition_columns_in_data_file=0", "wildcard"); + } + + // in theory, we should not accept wildcard partition strategy without a wildcard in the path + // but it has been made that way by default, it just won't include the partition id in the filepath + + return std::make_shared(partition_by, sample_block, context); + } +}; + +std::shared_ptr PartitionStrategyFactory::get(ASTPtr partition_by, + const Block & sample_block, + ContextPtr context, + bool globbed_path, + const std::string & partition_strategy, + bool partition_columns_in_data_file) +{ + if (partition_strategy == "hive") + { + return HivePartitionStrategyFactory::get( + partition_by, + sample_block, + context, + globbed_path, + partition_columns_in_data_file); + } + + if (partition_strategy == "wildcard") + { + return WildcardPartitionStrategyFactory::get(partition_by, sample_block, context, partition_columns_in_data_file); + } + + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Unknown partitioning style '{}'", + partition_strategy); +} + +std::shared_ptr PartitionStrategyFactory::get(ASTPtr partition_by, + const NamesAndTypesList & partition_columns, + ContextPtr context, + bool globbed_path, + const std::string & partition_strategy, + bool partition_columns_in_data_file) +{ + Block block; + for (const auto & partition_column : partition_columns) + { + block.insert({partition_column.type, partition_column.name}); + } + + return get(partition_by, block, context, globbed_path, partition_strategy, partition_columns_in_data_file); +} + +StringifiedPartitionStrategy::StringifiedPartitionStrategy(ASTPtr partition_by_, const Block & sample_block_, ContextPtr context_) + : PartitionStrategy(partition_by_, sample_block_, context_) +{ + ASTs arguments(1, partition_by); + ASTPtr partition_by_string = makeASTFunction("toString", std::move(arguments)); + auto syntax_result = TreeRewriter(context).analyze(partition_by_string, sample_block.getNamesAndTypesList()); + actions_with_column_name.actions = ExpressionAnalyzer(partition_by_string, syntax_result, context).getActions(false); + actions_with_column_name.column_name = partition_by_string->getColumnName(); +} + +HiveStylePartitionStrategy::HiveStylePartitionStrategy( + ASTPtr partition_by_, + const Block & sample_block_, + ContextPtr context_, + bool partition_columns_in_data_file_) + : PartitionStrategy(partition_by_, sample_block_, context_), + partition_columns_in_data_file(partition_columns_in_data_file_) +{ + for (const auto & partition_column : partition_columns) + { + partition_columns_name_set.insert(partition_column.name); + } + actions_with_column_name = buildExpressionHive(partition_by, partition_columns, sample_block, context); + block_without_partition_columns = buildBlockWithoutPartitionColumns(sample_block, partition_columns_name_set); +} + +Chunk HiveStylePartitionStrategy::getFormatChunk(const Chunk & chunk) +{ + Chunk result; + + if (partition_columns_in_data_file) + { + for (const auto & column : chunk.getColumns()) + { + result.addColumn(column); + } + + return result; + } + + chassert(chunk.getColumns().size() == sample_block.columns()); + + for (size_t i = 0; i < sample_block.columns(); i++) + { + if (!partition_columns_name_set.contains(sample_block.getByPosition(i).name)) + { + result.addColumn(chunk.getColumns()[i]); + } + } + + return result; +} + +Block HiveStylePartitionStrategy::getFormatHeader() +{ + if (partition_columns_in_data_file) + { + return sample_block; + } + + return block_without_partition_columns; +} + +} diff --git a/src/Storages/PartitionStrategy.h b/src/Storages/PartitionStrategy.h new file mode 100644 index 000000000000..5b6ad219c128 --- /dev/null +++ b/src/Storages/PartitionStrategy.h @@ -0,0 +1,95 @@ +#pragma once + +#include +#include + +#include + +namespace DB +{ + +struct PartitionStrategy +{ + struct PartitionExpressionActionsAndColumnName + { + ExpressionActionsPtr actions; + std::string column_name; + }; + + static std::unordered_map partition_strategy_to_wildcard_acceptance; + + PartitionStrategy(ASTPtr partition_by_, const Block & sample_block_, ContextPtr context_); + + virtual ~PartitionStrategy() = default; + + ColumnPtr computePartitionKey(const Chunk & chunk) + { + Block block_with_partition_by_expr = sample_block.cloneWithoutColumns(); + block_with_partition_by_expr.setColumns(chunk.getColumns()); + + return computePartitionKey(block_with_partition_by_expr); + } + + ColumnPtr computePartitionKey(Block & block) + { + actions_with_column_name.actions->execute(block); + + return block.getByName(actions_with_column_name.column_name).column; + } + + virtual Chunk getFormatChunk(const Chunk & chunk) { return chunk.clone(); } + + virtual Block getFormatHeader() { return sample_block; } + + const NamesAndTypesList & getPartitionColumns() const; + +protected: + ASTPtr partition_by; + Block sample_block; + ContextPtr context; + NamesAndTypesList partition_columns; + PartitionExpressionActionsAndColumnName actions_with_column_name; +}; + +struct PartitionStrategyFactory +{ + static std::shared_ptr get( + ASTPtr partition_by, + const Block & sample_block, + ContextPtr context, + bool globbed_path, + const std::string & partition_strategy, + bool partition_columns_in_data_file); + + static std::shared_ptr get( + ASTPtr partition_by, + const NamesAndTypesList & partition_columns, + ContextPtr context, + bool globbed_path, + const std::string & partition_strategy, + bool partition_columns_in_data_file); +}; + +struct StringifiedPartitionStrategy : PartitionStrategy +{ + StringifiedPartitionStrategy(ASTPtr partition_by_, const Block & sample_block_, ContextPtr context_); +}; + +struct HiveStylePartitionStrategy : PartitionStrategy +{ + HiveStylePartitionStrategy( + ASTPtr partition_by_, + const Block & sample_block_, + ContextPtr context_, + bool partition_columns_in_data_file_); + + Chunk getFormatChunk(const Chunk & chunk) override; + Block getFormatHeader() override; + +private: + bool partition_columns_in_data_file; + std::unordered_set partition_columns_name_set; + Block block_without_partition_columns; +}; + +} diff --git a/src/Storages/PartitionedSink.cpp b/src/Storages/PartitionedSink.cpp index ec43e4b4ca1f..940f11333a58 100644 --- a/src/Storages/PartitionedSink.cpp +++ b/src/Storages/PartitionedSink.cpp @@ -6,10 +6,10 @@ #include #include - #include - #include +#include +#include "boost/iostreams/concepts.hpp" namespace DB @@ -22,12 +22,14 @@ namespace ErrorCodes PartitionedSink::PartitionedSink( std::shared_ptr partition_strategy_, + std::shared_ptr sink_creator_, ContextPtr context_, - const Block & sample_block_) - : SinkToStorage(sample_block_) + const Block & source_header_) + : SinkToStorage(source_header_) , partition_strategy(partition_strategy_) + , sink_creator(sink_creator_) , context(context_) - , sample_block(sample_block_) + , source_header(source_header_) { } @@ -37,19 +39,49 @@ SinkPtr PartitionedSink::getSinkForPartitionKey(StringRef partition_key) auto it = partition_id_to_sink.find(partition_key); if (it == partition_id_to_sink.end()) { - auto sink = createSinkForPartition(partition_key.toString()); + auto sink = sink_creator->createSinkForPartition(partition_key.toString()); std::tie(it, std::ignore) = partition_id_to_sink.emplace(partition_key, sink); } return it->second; } +PartitionedSink::ChunkSplitStatistics PartitionedSink::getPartitioningStats() const +{ + return partitioning_stats; +} + +void PartitionedSink::consumeAssumeSamePartition(Chunk & source_chunk) +{ + if (!sink_to_storage) + { + const ColumnPtr partition_by_result_column = partition_strategy->computePartitionKey(source_chunk); + auto partition_key = partition_by_result_column->getDataAt(0); + sink_to_storage = sink_creator->createSinkForPartition(partition_key.toString()); + } + + auto format_chunk = partition_strategy->getFormatChunk(source_chunk); + sink_to_storage->consume(format_chunk); +} + void PartitionedSink::consume(Chunk & source_chunk) { + if (assume_same_partition) + { + consumeAssumeSamePartition(source_chunk); + return; + } + + auto start_calc = std::chrono::system_clock::now(); const ColumnPtr partition_by_result_column = partition_strategy->computePartitionKey(source_chunk); + auto end_calc = std::chrono::system_clock::now(); + + partitioning_stats.time_spent_on_partition_calculation += +std::chrono::duration_cast(end_calc - start_calc).count(); /// Not all columns are serialized using the format writer (e.g, hive partitioning stores partition columns in the file path) - const auto columns_to_consume = partition_strategy->getFormatChunkColumns(source_chunk); + auto format_chunk = partition_strategy->getFormatChunk(source_chunk); + const auto columns_to_consume = format_chunk.getColumns(); if (columns_to_consume.empty()) { @@ -58,6 +90,8 @@ void PartitionedSink::consume(Chunk & source_chunk) "Consider setting `partition_columns_in_data_file=1`"); } + auto start_split = std::chrono::system_clock::now(); + size_t chunk_rows = source_chunk.getNumRows(); chunk_row_index_to_partition_index.resize(chunk_rows); @@ -98,6 +132,10 @@ void PartitionedSink::consume(Chunk & source_chunk) } } + auto end_split = std::chrono::system_clock::now(); + + partitioning_stats.time_spent_on_chunk_split += std::chrono::duration_cast(end_split - start_split).count(); + for (const auto & [partition_key, partition_index] : partition_id_to_chunk_index) { auto sink = getSinkForPartitionKey(partition_key); @@ -107,6 +145,11 @@ void PartitionedSink::consume(Chunk & source_chunk) void PartitionedSink::onException(std::exception_ptr exception) { + if (assume_same_partition && sink_to_storage) + { + sink_to_storage->onException(exception); + return; + } for (auto & [_, sink] : partition_id_to_sink) { sink->onException(exception); @@ -115,6 +158,11 @@ void PartitionedSink::onException(std::exception_ptr exception) void PartitionedSink::onFinish() { + if (assume_same_partition && sink_to_storage) + { + sink_to_storage->onFinish(); + return; + } for (auto & [_, sink] : partition_id_to_sink) { sink->onFinish(); @@ -146,8 +194,16 @@ String PartitionedSink::replaceWildcards(const String & haystack, const String & PartitionedSink::~PartitionedSink() { if (isCancelled()) - for (auto & item : partition_id_to_sink) + { + if (assume_same_partition && sink_to_storage) + { + sink_to_storage->cancel(); + return; + } + for (auto &item: partition_id_to_sink) item.second->cancel(); + } + } } diff --git a/src/Storages/PartitionedSink.h b/src/Storages/PartitionedSink.h index 481230792db0..231276c9bfa1 100644 --- a/src/Storages/PartitionedSink.h +++ b/src/Storages/PartitionedSink.h @@ -17,12 +17,26 @@ namespace DB class PartitionedSink : public SinkToStorage { public: + struct SinkCreator + { + virtual ~SinkCreator() = default; + virtual SinkPtr createSinkForPartition(const String & partition_id) = 0; + }; + static constexpr auto PARTITION_ID_WILDCARD = "{_partition_id}"; PartitionedSink( std::shared_ptr partition_strategy_, + std::shared_ptr sink_creator_, ContextPtr context_, - const Block & sample_block_); + const Block & source_header_ + ); + + struct ChunkSplitStatistics + { + uint64_t time_spent_on_partition_calculation = 0; + uint64_t time_spent_on_chunk_split = 0; + }; ~PartitionedSink() override; @@ -34,18 +48,21 @@ class PartitionedSink : public SinkToStorage void onFinish() override; - virtual SinkPtr createSinkForPartition(const String & partition_id) = 0; + void assumeSamePartition() { + assume_same_partition = true; + } static void validatePartitionKey(const String & str, bool allow_slash); static String replaceWildcards(const String & haystack, const String & partition_id); -protected: - std::shared_ptr partition_strategy; + ChunkSplitStatistics getPartitioningStats() const; private: + std::shared_ptr partition_strategy; + std::shared_ptr sink_creator; ContextPtr context; - Block sample_block; + Block source_header; absl::flat_hash_map partition_id_to_sink; HashMapWithSavedHash partition_id_to_chunk_index; @@ -53,6 +70,11 @@ class PartitionedSink : public SinkToStorage Arena partition_keys_arena; SinkPtr getSinkForPartitionKey(StringRef partition_key); + ChunkSplitStatistics partitioning_stats; + bool assume_same_partition = false; + std::shared_ptr sink_to_storage; + + void consumeAssumeSamePartition(Chunk & chunk); }; } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 5a3637ec33c2..a26ee505cce4 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -74,6 +74,7 @@ #include #include +#include namespace ProfileEvents { @@ -1942,7 +1943,7 @@ class StorageFileSink final : public SinkToStorage, WithContext std::unique_lock lock; }; -class PartitionedStorageFileSink : public PartitionedSink +class PartitionedStorageFileSink : public PartitionedSink::SinkCreator { public: PartitionedStorageFileSink( @@ -1957,7 +1958,7 @@ class PartitionedStorageFileSink : public PartitionedSink const String format_name_, ContextPtr context_, int flags_) - : PartitionedSink(partition_strategy_, context_, metadata_snapshot_->getSampleBlock()) + : partition_strategy(partition_strategy_) , path(path_) , metadata_snapshot(metadata_snapshot_) , table_name_for_log(table_name_for_log_) @@ -1973,11 +1974,13 @@ class PartitionedStorageFileSink : public PartitionedSink SinkPtr createSinkForPartition(const String & partition_id) override { - std::string filepath = partition_strategy->getPathForWrite(path, partition_id); + const auto file_path_generator = std::make_shared(path); + std::string filepath = file_path_generator->getWritingPath(partition_id); fs::create_directories(fs::path(filepath).parent_path()); - validatePartitionKey(filepath, true); + PartitionedSink::validatePartitionKey(filepath, true); + checkCreationIsAllowed(context, context->getUserFilesPath(), filepath, /*can_be_directory=*/ true); return std::make_shared( metadata_snapshot, @@ -1994,6 +1997,7 @@ class PartitionedStorageFileSink : public PartitionedSink } private: + std::shared_ptr partition_strategy; const String path; StorageMetadataPtr metadata_snapshot; String table_name_for_log; @@ -2045,7 +2049,7 @@ SinkToStoragePtr StorageFile::write( has_wildcards, /* partition_columns_in_data_file */true); - return std::make_shared( + auto sink_creator = std::make_shared( partition_strategy, metadata_snapshot, getStorageID().getNameForLogs(), @@ -2057,6 +2061,13 @@ SinkToStoragePtr StorageFile::write( format_name, context, flags); + + return std::make_shared( + partition_strategy, + sink_creator, + context, + metadata_snapshot->getSampleBlock() + ); } String path; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 716733224dbf..c92f9a31a4cd 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -47,6 +48,7 @@ #include #include "Core/BackgroundSchedulePool.h" #include "Core/Names.h" +#include "ObjectStorage/MergeTree/StorageObjectStorageSinkMTPartImportDecorator.h" namespace DB { @@ -484,6 +486,82 @@ void StorageMergeTree::alter( } } +/* + * For now, this function is meant to be used when exporting to different formats (i.e, the case where data needs to be re-encoded / serialized) + * For the cases where this is not necessary, there are way more optimal ways of doing that, such as hard links implemented by `movePartitionToTable` + * */ +void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, ContextPtr query_context) +{ + String dest_database = query_context->resolveDatabase(command.to_database); + auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, command.to_table}, query_context); + + /// The target table and the source table are the same. + if (dest_storage->getStorageID() == this->getStorageID()) + return; + + auto query = std::make_shared(); + + String partition_id = getPartitionIDFromQuery(command.partition, getContext()); + + background_moves_assignee.scheduleMoveTask(std::make_shared( + [this, query_context, partition_id, dest_storage] () mutable + { + const auto src_parts = getVisibleDataPartsVectorInPartition(getContext(), partition_id); + + if (src_parts.empty()) + { + return true; + } + + auto lock1 = lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef()[Setting::lock_acquire_timeout]); + auto lock2 = dest_storage->lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef()[Setting::lock_acquire_timeout]); + auto merges_blocker = stopMergesAndWait(); + + /// todo should stopMergesAndWait be accounted as well? + const auto start_time = std::chrono::system_clock::now(); + + dest_storage->importMergeTreePartition(*this, src_parts, getContext(), [&](MergeTreePartImportStats stats) + { + auto table_id = getStorageID(); + auto part_log = getContext()->getPartLog(table_id.database_name); + if (!part_log) + return; + + PartLogElement part_log_elem; + part_log_elem.event_type = PartLogElement::Type::EXPORT_PART; + part_log_elem.merge_algorithm = PartLogElement::PartMergeAlgorithm::UNDECIDED; + part_log_elem.merge_reason = PartLogElement::MergeReasonType::NOT_A_MERGE; + + part_log_elem.database_name = table_id.database_name; + part_log_elem.table_name = table_id.table_name; + part_log_elem.table_uuid = table_id.uuid; + part_log_elem.partition_id = MergeTreePartInfo::fromPartName(stats.part->name, format_version).getPartitionId(); + // construct event_time and event_time_microseconds using the same time point + // so that the two times will always be equal up to a precision of a second. + const auto time_now = std::chrono::system_clock::now(); + part_log_elem.event_time = timeInSeconds(time_now); + part_log_elem.event_time_microseconds = timeInMicroseconds(time_now); + + part_log_elem.duration_ms = std::chrono::duration_cast(start_time - time_now).count() / 1000000; + part_log_elem.error = static_cast(stats.status.code); + part_log_elem.exception = stats.status.message; + part_log_elem.path_on_disk = stats.file_path; + part_log_elem.part_name = stats.part->name; + part_log_elem.bytes_compressed_on_disk = stats.bytes_on_disk; + part_log_elem.rows = stats.part->rows_count; + part_log_elem.disk_name = dest_storage->getName(); + part_log_elem.part_type = stats.part->getType(); + part_log_elem.source_part_names = {stats.part->name}; + part_log_elem.rows_read = stats.read_rows; + part_log_elem.bytes_read_uncompressed = stats.read_bytes; + + part_log->add(std::move(part_log_elem)); + }); + + /// Perhaps this is a good way to trigger re-tries? + return true; + }, moves_assignee_trigger, getStorageID())); +} /// While exists, marks parts as 'currently_merging_mutating_parts' and reserves free space on filesystem. CurrentlyMergingPartsTagger::CurrentlyMergingPartsTagger( diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 36ac29f2918b..c28f965f88c7 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -250,6 +250,7 @@ class StorageMergeTree final : public MergeTreeData void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr context) override; void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr context) override; + void exportPartitionToTable(const PartitionCommand & command, ContextPtr query_context) override; bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const override; /// Update mutation entries after part mutation execution. May reset old /// errors if mutation was successful. Otherwise update last_failed* fields diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 354fd8c3b2b5..d82e410d29db 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -56,6 +56,11 @@ #include #include +#include "PartitionedSink.h" +#include "Formats/EscapingRuleUtils.h" +#include "Interpreters/convertFieldToType.h" +#include "ObjectStorage/FilePathGenerator.h" + namespace ProfileEvents { extern const Event EngineFileLikeReadFiles; @@ -729,7 +734,7 @@ void StorageURLSink::cancelBuffers() write_buf->cancel(); } -class PartitionedStorageURLSink : public PartitionedSink +class PartitionedStorageURLSink : public PartitionedSink::SinkCreator { public: PartitionedStorageURLSink( @@ -743,7 +748,7 @@ class PartitionedStorageURLSink : public PartitionedSink const CompressionMethod compression_method_, const HTTPHeaderEntries & headers_, const String & http_method_) - : PartitionedSink(partition_strategy_, context_, sample_block_) + : partition_strategy(partition_strategy_) , uri(uri_) , format(format_) , format_settings(format_settings_) @@ -758,7 +763,8 @@ class PartitionedStorageURLSink : public PartitionedSink SinkPtr createSinkForPartition(const String & partition_id) override { - std::string partition_path = partition_strategy->getPathForWrite(uri, partition_id); + const auto file_path_generator = std::make_shared(uri); + std::string partition_path = file_path_generator->getWritingPath(partition_id); context->getRemoteHostFilter().checkURL(Poco::URI(partition_path)); return std::make_shared( @@ -766,6 +772,7 @@ class PartitionedStorageURLSink : public PartitionedSink } private: + std::shared_ptr partition_strategy; const String uri; const String format; const std::optional format_settings; @@ -1408,7 +1415,7 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad has_wildcards, /* partition_columns_in_data_file */true); - return std::make_shared( + auto sink_creator = std::make_shared( partition_strategy, uri, format_name, @@ -1419,6 +1426,8 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad compression_method, headers, http_method); + + return std::make_shared(partition_strategy, sink_creator, context, metadata_snapshot->getSampleBlock()); } return std::make_shared( diff --git a/src/Storages/System/StorageSystemExports.cpp b/src/Storages/System/StorageSystemExports.cpp new file mode 100644 index 000000000000..3b4c938ac115 --- /dev/null +++ b/src/Storages/System/StorageSystemExports.cpp @@ -0,0 +1,50 @@ +#include +#include +#include +#include +#include "DataTypes/DataTypeString.h" +#include "DataTypes/DataTypesNumber.h" + + +namespace DB +{ + +ColumnsDescription StorageSystemExports::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."}, + {"elapsed", std::make_shared(), "Time elapsed (in seconds) since data part movement started."}, + {"destination_path", std::make_shared(), "Path to the destination file in the destination storage."}, + {"part_name", std::make_shared(), "Name of the data part being moved."}, + {"thread_id", std::make_shared(), "Identifier of a thread performing the movement."}, + }; +} + +void StorageSystemExports::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const +{ + const auto access = context->getAccess(); + const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_TABLES); + const auto & move_list = context->getExportsList(); + + for (const auto & move : move_list.get()) + { + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, move.source_database, move.source_table)) + continue; + + size_t i = 0; + res_columns[i++]->insert(move.source_database); + res_columns[i++]->insert(move.source_table); + res_columns[i++]->insert(move.destination_database); + res_columns[i++]->insert(move.destination_table); + res_columns[i++]->insert(move.elapsed); + res_columns[i++]->insert(move.destination_path); + res_columns[i++]->insert(move.part_name); + res_columns[i++]->insert(move.thread_id); + } +} + +} diff --git a/src/Storages/System/StorageSystemExports.h b/src/Storages/System/StorageSystemExports.h new file mode 100644 index 000000000000..e13fbfa26aaa --- /dev/null +++ b/src/Storages/System/StorageSystemExports.h @@ -0,0 +1,25 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + + +class StorageSystemExports final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemExports"; } + + 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/StorageSystemMoves.cpp b/src/Storages/System/StorageSystemMoves.cpp index 2c43043d8207..7faa6d58f0e4 100644 --- a/src/Storages/System/StorageSystemMoves.cpp +++ b/src/Storages/System/StorageSystemMoves.cpp @@ -27,8 +27,9 @@ void StorageSystemMoves::fillData(MutableColumns & res_columns, ContextPtr conte { const auto access = context->getAccess(); const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_TABLES); + const auto & move_list = context->getMovesList(); - for (const auto & move : context->getMovesList().get()) + for (const auto & move : move_list.get()) { if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, move.database, move.table)) continue; diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 9249fb3530b6..4f2977d1d07b 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -106,6 +106,8 @@ #include +#include "StorageSystemExports.h" + #if defined(__ELF__) && !defined(OS_FREEBSD) #include #endif @@ -206,6 +208,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "histogram_metrics", "Contains histogram metrics which can be calculated instantly and exported in the Prometheus format. For example, the keeper response time. This table is always up to date."); 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 information about in-progress data part exports of MergeTree tables. Each data part movement 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/src/TableFunctions/ITableFunction.h b/src/TableFunctions/ITableFunction.h index fe971bdbed7c..44039b09ea8f 100644 --- a/src/TableFunctions/ITableFunction.h +++ b/src/TableFunctions/ITableFunction.h @@ -15,6 +15,7 @@ namespace DB { class Context; +class ASTInsertQuery; /** Interface for table functions. * From 65397b8423c5f124d0afee3c3e9e90c779941025 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 28 Jul 2025 17:46:54 -0300 Subject: [PATCH 02/48] fix build1 --- src/Storages/ObjectStorage/StorageObjectStorage.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index ff7aa493e742..195a3bd8c194 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -317,8 +317,6 @@ class StorageObjectStorage::Configuration bool if_not_updated_before, bool check_consistent_with_previous_metadata); - void initPartitionStrategy(ASTPtr partition_by, const ColumnsDescription & columns, ContextPtr context); - const StorageObjectStorageSettings & getSettingsRef() const; virtual const DataLakeStorageSettings & getDataLakeSettings() const { From 55a7ac12a2c788363b99121bfd094228e4e99fdc Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 29 Jul 2025 08:29:01 -0300 Subject: [PATCH 03/48] fix build for sure --- src/Storages/IPartitionStrategy.h | 12 +- .../ObjectStorage/S3/Configuration.cpp | 16 +- .../ObjectStorage/StorageObjectStorage.h | 1 + src/Storages/PartitionStrategy.cpp | 261 ------------------ src/Storages/PartitionStrategy.h | 95 ------- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageURL.cpp | 2 +- 7 files changed, 25 insertions(+), 364 deletions(-) delete mode 100644 src/Storages/PartitionStrategy.cpp delete mode 100644 src/Storages/PartitionStrategy.h diff --git a/src/Storages/IPartitionStrategy.h b/src/Storages/IPartitionStrategy.h index 2f053013a516..cfd6294309a0 100644 --- a/src/Storages/IPartitionStrategy.h +++ b/src/Storages/IPartitionStrategy.h @@ -29,6 +29,13 @@ struct IPartitionStrategy virtual ColumnPtr computePartitionKey(const Chunk & chunk) = 0; + ColumnPtr computePartitionKey(Block & block) + { + actions_with_column_name.actions->execute(block); + + return block.getByName(actions_with_column_name.column_name).column; + } + virtual std::string getPathForRead(const std::string & prefix) = 0; virtual std::string getPathForWrite(const std::string & prefix, const std::string & partition_key) = 0; @@ -43,6 +50,7 @@ struct IPartitionStrategy const KeyDescription partition_key_description; const Block sample_block; ContextPtr context; + PartitionExpressionActionsAndColumnName actions_with_column_name; }; /* @@ -81,9 +89,6 @@ struct WildcardPartitionStrategy : IPartitionStrategy ColumnPtr computePartitionKey(const Chunk & chunk) override; std::string getPathForRead(const std::string & prefix) override; std::string getPathForWrite(const std::string & prefix, const std::string & partition_key) override; - -private: - PartitionExpressionActionsAndColumnName actions_with_column_name; }; /* @@ -111,7 +116,6 @@ struct HiveStylePartitionStrategy : IPartitionStrategy const std::string file_format; const bool partition_columns_in_data_file; std::unordered_set partition_columns_name_set; - PartitionExpressionActionsAndColumnName actions_with_column_name; Block block_without_partition_columns; }; diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index fb7d62d054f8..790861fee194 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -168,8 +168,20 @@ void StorageS3Configuration::fromNamedCollection(const NamedCollection & collect else url = S3::URI(collection.get("url"), settings[Setting::allow_archive_path_syntax]); - partition_strategy_name = collection.getOrDefault("partition_strategy", "wildcard"); - partition_columns_in_data_file = collection.getOrDefault("partition_columns_in_data_file", partition_strategy_name != "hive"); + if (collection.has("partition_strategy")) + { + const auto partition_strategy_name = collection.get("partition_strategy"); + const auto partition_strategy_type_opt = magic_enum::enum_cast(partition_strategy_name, magic_enum::case_insensitive); + + if (!partition_strategy_type_opt) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition strategy {} is not supported", partition_strategy_name); + } + + partition_strategy_type = partition_strategy_type_opt.value(); + } + + partition_columns_in_data_file = collection.getOrDefault("partition_columns_in_data_file", partition_strategy_type != PartitionStrategyFactory::StrategyType::HIVE); auth_settings[S3AuthSetting::access_key_id] = collection.getOrDefault("access_key_id", ""); auth_settings[S3AuthSetting::secret_access_key] = collection.getOrDefault("secret_access_key", ""); diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 195a3bd8c194..acf11cb6dbcd 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -331,6 +331,7 @@ class StorageObjectStorage::Configuration /// And alternative is with hive partitioning, when they are contained in file path. bool partition_columns_in_data_file = true; std::shared_ptr partition_strategy; + std::shared_ptr file_path_generator; protected: virtual void fromNamedCollection(const NamedCollection & collection, ContextPtr context) = 0; diff --git a/src/Storages/PartitionStrategy.cpp b/src/Storages/PartitionStrategy.cpp deleted file mode 100644 index d3bcb6c0991f..000000000000 --- a/src/Storages/PartitionStrategy.cpp +++ /dev/null @@ -1,261 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ -extern const int LOGICAL_ERROR; -extern const int BAD_ARGUMENTS; -} - -namespace -{ - HiveStylePartitionStrategy::PartitionExpressionActionsAndColumnName buildExpressionHive( - ASTPtr partition_by, - const NamesAndTypesList & partition_columns, - const Block & sample_block, - ContextPtr context) - { - HiveStylePartitionStrategy::PartitionExpressionActionsAndColumnName actions_with_column_name; - ASTs concat_args; - - if (const auto * tuple_function = partition_by->as(); - tuple_function && tuple_function->name == "tuple") - { - chassert(tuple_function->arguments->children.size() == partition_columns.size()); - - std::size_t index = 0; - - for (const auto & partition_column : partition_columns) - { - const auto & child = tuple_function->arguments->children[index++]; - - concat_args.push_back(std::make_shared(partition_column.name + "=")); - - concat_args.push_back(makeASTFunction("toString", child)); - - concat_args.push_back(std::make_shared("/")); - } - } - else - { - chassert(partition_columns.size() == 1); - - ASTs to_string_args = {1, partition_by}; - concat_args.push_back(std::make_shared(partition_columns.front().name + "=")); - concat_args.push_back(makeASTFunction("toString", std::move(to_string_args))); - concat_args.push_back(std::make_shared("/")); - } - - ASTPtr hive_expr = makeASTFunction("concat", std::move(concat_args)); - auto hive_syntax_result = TreeRewriter(context).analyze(hive_expr, sample_block.getNamesAndTypesList()); - actions_with_column_name.actions = ExpressionAnalyzer(hive_expr, hive_syntax_result, context).getActions(false); - actions_with_column_name.column_name = hive_expr->getColumnName(); - - return actions_with_column_name; - } - - Block buildBlockWithoutPartitionColumns( - const Block & sample_block, - const std::unordered_set & partition_expression_required_columns_set) - { - Block result; - for (size_t i = 0; i < sample_block.columns(); i++) - { - if (!partition_expression_required_columns_set.contains(sample_block.getByPosition(i).name)) - { - result.insert(sample_block.getByPosition(i)); - } - } - - return result; - } -} - -std::unordered_map PartitionStrategy::partition_strategy_to_wildcard_acceptance = -{ - {"wildcard", true}, - {"hive", false} -}; - -PartitionStrategy::PartitionStrategy(ASTPtr partition_by_, const Block & sample_block_, ContextPtr context_) -: partition_by(partition_by_), sample_block(sample_block_), context(context_) -{ - auto key_description = KeyDescription::getKeyFromAST(partition_by, ColumnsDescription::fromNamesAndTypes(sample_block.getNamesAndTypes()), context); - partition_columns = key_description.sample_block.getNamesAndTypesList(); -} - -const NamesAndTypesList & PartitionStrategy::getPartitionColumns() const -{ - return partition_columns; -} - -struct HivePartitionStrategyFactory -{ - static std::shared_ptr get( - ASTPtr partition_by, - const Block & sample_block, - ContextPtr context, - bool globbed_path, - bool partition_columns_in_data_file) - { - if (!partition_by) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition strategy hive can not be used without a PARTITION BY expression"); - } - - if (globbed_path) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition strategy {} can not be used with a globbed path", "hive"); - } - - return std::make_shared( - partition_by, - sample_block, - context, - partition_columns_in_data_file); - } -}; - -struct WildcardPartitionStrategyFactory -{ - static std::shared_ptr get( - ASTPtr partition_by, - const Block & sample_block, - ContextPtr context, - bool partition_columns_in_data_file) - { - if (!partition_by) - { - return nullptr; - } - - if (!partition_columns_in_data_file) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition strategy {} can not be used with partition_columns_in_data_file=0", "wildcard"); - } - - // in theory, we should not accept wildcard partition strategy without a wildcard in the path - // but it has been made that way by default, it just won't include the partition id in the filepath - - return std::make_shared(partition_by, sample_block, context); - } -}; - -std::shared_ptr PartitionStrategyFactory::get(ASTPtr partition_by, - const Block & sample_block, - ContextPtr context, - bool globbed_path, - const std::string & partition_strategy, - bool partition_columns_in_data_file) -{ - if (partition_strategy == "hive") - { - return HivePartitionStrategyFactory::get( - partition_by, - sample_block, - context, - globbed_path, - partition_columns_in_data_file); - } - - if (partition_strategy == "wildcard") - { - return WildcardPartitionStrategyFactory::get(partition_by, sample_block, context, partition_columns_in_data_file); - } - - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unknown partitioning style '{}'", - partition_strategy); -} - -std::shared_ptr PartitionStrategyFactory::get(ASTPtr partition_by, - const NamesAndTypesList & partition_columns, - ContextPtr context, - bool globbed_path, - const std::string & partition_strategy, - bool partition_columns_in_data_file) -{ - Block block; - for (const auto & partition_column : partition_columns) - { - block.insert({partition_column.type, partition_column.name}); - } - - return get(partition_by, block, context, globbed_path, partition_strategy, partition_columns_in_data_file); -} - -StringifiedPartitionStrategy::StringifiedPartitionStrategy(ASTPtr partition_by_, const Block & sample_block_, ContextPtr context_) - : PartitionStrategy(partition_by_, sample_block_, context_) -{ - ASTs arguments(1, partition_by); - ASTPtr partition_by_string = makeASTFunction("toString", std::move(arguments)); - auto syntax_result = TreeRewriter(context).analyze(partition_by_string, sample_block.getNamesAndTypesList()); - actions_with_column_name.actions = ExpressionAnalyzer(partition_by_string, syntax_result, context).getActions(false); - actions_with_column_name.column_name = partition_by_string->getColumnName(); -} - -HiveStylePartitionStrategy::HiveStylePartitionStrategy( - ASTPtr partition_by_, - const Block & sample_block_, - ContextPtr context_, - bool partition_columns_in_data_file_) - : PartitionStrategy(partition_by_, sample_block_, context_), - partition_columns_in_data_file(partition_columns_in_data_file_) -{ - for (const auto & partition_column : partition_columns) - { - partition_columns_name_set.insert(partition_column.name); - } - actions_with_column_name = buildExpressionHive(partition_by, partition_columns, sample_block, context); - block_without_partition_columns = buildBlockWithoutPartitionColumns(sample_block, partition_columns_name_set); -} - -Chunk HiveStylePartitionStrategy::getFormatChunk(const Chunk & chunk) -{ - Chunk result; - - if (partition_columns_in_data_file) - { - for (const auto & column : chunk.getColumns()) - { - result.addColumn(column); - } - - return result; - } - - chassert(chunk.getColumns().size() == sample_block.columns()); - - for (size_t i = 0; i < sample_block.columns(); i++) - { - if (!partition_columns_name_set.contains(sample_block.getByPosition(i).name)) - { - result.addColumn(chunk.getColumns()[i]); - } - } - - return result; -} - -Block HiveStylePartitionStrategy::getFormatHeader() -{ - if (partition_columns_in_data_file) - { - return sample_block; - } - - return block_without_partition_columns; -} - -} diff --git a/src/Storages/PartitionStrategy.h b/src/Storages/PartitionStrategy.h deleted file mode 100644 index 5b6ad219c128..000000000000 --- a/src/Storages/PartitionStrategy.h +++ /dev/null @@ -1,95 +0,0 @@ -#pragma once - -#include -#include - -#include - -namespace DB -{ - -struct PartitionStrategy -{ - struct PartitionExpressionActionsAndColumnName - { - ExpressionActionsPtr actions; - std::string column_name; - }; - - static std::unordered_map partition_strategy_to_wildcard_acceptance; - - PartitionStrategy(ASTPtr partition_by_, const Block & sample_block_, ContextPtr context_); - - virtual ~PartitionStrategy() = default; - - ColumnPtr computePartitionKey(const Chunk & chunk) - { - Block block_with_partition_by_expr = sample_block.cloneWithoutColumns(); - block_with_partition_by_expr.setColumns(chunk.getColumns()); - - return computePartitionKey(block_with_partition_by_expr); - } - - ColumnPtr computePartitionKey(Block & block) - { - actions_with_column_name.actions->execute(block); - - return block.getByName(actions_with_column_name.column_name).column; - } - - virtual Chunk getFormatChunk(const Chunk & chunk) { return chunk.clone(); } - - virtual Block getFormatHeader() { return sample_block; } - - const NamesAndTypesList & getPartitionColumns() const; - -protected: - ASTPtr partition_by; - Block sample_block; - ContextPtr context; - NamesAndTypesList partition_columns; - PartitionExpressionActionsAndColumnName actions_with_column_name; -}; - -struct PartitionStrategyFactory -{ - static std::shared_ptr get( - ASTPtr partition_by, - const Block & sample_block, - ContextPtr context, - bool globbed_path, - const std::string & partition_strategy, - bool partition_columns_in_data_file); - - static std::shared_ptr get( - ASTPtr partition_by, - const NamesAndTypesList & partition_columns, - ContextPtr context, - bool globbed_path, - const std::string & partition_strategy, - bool partition_columns_in_data_file); -}; - -struct StringifiedPartitionStrategy : PartitionStrategy -{ - StringifiedPartitionStrategy(ASTPtr partition_by_, const Block & sample_block_, ContextPtr context_); -}; - -struct HiveStylePartitionStrategy : PartitionStrategy -{ - HiveStylePartitionStrategy( - ASTPtr partition_by_, - const Block & sample_block_, - ContextPtr context_, - bool partition_columns_in_data_file_); - - Chunk getFormatChunk(const Chunk & chunk) override; - Block getFormatHeader() override; - -private: - bool partition_columns_in_data_file; - std::unordered_set partition_columns_name_set; - Block block_without_partition_columns; -}; - -} diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index a26ee505cce4..f2ca6372937e 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1997,7 +1997,7 @@ class PartitionedStorageFileSink : public PartitionedSink::SinkCreator } private: - std::shared_ptr partition_strategy; + std::shared_ptr partition_strategy; const String path; StorageMetadataPtr metadata_snapshot; String table_name_for_log; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index d82e410d29db..575f1cebb2b4 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -772,7 +772,7 @@ class PartitionedStorageURLSink : public PartitionedSink::SinkCreator } private: - std::shared_ptr partition_strategy; + std::shared_ptr partition_strategy; const String uri; const String format; const std::optional format_settings; From 92f2f33e370c60b21ddd58b9ea5811926cd27060 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 29 Jul 2025 08:58:22 -0300 Subject: [PATCH 04/48] extension to lower --- src/Storages/ObjectStorage/FilePathGenerator.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/FilePathGenerator.h b/src/Storages/ObjectStorage/FilePathGenerator.h index bc1d5d929829..27e0aa4f7c1d 100644 --- a/src/Storages/ObjectStorage/FilePathGenerator.h +++ b/src/Storages/ObjectStorage/FilePathGenerator.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -38,7 +39,7 @@ namespace DB const std::string & raw_path_, const std::string & file_format_, const std::shared_ptr & filename_generator_) - : raw_path(raw_path_), file_format(file_format_), filename_generator(filename_generator_){} + : raw_path(raw_path_), file_format(Poco::toLower(file_format_)), filename_generator(filename_generator_){} std::string getWritingPath(const std::string & partition_id, std::optional filename_override) const override { From 37ea31f645508404d139f0edaefa03358226e1b8 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 29 Jul 2025 18:03:10 -0300 Subject: [PATCH 05/48] add tests and fix prefix --- src/Core/Settings.cpp | 6 ++ .../ObjectStorage/StorageObjectStorage.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 24 ++++++- ...export_mt_part_to_object_storage.reference | 46 +++++++++++++ .../03572_export_mt_part_to_object_storage.sh | 66 +++++++++++++++++++ 5 files changed, 140 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03572_export_mt_part_to_object_storage.reference create mode 100755 tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 77db44fe35d7..db3cfafe702f 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -6874,6 +6874,12 @@ Possible values: DECLARE_WITH_ALIAS(Bool, allow_experimental_time_series_aggregate_functions, false, R"( Experimental timeSeries* aggregate functions for Prometheus-like timeseries resampling, rate, delta calculation. )", EXPERIMENTAL, allow_experimental_ts_to_grid_aggregate_function) \ + DECLARE_WITH_ALIAS(Bool, allow_experimental_export_merge_tree_partition, false, R"( +Experimental export merge tree partition. +)", EXPERIMENTAL, allow_experimental_export_merge_tree_partition) \ + DECLARE_WITH_ALIAS(Bool, export_merge_tree_partition_background_execution, true, R"( + Process exports asynchronously in background threads +)", EXPERIMENTAL, export_merge_tree_partition_background_execution) \ \ /* ####################################################### */ \ /* ############ END OF EXPERIMENTAL FEATURES ############# */ \ diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 001a75b8fc49..42e0e9d2c241 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -555,7 +555,7 @@ void StorageObjectStorage::importMergeTreePartition( return; RelativePathsWithMetadata relative_paths_with_metadata; - object_storage->listObjects("", relative_paths_with_metadata, 1000); + object_storage->listObjects(configuration->getRawPath().path, relative_paths_with_metadata, 1000); std::vector part_plans; part_plans.reserve(data_parts.size()); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c92f9a31a4cd..488d8387ffc6 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -74,6 +74,8 @@ namespace Setting extern const SettingsBool parallel_replicas_for_non_replicated_merge_tree; extern const SettingsBool throw_on_unsupported_query_inside_transaction; extern const SettingsUInt64 max_parts_to_move; + extern const SettingsBool export_merge_tree_partition_background_execution; + extern const SettingsBool allow_experimental_export_merge_tree_partition; } namespace MergeTreeSetting @@ -492,6 +494,13 @@ void StorageMergeTree::alter( * */ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, ContextPtr query_context) { + + if (!query_context->getSettingsRef()[Setting::allow_experimental_export_merge_tree_partition]) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Exporting merge tree partition is experimental. Set `allow_experimental_export_merge_tree_partition` 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); @@ -503,8 +512,7 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, String partition_id = getPartitionIDFromQuery(command.partition, getContext()); - background_moves_assignee.scheduleMoveTask(std::make_shared( - [this, query_context, partition_id, dest_storage] () mutable + auto export_partition_function = [this, query_context, partition_id, dest_storage] () mutable { const auto src_parts = getVisibleDataPartsVectorInPartition(getContext(), partition_id); @@ -560,7 +568,17 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, /// Perhaps this is a good way to trigger re-tries? return true; - }, moves_assignee_trigger, getStorageID())); + }; + + if (query_context->getSettingsRef()[Setting::export_merge_tree_partition_background_execution]) + { + background_moves_assignee.scheduleMoveTask( + std::make_shared(export_partition_function, moves_assignee_trigger, getStorageID())); + } + else + { + export_partition_function(); + } } /// While exists, marks parts as 'currently_merging_mutating_parts' and reserves free space on filesystem. diff --git a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.reference b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.reference new file mode 100644 index 000000000000..09f6b30f7552 --- /dev/null +++ b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.reference @@ -0,0 +1,46 @@ +---- Insert some more data in the already exported partition +test/s3_table_NAME/year=2020/2020_1_1_0.parquet +test/s3_table_NAME/year=2020/2020_1_1_0.parquet +test/s3_table_NAME/year=2020/2020_1_1_0.parquet +---- Export the parts, only the diff should be exported +---- New data part should appear (2020_2_2_0.parquet) with id 5 and 6 +test/s3_table_NAME/year=2020/2020_1_1_0.parquet +test/s3_table_NAME/year=2020/2020_1_1_0.parquet +test/s3_table_NAME/year=2020/2020_1_1_0.parquet +test/s3_table_NAME/year=2020/2020_3_3_0.parquet +test/s3_table_NAME/year=2020/2020_3_3_0.parquet +---- Merge all parts +---- Nothing should be exported even though the parts in the merge tree table have been merged +---- Check the parts in the remote storage have not been touched +test/s3_table_NAME/year=2020/2020_1_3_1.parquet +test/s3_table_NAME/year=2020/2020_1_3_1.parquet +test/s3_table_NAME/year=2020/2020_1_3_1.parquet +test/s3_table_NAME/year=2020/2020_1_3_1.parquet +test/s3_table_NAME/year=2020/2020_1_3_1.parquet +---- Yet another part +---- Merge the new part +---- The new part that cover everything should be exported +---- Assert there is only one data part +test/s3_table_NAME/year=2020/2020_1_4_2.parquet +test/s3_table_NAME/year=2020/2020_1_4_2.parquet +test/s3_table_NAME/year=2020/2020_1_4_2.parquet +test/s3_table_NAME/year=2020/2020_1_4_2.parquet +test/s3_table_NAME/year=2020/2020_1_4_2.parquet +test/s3_table_NAME/year=2020/2020_1_4_2.parquet +---- Finally, export the other partition (2021) +---- Assert both partitions are there +test/s3_table_NAME/year=2020/2020_1_4_2.parquet +test/s3_table_NAME/year=2020/2020_1_4_2.parquet +test/s3_table_NAME/year=2020/2020_1_4_2.parquet +test/s3_table_NAME/year=2021/2021_2_2_2.parquet +test/s3_table_NAME/year=2020/2020_1_4_2.parquet +test/s3_table_NAME/year=2020/2020_1_4_2.parquet +test/s3_table_NAME/year=2020/2020_1_4_2.parquet +---- Selecting from merge tree to have a sample of what the data in remote storage should look like +1 2020 +2 2020 +3 2020 +4 2021 +5 2020 +6 2020 +7 2020 diff --git a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh new file mode 100755 index 000000000000..ee46788c90cc --- /dev/null +++ b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh @@ -0,0 +1,66 @@ +#!/usr/bin/env bash + +mt_table="mt_table_${RANDOM}" +s3_table="s3_table_${RANDOM}" + +query() { + $CLICKHOUSE_CLIENT --query "$1" +} + +query "DROP TABLE IF EXISTS $mt_table, $s3_table" + +query "CREATE TABLE $mt_table (id UInt64, year UInt16) ENGINE = MergeTree() 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 "SYSTEM STOP MERGES" + +query "INSERT INTO $mt_table VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)" +query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_partition = 1, export_merge_tree_partition_background_execution = 0" + +echo "---- Insert some more data in the already exported partition" +query "SELECT DISTINCT ON (id) replaceRegexpAll(_path, '$s3_table', 's3_table_NAME') FROM $s3_table ORDER BY id" + +echo "---- Export the parts, only the diff should be exported" +query "INSERT INTO $mt_table VALUES (5, 2020), (6, 2020)" +query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_partition = 1, export_merge_tree_partition_background_execution = 0" + +echo "---- New data part should appear (2020_2_2_0.parquet) with id 5 and 6" +query "SELECT DISTINCT ON (id) replaceRegexpAll(_path, '$s3_table', 's3_table_NAME') FROM $s3_table ORDER BY id" + +echo "---- Merge all parts" +query "SYSTEM START MERGES" +query "OPTIMIZE TABLE $mt_table FINAL" +query "SYSTEM STOP MERGES" + +echo "---- Nothing should be exported even though the parts in the merge tree table have been merged" +query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_partition = 1, export_merge_tree_partition_background_execution = 0" + +echo "---- Check the parts in the remote storage have not been touched" +query "SELECT DISTINCT ON (id) replaceRegexpAll(_path, '$s3_table', 's3_table_NAME') FROM $s3_table ORDER BY id" + +echo "---- Yet another part" +query "INSERT INTO $mt_table VALUES (7, 2020)" + +echo "---- Merge the new part" +query "SYSTEM START MERGES" +query "OPTIMIZE TABLE $mt_table FINAL" +query "SYSTEM STOP MERGES" + +echo "---- The new part that cover everything should be exported" +query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_partition = 1, export_merge_tree_partition_background_execution = 0" + +echo "---- Assert there is only one data part" +query "SELECT DISTINCT ON (id) replaceRegexpAll(_path, '$s3_table', 's3_table_NAME') FROM $s3_table ORDER BY id" + +echo "---- Finally, export the other partition (2021)" +query "ALTER TABLE $mt_table EXPORT PARTITION ID '2021' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_partition = 1, export_merge_tree_partition_background_execution = 0" + +echo "---- Assert both partitions are there" +query "SELECT DISTINCT ON (id) replaceRegexpAll(_path, '$s3_table', 's3_table_NAME') FROM $s3_table ORDER BY id" + +echo "---- Selecting from merge tree to have a sample of what the data in remote storage should look like" +query "SELECT DISTINCT ON (id) id, year FROM $mt_table ORDER BY id" + +query "SYSTEM START MERGES" +query "DROP TABLE IF EXISTS $mt_table, $s3_table" + From 387cae41fa48b7e8fd594de95218f2c0d6f745f5 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 29 Jul 2025 19:36:40 -0300 Subject: [PATCH 06/48] fix test --- .../0_stateless/03572_export_mt_part_to_object_storage.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh index ee46788c90cc..2bf6f6ac3704 100755 --- a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh +++ b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh @@ -1,5 +1,9 @@ #!/usr/bin/env bash +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + mt_table="mt_table_${RANDOM}" s3_table="s3_table_${RANDOM}" From 43abc4c5233d7351e272e916b56f5932a23a3c2f Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 30 Jul 2025 13:46:16 -0300 Subject: [PATCH 07/48] reduce changes --- .../ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h | 2 -- src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h | 1 - 2 files changed, 3 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h index bc10960fa64d..679214d5c489 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadataDeltaKernel.h @@ -29,8 +29,6 @@ class DeltaLakeMetadataDeltaKernel final : public IDataLakeMetadata ObjectStoragePtr object_storage_, ConfigurationObserverPtr configuration_); - DeltaLakeMetadataDeltaKernel(const DeltaLakeMetadataDeltaKernel & other) : log(other.log), table_snapshot(other.table_snapshot) {} - bool supportsUpdate() const override { return true; } bool update(const ContextPtr & context) override; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h index 107d6c950864..41c54feb0ee7 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/SchemaProcessor.h @@ -78,7 +78,6 @@ class IcebergSchemaProcessor using Node = ActionsDAG::Node; public: - void addIcebergTableSchema(Poco::JSON::Object::Ptr schema_ptr); std::shared_ptr getClickhouseTableSchemaById(Int32 id); std::shared_ptr getSchemaTransformationDagByIds(Int32 old_id, Int32 new_id); From c7003ad33853b4e5e5877e9c89f1500e1d78047c Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 30 Jul 2025 13:49:28 -0300 Subject: [PATCH 08/48] reduce changes even further --- src/Storages/ObjectStorage/S3/Configuration.cpp | 15 --------------- src/Storages/System/StorageSystemMoves.cpp | 3 +-- 2 files changed, 1 insertion(+), 17 deletions(-) diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 790861fee194..dc552fad8518 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -168,21 +168,6 @@ void StorageS3Configuration::fromNamedCollection(const NamedCollection & collect else url = S3::URI(collection.get("url"), settings[Setting::allow_archive_path_syntax]); - if (collection.has("partition_strategy")) - { - const auto partition_strategy_name = collection.get("partition_strategy"); - const auto partition_strategy_type_opt = magic_enum::enum_cast(partition_strategy_name, magic_enum::case_insensitive); - - if (!partition_strategy_type_opt) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition strategy {} is not supported", partition_strategy_name); - } - - partition_strategy_type = partition_strategy_type_opt.value(); - } - - partition_columns_in_data_file = collection.getOrDefault("partition_columns_in_data_file", partition_strategy_type != PartitionStrategyFactory::StrategyType::HIVE); - auth_settings[S3AuthSetting::access_key_id] = collection.getOrDefault("access_key_id", ""); auth_settings[S3AuthSetting::secret_access_key] = collection.getOrDefault("secret_access_key", ""); auth_settings[S3AuthSetting::use_environment_credentials] = collection.getOrDefault("use_environment_credentials", 1); diff --git a/src/Storages/System/StorageSystemMoves.cpp b/src/Storages/System/StorageSystemMoves.cpp index 7faa6d58f0e4..2c43043d8207 100644 --- a/src/Storages/System/StorageSystemMoves.cpp +++ b/src/Storages/System/StorageSystemMoves.cpp @@ -27,9 +27,8 @@ void StorageSystemMoves::fillData(MutableColumns & res_columns, ContextPtr conte { const auto access = context->getAccess(); const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_TABLES); - const auto & move_list = context->getMovesList(); - for (const auto & move : move_list.get()) + for (const auto & move : context->getMovesList().get()) { if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, move.database, move.table)) continue; From bb156ab99a776273340e903b120d879c6a9cd078 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 30 Jul 2025 13:59:28 -0300 Subject: [PATCH 09/48] some adjustments --- src/Access/Common/AccessType.h | 1 + src/Interpreters/InterpreterAlterQuery.cpp | 4 ++-- src/Parsers/ASTAlterQuery.cpp | 2 +- src/Parsers/ASTAlterQuery.h | 2 +- src/Parsers/CommonParsers.h | 1 + src/Parsers/ParserAlterQuery.cpp | 23 +++++++++++++++++++--- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/PartitionCommands.cpp | 4 ++-- src/Storages/PartitionCommands.h | 2 +- 9 files changed, 30 insertions(+), 11 deletions(-) diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 961a90283160..d73c2f4fd1ad 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -74,6 +74,7 @@ enum class AccessType : uint8_t enabled implicitly by the grant ALTER_TABLE */\ 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_PARTITION, "ALTER EXPORT PART, EXPORT PARTITION, EXPORT PART", 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 f407a17aeedf..81c8a7e5b6d0 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -502,9 +502,9 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS required_access.emplace_back(AccessType::ALTER_DELETE | AccessType::INSERT, database, table); break; } - case ASTAlterCommand::EXPORT_PART: + case ASTAlterCommand::EXPORT_PARTITION: { - required_access.emplace_back(AccessType::ALTER_MOVE_PARTITION, database, 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/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index ab9d8cd1bf33..fac9724694de 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -355,7 +355,7 @@ void ASTAlterCommand::formatImpl(WriteBuffer & ostr, const FormatSettings & sett ostr << quoteString(move_destination_name); } } - else if (type == ASTAlterCommand::EXPORT_PART) + else if (type == ASTAlterCommand::EXPORT_PARTITION) { ostr << (settings.hilite ? hilite_keyword : "") << "EXPORT " << (part ? "PART " : "PARTITION ") << (settings.hilite ? hilite_none : ""); diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index e6f8a37997ba..7009e48fb7f5 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -71,7 +71,7 @@ class ASTAlterCommand : public IAST FREEZE_ALL, UNFREEZE_PARTITION, UNFREEZE_ALL, - EXPORT_PART, + EXPORT_PARTITION, DELETE, UPDATE, diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index a634df26603a..a0a59288b4a3 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -331,6 +331,7 @@ namespace DB MR_MACROS(MONTHS, "MONTHS") \ 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") \ diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index ace9bbbc566e..cb9c541f3a9f 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -82,7 +82,8 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_forget_partition(Keyword::FORGET_PARTITION); ParserKeyword s_move_partition(Keyword::MOVE_PARTITION); ParserKeyword s_move_part(Keyword::MOVE_PART); - ParserKeyword s_export_part(Keyword::EXPORT_PARTITION); + 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); @@ -565,13 +566,29 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->move_destination_name = ast_space_name->as().value.safeGet(); } } + 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_export_part.ignore(pos, expected)) { if (!parser_partition.parse(pos, command_partition, expected)) return false; - command->type = ASTAlterCommand::EXPORT_PART; - // command->part = true; + command->type = ASTAlterCommand::EXPORT_PARTITION; + command->part = true; if (!s_to_table.ignore(pos, expected)) { diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 305b828371f1..3c82f7a24187 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5940,7 +5940,7 @@ Pipe MergeTreeData::alterPartition( } } break; - case PartitionCommand::EXPORT_PART: + case PartitionCommand::EXPORT_PARTITION: { exportPartitionToTable(command, query_context); break; diff --git a/src/Storages/PartitionCommands.cpp b/src/Storages/PartitionCommands.cpp index 0d86bdc5802f..6fb3c2bfd6d3 100644 --- a/src/Storages/PartitionCommands.cpp +++ b/src/Storages/PartitionCommands.cpp @@ -52,10 +52,10 @@ std::optional PartitionCommand::parse(const ASTAlterCommand * res.part = command_ast->part; return res; } - if (command_ast->type == ASTAlterCommand::EXPORT_PART) + if (command_ast->type == ASTAlterCommand::EXPORT_PARTITION) { PartitionCommand res; - res.type = EXPORT_PART; + res.type = EXPORT_PARTITION; res.partition = command_ast->partition->clone(); res.part = command_ast->part; res.move_destination_type = PartitionCommand::MoveDestinationType::TABLE; diff --git a/src/Storages/PartitionCommands.h b/src/Storages/PartitionCommands.h index 15d2a7fb869f..6a844567d073 100644 --- a/src/Storages/PartitionCommands.h +++ b/src/Storages/PartitionCommands.h @@ -33,7 +33,7 @@ struct PartitionCommand UNFREEZE_ALL_PARTITIONS, UNFREEZE_PARTITION, REPLACE_PARTITION, - EXPORT_PART, + EXPORT_PARTITION, }; Type type = UNKNOWN; From bb742af11f4c6a451cb852d6662c9899716ff21b Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 30 Jul 2025 17:46:48 -0300 Subject: [PATCH 10/48] rmv unused files --- ...geObjectStorageMergeTreePartImportSink.cpp | 96 ------------------- ...rageObjectStorageMergeTreePartImportSink.h | 38 -------- 2 files changed, 134 deletions(-) delete mode 100644 src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImportSink.cpp delete mode 100644 src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImportSink.h diff --git a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImportSink.cpp b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImportSink.cpp deleted file mode 100644 index 0c729daf76f8..000000000000 --- a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImportSink.cpp +++ /dev/null @@ -1,96 +0,0 @@ -#include -#include -#include "Core/Settings.h" -#include "Disks/ObjectStorages/IObjectStorage.h" -#include "Disks/ObjectStorages/StoredObject.h" -#include "Formats/FormatFactory.h" -#include "IO/CompressionMethod.h" -#include "Processors/Formats/IOutputFormat.h" - -namespace DB -{ - -namespace Setting -{ -extern const SettingsUInt64 output_format_compression_level; -extern const SettingsUInt64 output_format_compression_zstd_window_log; -} - - -StorageObjectStorageMergeTreePartSink::StorageObjectStorageMergeTreePartSink( - const DataPartPtr & part, - ObjectStoragePtr object_storage, - const std::string & format, - const std::string & compression_method, - const std::optional & format_settings_, - const Block & sample_block_, - ContextPtr context) -: SinkToStorage(sample_block_) -{ - const auto & settings = context->getSettingsRef(); - const auto chosen_compression_method = chooseCompressionMethod(path, compression_method); - - auto buffer = object_storage->writeObject( - StoredObject(path), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, context->getWriteSettings()); - - write_buf = wrapWriteBufferWithCompressionMethod( - std::move(buffer), - chosen_compression_method, - static_cast(settings[Setting::output_format_compression_level]), - static_cast(settings[Setting::output_format_compression_zstd_window_log])); - - writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, format_settings_); -} - -void StorageObjectStorageMergeTreePartSink::consume(Chunk & chunk) -{ - if (isCancelled()) - return; - writer->write(getHeader().cloneWithColumns(chunk.getColumns())); -} - -void StorageObjectStorageMergeTreePartSink::onFinish() -{ - if (isCancelled()) - return; - - finalizeBuffers(); - releaseBuffers(); -} - -void StorageObjectStorageMergeTreePartSink::finalizeBuffers() -{ - if (!writer) - return; - - try - { - writer->flush(); - writer->finalize(); - } - catch (...) - { - /// Stop ParallelFormattingOutputFormat correctly. - cancelBuffers(); - releaseBuffers(); - throw; - } - - write_buf->finalize(); -} - -void StorageObjectStorageMergeTreePartSink::releaseBuffers() -{ - writer.reset(); - write_buf.reset(); -} - -void StorageObjectStorageMergeTreePartSink::cancelBuffers() -{ - if (writer) - writer->cancel(); - if (write_buf) - write_buf->cancel(); -} - -} diff --git a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImportSink.h b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImportSink.h deleted file mode 100644 index 80aa6c19dea1..000000000000 --- a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImportSink.h +++ /dev/null @@ -1,38 +0,0 @@ -#pragma once - -#include -#include -#include "Storages/MergeTree/RangesInDataPart.h" - -namespace DB -{ -class StorageObjectStorageMergeTreePartSink : public SinkToStorage -{ -public: - StorageObjectStorageMergeTreePartSink( - const DataPartPtr & part, - ObjectStoragePtr object_storage, - const std::string & format, - const std::string & compression_method, - const std::optional & format_settings_, - const Block & sample_block_, - ContextPtr context); - - String getName() const override { return "StorageObjectStorageMergeTreePartSink"; } - - void consume(Chunk & chunk) override; - - void onFinish() override; - -private: - const String path; - const Block sample_block; - std::unique_ptr write_buf; - OutputFormatPtr writer; - - void finalizeBuffers(); - void releaseBuffers(); - void cancelBuffers(); -}; - -} From 4bac44a74caed82550bf634c52498d5460b4e106 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 30 Jul 2025 19:47:22 -0300 Subject: [PATCH 11/48] rename a few things --- src/CMakeLists.txt | 1 + ...ObjectStorageMergeTreePartImporterSink.cpp | 63 +++++++++++ ...geObjectStorageMergeTreePartImporterSink.h | 64 +++++++++++ ...ObjectStorageSinkMTPartImportDecorator.cpp | 0 ...geObjectStorageSinkMTPartImportDecorator.h | 102 ------------------ .../ObjectStorage/StorageObjectStorage.cpp | 10 +- .../ObjectStorage/StorageObjectStorageSink.h | 1 + src/Storages/StorageMergeTree.cpp | 2 +- 8 files changed, 135 insertions(+), 108 deletions(-) create mode 100644 src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.cpp create mode 100644 src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.h delete mode 100644 src/Storages/ObjectStorage/MergeTree/StorageObjectStorageSinkMTPartImportDecorator.cpp delete mode 100644 src/Storages/ObjectStorage/MergeTree/StorageObjectStorageSinkMTPartImportDecorator.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 0f7a3cf851b1..510c8b7bfa4b 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -130,6 +130,7 @@ add_headers_and_sources(dbms Storages/ObjectStorage/Azure) add_headers_and_sources(dbms Storages/ObjectStorage/S3) add_headers_and_sources(dbms Storages/ObjectStorage/HDFS) add_headers_and_sources(dbms Storages/ObjectStorage/Local) +add_headers_and_sources(dbms Storages/ObjectStorage/MergeTree) add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes) add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes/Iceberg) add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes/DeltaLake) diff --git a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.cpp b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.cpp new file mode 100644 index 000000000000..be8bb291f4b3 --- /dev/null +++ b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.cpp @@ -0,0 +1,63 @@ +#include + +namespace DB +{ + +StorageObjectStorageMergeTreePartImporterSink::StorageObjectStorageMergeTreePartImporterSink( + const DataPartPtr & part_, + const std::string & path_, + const ObjectStoragePtr & object_storage_, + const ConfigurationPtr & configuration_, + const std::optional & format_settings_, + const Block & sample_block_, + const std::function & part_log_, + const ContextPtr & context_) + : SinkToStorage(sample_block_) + , object_storage(object_storage_) + , configuration(configuration_) + , format_settings(format_settings_) + , sample_block(sample_block_) + , context(context_) + , part_log(part_log_) +{ + stats.part = part_; + stats.file_path = path_; + sink = std::make_shared( + stats.file_path, + object_storage, + configuration, + format_settings, + sample_block, + context); +} + +String StorageObjectStorageMergeTreePartImporterSink::getName() const +{ + return "StorageObjectStorageMergeTreePartImporterSink"; +} + +void StorageObjectStorageMergeTreePartImporterSink::consume(Chunk & chunk) +{ + sink->consume(chunk); + + stats.read_bytes += chunk.bytes(); + stats.read_rows += chunk.getNumRows(); +} + +void StorageObjectStorageMergeTreePartImporterSink::onFinish() +{ + sink->onFinish(); + if (const auto object_metadata = object_storage->tryGetObjectMetadata(stats.file_path)) + { + stats.bytes_on_disk = object_metadata->size_bytes; + } + part_log(stats); +} + +void StorageObjectStorageMergeTreePartImporterSink::onException(std::exception_ptr exception) +{ + sink->onException(exception); + part_log(stats); +} + +} diff --git a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.h b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.h new file mode 100644 index 000000000000..d5f1677c69ae --- /dev/null +++ b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.h @@ -0,0 +1,64 @@ +#pragma once + +#include +#include +#include "Core/Settings.h" +#include "Disks/ObjectStorages/IObjectStorage.h" +#include "Disks/ObjectStorages/StoredObject.h" +#include "Formats/FormatFactory.h" +#include "IO/CompressionMethod.h" +#include "Processors/Formats/IOutputFormat.h" +#include "Storages/MergeTree/IMergeTreeDataPart.h" + +namespace DB +{ + +struct MergeTreePartImportStats +{ + ExecutionStatus status; + std::size_t bytes_on_disk = 0; + std::size_t read_rows = 0; + std::size_t read_bytes = 0; + std::string file_path = ""; + DataPartPtr part = nullptr; +}; + +/* + * Wrapper around `StorageObjectsStorageSink` that takes care of accounting & metrics for partition export + */ +class StorageObjectStorageMergeTreePartImporterSink : public SinkToStorage +{ +public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + + StorageObjectStorageMergeTreePartImporterSink( + const DataPartPtr & part_, + const std::string & path_, + const ObjectStoragePtr & object_storage_, + const ConfigurationPtr & configuration_, + const std::optional & format_settings_, + const Block & sample_block_, + const std::function & part_log_, + const ContextPtr & context_); + + String getName() const override; + + void consume(Chunk & chunk) override; + + void onFinish() override; + + void onException(std::exception_ptr exception) override; + +private: + std::shared_ptr sink; + ObjectStoragePtr object_storage; + ConfigurationPtr configuration; + std::optional format_settings; + Block sample_block; + ContextPtr context; + std::function part_log; + + MergeTreePartImportStats stats; +}; + +} diff --git a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageSinkMTPartImportDecorator.cpp b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageSinkMTPartImportDecorator.cpp deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageSinkMTPartImportDecorator.h b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageSinkMTPartImportDecorator.h deleted file mode 100644 index 5d0f3a0e45f7..000000000000 --- a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageSinkMTPartImportDecorator.h +++ /dev/null @@ -1,102 +0,0 @@ -#pragma once - -#include -#include -#include "Core/Settings.h" -#include "Disks/ObjectStorages/IObjectStorage.h" -#include "Disks/ObjectStorages/StoredObject.h" -#include "Formats/FormatFactory.h" -#include "IO/CompressionMethod.h" -#include "Processors/Formats/IOutputFormat.h" -#include "Storages/MergeTree/IMergeTreeDataPart.h" - -namespace DB -{ - -struct MergeTreePartImportStats -{ - ExecutionStatus status; - std::size_t bytes_on_disk = 0; - std::size_t read_rows = 0; - std::size_t read_bytes = 0; - std::string file_path = ""; - DataPartPtr part = nullptr; -}; - -class StorageObjectStorageSinkMTPartImportDecorator : public SinkToStorage -{ -public: - using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; - - StorageObjectStorageSinkMTPartImportDecorator( - const DataPartPtr & part_, - const std::string & path_, - const ObjectStoragePtr & object_storage_, - const ConfigurationPtr & configuration_, - const std::optional & format_settings_, - const Block & sample_block_, - const std::function & part_log_, - const ContextPtr & context_) - : SinkToStorage(sample_block_) - , object_storage(object_storage_) - , configuration(configuration_) - , format_settings(format_settings_) - , sample_block(sample_block_) - , context(context_) - , part_log(part_log_) - { - stats.part = part_; - stats.file_path = path_; - } - - String getName() const override { return "StorageObjectStorageSinkMTPartImportDecorator"; } - - void consume(Chunk & chunk) override - { - if (!sink) - { - sink = std::make_shared( - stats.file_path, - object_storage, - configuration, - format_settings, - sample_block, - context - ); - } - - stats.read_bytes += chunk.bytes(); - stats.read_rows += chunk.getNumRows(); - - sink->consume(chunk); - } - - void onFinish() override - { - sink->onFinish(); - if (const auto object_metadata = object_storage->tryGetObjectMetadata(stats.file_path)) - { - stats.bytes_on_disk = object_metadata->size_bytes; - } - part_log(stats); - } - - void onException(std::exception_ptr exception) override - { - SinkToStorage::onException(exception); - part_log(stats); - } - -private: - std::shared_ptr sink; - ObjectStoragePtr object_storage; - ConfigurationPtr configuration; - std::optional format_settings; - Block sample_block; - ContextPtr context; - std::function part_log; - - MergeTreePartImportStats stats; -}; - -} diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 42e0e9d2c241..c3069de61706 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -1,15 +1,15 @@ #include #include #include -#include "MergeTree/StorageObjectStorageSinkMTPartImportDecorator.h" +#include "MergeTree/StorageObjectStorageMergeTreePartImporterSink.h" -#include #include #include -#include #include -#include #include +#include +#include +#include #include #include @@ -663,7 +663,7 @@ void StorageObjectStorage::importMergeTreePartition( auto builder = plan_for_part.buildQueryPipeline(optimization_settings, pipeline_settings); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); - auto sink = std::make_shared( + auto sink = std::make_shared( data_part, file_path, object_storage, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h index d515d9c6137a..41ce4a07dda6 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -8,6 +8,7 @@ namespace DB { class StorageObjectStorageSink : public SinkToStorage { +friend class StorageObjectStorageMergeTreePartImporterSink; public: using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 488d8387ffc6..a634e7600daa 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -48,7 +48,7 @@ #include #include "Core/BackgroundSchedulePool.h" #include "Core/Names.h" -#include "ObjectStorage/MergeTree/StorageObjectStorageSinkMTPartImportDecorator.h" +#include "ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.h" namespace DB { From ea3a2a565361cee28377cb448e9d83e436345c76 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 19 Aug 2025 10:11:23 -0300 Subject: [PATCH 12/48] rewind the part names logic --- .../ObjectStorage/FilePathGenerator.h | 8 ++-- .../ObjectStorage/StorageObjectStorage.cpp | 39 +------------------ .../StorageObjectStorageSink.cpp | 6 +-- .../ObjectStorage/StorageObjectStorageSink.h | 4 +- 4 files changed, 8 insertions(+), 49 deletions(-) diff --git a/src/Storages/ObjectStorage/FilePathGenerator.h b/src/Storages/ObjectStorage/FilePathGenerator.h index 27e0aa4f7c1d..c2ba6a6f5d94 100644 --- a/src/Storages/ObjectStorage/FilePathGenerator.h +++ b/src/Storages/ObjectStorage/FilePathGenerator.h @@ -10,7 +10,7 @@ namespace DB struct ObjectStorageFilePathGenerator { virtual ~ObjectStorageFilePathGenerator() = default; - virtual std::string getWritingPath(const std::string & partition_id, std::optional filename_override = {}) const = 0; + virtual std::string getWritingPath(const std::string & partition_id) const = 0; virtual std::string getReadingPath() const = 0; }; @@ -18,7 +18,7 @@ namespace DB { explicit ObjectStorageWildcardFilePathGenerator(const std::string & raw_path_) : raw_path(raw_path_) {} - std::string getWritingPath(const std::string & partition_id, std::optional /**/ = {}) const override + std::string getWritingPath(const std::string & partition_id) const override { return PartitionedSink::replaceWildcards(raw_path, partition_id); } @@ -41,9 +41,9 @@ namespace DB const std::shared_ptr & filename_generator_) : raw_path(raw_path_), file_format(Poco::toLower(file_format_)), filename_generator(filename_generator_){} - std::string getWritingPath(const std::string & partition_id, std::optional filename_override) const override + std::string getWritingPath(const std::string & partition_id) const override { - return raw_path + "/" + partition_id + "/" + (filename_override ? *filename_override : filename_generator->generate()) + "." + file_format; + return raw_path + "/" + partition_id + "/" + filename_generator->generate() + "." + file_format; } std::string getReadingPath() const override diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index c3069de61706..66f6f90bd49f 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -554,9 +554,6 @@ void StorageObjectStorage::importMergeTreePartition( if (data_parts.empty()) return; - RelativePathsWithMetadata relative_paths_with_metadata; - object_storage->listObjects(configuration->getRawPath().path, relative_paths_with_metadata, 1000); - std::vector part_plans; part_plans.reserve(data_parts.size()); @@ -580,44 +577,13 @@ void StorageObjectStorage::importMergeTreePartition( std::vector files_to_be_deleted; for (const auto & data_part : data_parts) { - bool upload_part = true; - for (const auto & object_with_metadata : relative_paths_with_metadata) - { - const auto remote_object_filename = object_with_metadata->getFileNameWithoutExtension(); - if (remote_object_filename == data_part->name) - { - upload_part = false; - break; - } - - const auto remote_fake_part = MergeTreePartInfo::tryParsePartName(remote_object_filename, merge_tree_data.format_version); - - if (!remote_fake_part) - { - continue; - } - - /// If the part does not intersect, proceed to the next file - if (data_part->info.isDisjoint(remote_fake_part.value())) - { - continue; - } - - files_to_be_deleted.emplace_back(object_with_metadata->relative_path); - } - - if (!upload_part) - { - continue; - } - const auto partition_columns = configuration->partition_strategy->getPartitionColumns(); auto block_with_partition_values = data_part->partition.getBlockWithPartitionValues(partition_columns); const auto column_with_partition_key = configuration->partition_strategy->computePartitionKey(block_with_partition_values); - const auto file_path = configuration->file_path_generator->getWritingPath(column_with_partition_key->getDataAt(0).toString(), data_part->name); + const auto file_path = configuration->file_path_generator->getWritingPath(column_with_partition_key->getDataAt(0).toString()); export_list_entries.emplace_back(local_context->getGlobalContext()->getExportsList().insert( merge_tree_data.getStorageID(), @@ -683,9 +649,6 @@ void StorageObjectStorage::importMergeTreePartition( { root_pipeline.setNumThreads(local_context->getSettingsRef()[Setting::max_threads]); - /// shouldn't this be part of the sink and or pipeline? - object_storage->removeObjectsIfExist(files_to_be_deleted); - CompletedPipelineExecutor exec(root_pipeline); exec.execute(); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index 7eb24fa85d6c..08a272c9d1f2 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -133,8 +133,7 @@ PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( const std::shared_ptr & file_path_generator_, std::optional format_settings_, const Block & sample_block_, - ContextPtr context_, - std::optional filename_override_) + ContextPtr context_) : object_storage(object_storage_) , configuration(configuration_) , file_path_generator(file_path_generator_) @@ -142,7 +141,6 @@ PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( , format_settings(format_settings_) , sample_block(sample_block_) , context(context_) - , filename_override(filename_override_) { } @@ -154,7 +152,7 @@ StorageObjectStorageSink::~StorageObjectStorageSink() SinkPtr PartitionedStorageObjectStorageSink::createSinkForPartition(const String & partition_id) { - auto file_path = file_path_generator->getWritingPath(partition_id, filename_override); + auto file_path = file_path_generator->getWritingPath(partition_id); validateNamespace(configuration->getNamespace(), configuration); validateKey(file_path); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h index 41ce4a07dda6..d46bd53bb355 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -50,8 +50,7 @@ class PartitionedStorageObjectStorageSink : public PartitionedSink::SinkCreator const std::shared_ptr & file_path_generator_, std::optional format_settings_, const Block & sample_block_, - ContextPtr context_, - std::optional filename_override_ = std::nullopt); + ContextPtr context_); SinkPtr createSinkForPartition(const String & partition_id) override; @@ -64,7 +63,6 @@ class PartitionedStorageObjectStorageSink : public PartitionedSink::SinkCreator const std::optional format_settings; const Block sample_block; const ContextPtr context; - std::optional filename_override; }; } From 180fda88bbe58d14fd200f250bc80d04dacc8525 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 21 Aug 2025 17:10:18 -0300 Subject: [PATCH 13/48] tmp --- .../MergeTree/MergeTreeExportPartEntry.h | 104 +++++++ ...ObjectStorageMergeTreePartImporterSink.cpp | 1 + .../ObjectStorage/StorageObjectStorage.cpp | 37 ++- src/Storages/StorageMergeTree.cpp | 282 +++++++++++++++--- src/Storages/StorageMergeTree.h | 31 +- 5 files changed, 399 insertions(+), 56 deletions(-) create mode 100644 src/Storages/MergeTree/MergeTreeExportPartEntry.h diff --git a/src/Storages/MergeTree/MergeTreeExportPartEntry.h b/src/Storages/MergeTree/MergeTreeExportPartEntry.h new file mode 100644 index 000000000000..f71e6682cc2f --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeExportPartEntry.h @@ -0,0 +1,104 @@ +#pragma once + +#include +#include +#include +#include "IO/WriteBufferFromFileBase.h" +#include "IO/ReadBuffer.h" +#include "IO/WriteHelpers.h" +#include "IO/ReadHelpers.h" +#include "Common/Exception.h" +#include + +namespace DB +{ + +struct MergeTreeExportPartEntry +{ + String part_name; + String commit_id; + std::filesystem::path file_path; + DiskPtr disk; + StorageID destination_storage_id; // Added StorageID field + + MergeTreeExportPartEntry(const String & part_name_, const String & commit_id_, DiskPtr disk_, + const String & path_prefix, const StorageID & destination_storage_id_) + : part_name(part_name_) + , commit_id(commit_id_) + , file_path(std::filesystem::path(path_prefix) / ("export_part_" + part_name + "_" + commit_id)) + , disk(std::move(disk_)) + , destination_storage_id(destination_storage_id_) + {} + + MergeTreeExportPartEntry(DiskPtr disk_, const String & file_path_) + : file_path(file_path_) + , disk(std::move(disk_)) + , destination_storage_id(StorageID::createEmpty()) // Initialize with empty StorageID + { + constexpr std::string_view prefix = "export_part_"; + + const auto & filename = file_path.filename().string(); + + if (!filename.starts_with(prefix)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Export part entry file path does not start with 'export_part_': {}", file_path_); + + auto rest = filename.substr(prefix.size()); + + auto pos = rest.rfind('_'); + if (pos == std::string::npos) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Export part entry file name does not contain commit id: {}", filename); + + part_name = rest.substr(0, pos); + commit_id = rest.substr(pos + 1); + + // Parse StorageID from the file content + destination_storage_id = parseStorageIDFromFile(); + } + + void write() const + { + try + { + auto out = disk->writeFile(file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite); + + // Serialize the QualifiedTableName (database.table format) + String serialized_qualified_name = destination_storage_id.getQualifiedName().getFullName(); + writeString(serialized_qualified_name, *out); + + out->finalize(); + out->sync(); + } + catch (const Exception &) + { + remove(); + throw; + } + } + + void remove() const + { + disk->removeFileIfExists(file_path); + } + +private: + StorageID parseStorageIDFromFile() const + { + try + { + auto in = disk->readFile(file_path, ReadSettings{}); + + String serialized_qualified_name; + readString(serialized_qualified_name, *in); + + return StorageID(QualifiedTableName::parseFromString(serialized_qualified_name)); + } + catch (const Exception &) + { + // If file doesn't exist or is corrupted, return empty StorageID + // This handles backward compatibility with existing files + return StorageID::createEmpty(); + } + } +}; + +} diff --git a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.cpp b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.cpp index be8bb291f4b3..fc7943b467b0 100644 --- a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.cpp +++ b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.cpp @@ -57,6 +57,7 @@ void StorageObjectStorageMergeTreePartImporterSink::onFinish() void StorageObjectStorageMergeTreePartImporterSink::onException(std::exception_ptr exception) { sink->onException(exception); + stats.status = ExecutionStatus(-1, "Error importing part"); part_log(stats); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 66f6f90bd49f..f1d5a61e09e7 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -22,6 +22,8 @@ #include #include #include +#include +#include #include #include #include @@ -570,11 +572,10 @@ void StorageObjectStorage::importMergeTreePartition( bool read_with_direct_io = false; bool prefetch = false; - QueryPipeline root_pipeline; + QueryPipeline export_pipeline; std::vector export_list_entries; - std::vector files_to_be_deleted; for (const auto & data_part : data_parts) { const auto partition_columns = configuration->partition_strategy->getPartitionColumns(); @@ -641,17 +642,38 @@ void StorageObjectStorage::importMergeTreePartition( ); pipeline.complete(sink); + export_pipeline.addCompletedPipeline(std::move(pipeline)); + } - root_pipeline.addCompletedPipeline(std::move(pipeline)); + if (!export_pipeline.completed()) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Root pipeline is not completed"); } - if (root_pipeline.completed()) + export_pipeline.setNumThreads(local_context->getSettingsRef()[Setting::max_threads]); + + CompletedPipelineExecutor exec(export_pipeline); + exec.execute(); + + Strings exported_paths; + exported_paths.reserve(export_list_entries.size()); + for (const auto & entry : export_list_entries) { - root_pipeline.setNumThreads(local_context->getSettingsRef()[Setting::max_threads]); + const auto info = entry->operator->()->getInfo(); + exported_paths.emplace_back(info.destination_path); + } - CompletedPipelineExecutor exec(root_pipeline); - exec.execute(); + // Generate commit id and write commit file + pcg64_fast rng(randomSeed()); + const String commit_id = toString(rng()); + const String commit_object = configuration->getRawPath().path + "/commit_" + commit_id; + 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(); } SinkToStoragePtr StorageObjectStorage::write( @@ -1007,3 +1029,4 @@ void StorageObjectStorage::Configuration::assertInitialized() const } } + diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a634e7600daa..a36afe31337a 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -49,6 +49,7 @@ #include "Core/BackgroundSchedulePool.h" #include "Core/Names.h" #include "ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.h" +#include "Storages/MergeTree/MergeTreeExportPartEntry.h" namespace DB { @@ -175,6 +176,7 @@ StorageMergeTree::StorageMergeTree( increment.set(getMaxBlockNumber()); loadMutations(); + loadExportPartition(); loadDeduplicationLog(); prewarmCaches( @@ -488,13 +490,95 @@ void StorageMergeTree::alter( } } +void StorageMergeTree::exportPartsImpl( + const std::vector & src_parts, + const String & /*commit_id*/, + const StoragePtr & dest_storage) +{ + /// todo should stopMergesAndWait be accounted as well? + const auto start_time = std::chrono::system_clock::now(); + + dest_storage->importMergeTreePartition(*this, src_parts, getContext(), [&](MergeTreePartImportStats stats) + { + auto table_id = getStorageID(); + + if (stats.status.code != 0) + { + LOG_ERROR(log, "Error importing part {}: {}", stats.part->name, stats.status.message); + return; + } + + auto export_entry_it = export_part_entries.end(); + for (auto it = export_part_entries.begin(); it != export_part_entries.end(); ++it) + { + if (it->second.part_name == stats.part->name) + { + export_entry_it = it; + break; + } + } + + if (export_entry_it == export_part_entries.end()) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Export entry not found for part {}", stats.part->name); + } + + export_entry_it->second.remove(); + export_part_entries.erase(export_entry_it); + + auto part_log = getContext()->getPartLog(table_id.database_name); + if (!part_log) + return; + + PartLogElement part_log_elem; + part_log_elem.event_type = PartLogElement::Type::EXPORT_PART; + part_log_elem.merge_algorithm = PartLogElement::PartMergeAlgorithm::UNDECIDED; + part_log_elem.merge_reason = PartLogElement::MergeReasonType::NOT_A_MERGE; + + part_log_elem.database_name = table_id.database_name; + part_log_elem.table_name = table_id.table_name; + part_log_elem.table_uuid = table_id.uuid; + part_log_elem.partition_id = MergeTreePartInfo::fromPartName(stats.part->name, format_version).getPartitionId(); + // construct event_time and event_time_microseconds using the same time point + // so that the two times will always be equal up to a precision of a second. + const auto time_now = std::chrono::system_clock::now(); + part_log_elem.event_time = timeInSeconds(time_now); + part_log_elem.event_time_microseconds = timeInMicroseconds(time_now); + + part_log_elem.duration_ms = std::chrono::duration_cast(start_time - time_now).count() / 1000000; + part_log_elem.error = static_cast(stats.status.code); + part_log_elem.exception = stats.status.message; + part_log_elem.path_on_disk = stats.file_path; + part_log_elem.part_name = stats.part->name; + part_log_elem.bytes_compressed_on_disk = stats.bytes_on_disk; + part_log_elem.rows = stats.part->rows_count; + part_log_elem.disk_name = dest_storage->getName(); + part_log_elem.part_type = stats.part->getType(); + part_log_elem.source_part_names = {stats.part->name}; + part_log_elem.rows_read = stats.read_rows; + part_log_elem.bytes_read_uncompressed = stats.read_bytes; + + part_log->add(std::move(part_log_elem)); + }); + + for (const auto & part : src_parts) + { + if (!currently_merging_mutating_parts.contains(part)) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} is not in currently_merging_mutating_parts", part->name); + } + + currently_merging_mutating_parts.erase(part); + dropPart(part->name, false, getContext()); + } +} + /* * For now, this function is meant to be used when exporting to different formats (i.e, the case where data needs to be re-encoded / serialized) * For the cases where this is not necessary, there are way more optimal ways of doing that, such as hard links implemented by `movePartitionToTable` * */ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, ContextPtr query_context) { - if (!query_context->getSettingsRef()[Setting::allow_experimental_export_merge_tree_partition]) { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, @@ -514,6 +598,8 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, auto export_partition_function = [this, query_context, partition_id, dest_storage] () mutable { + auto lock1 = lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef()[Setting::lock_acquire_timeout]); + auto merges_blocker = stopMergesAndWait(); const auto src_parts = getVisibleDataPartsVectorInPartition(getContext(), partition_id); if (src_parts.empty()) @@ -521,52 +607,30 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, return true; } - auto lock1 = lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef()[Setting::lock_acquire_timeout]); - auto lock2 = dest_storage->lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef()[Setting::lock_acquire_timeout]); - auto merges_blocker = stopMergesAndWait(); - - /// todo should stopMergesAndWait be accounted as well? - const auto start_time = std::chrono::system_clock::now(); + std::string commit_id; + { + std::random_device rd; + std::mt19937_64 gen(rd()); + std::uniform_int_distribution dis; + uint64_t random_value = dis(gen); + std::ostringstream oss; + oss << std::hex << random_value; + commit_id = oss.str(); + } - dest_storage->importMergeTreePartition(*this, src_parts, getContext(), [&](MergeTreePartImportStats stats) + for (const auto & part : src_parts) { - auto table_id = getStorageID(); - auto part_log = getContext()->getPartLog(table_id.database_name); - if (!part_log) - return; - - PartLogElement part_log_elem; - part_log_elem.event_type = PartLogElement::Type::EXPORT_PART; - part_log_elem.merge_algorithm = PartLogElement::PartMergeAlgorithm::UNDECIDED; - part_log_elem.merge_reason = PartLogElement::MergeReasonType::NOT_A_MERGE; - - part_log_elem.database_name = table_id.database_name; - part_log_elem.table_name = table_id.table_name; - part_log_elem.table_uuid = table_id.uuid; - part_log_elem.partition_id = MergeTreePartInfo::fromPartName(stats.part->name, format_version).getPartitionId(); - // construct event_time and event_time_microseconds using the same time point - // so that the two times will always be equal up to a precision of a second. - const auto time_now = std::chrono::system_clock::now(); - part_log_elem.event_time = timeInSeconds(time_now); - part_log_elem.event_time_microseconds = timeInMicroseconds(time_now); - - part_log_elem.duration_ms = std::chrono::duration_cast(start_time - time_now).count() / 1000000; - part_log_elem.error = static_cast(stats.status.code); - part_log_elem.exception = stats.status.message; - part_log_elem.path_on_disk = stats.file_path; - part_log_elem.part_name = stats.part->name; - part_log_elem.bytes_compressed_on_disk = stats.bytes_on_disk; - part_log_elem.rows = stats.part->rows_count; - part_log_elem.disk_name = dest_storage->getName(); - part_log_elem.part_type = stats.part->getType(); - part_log_elem.source_part_names = {stats.part->name}; - part_log_elem.rows_read = stats.read_rows; - part_log_elem.bytes_read_uncompressed = stats.read_bytes; - - part_log->add(std::move(part_log_elem)); - }); + MergeTreeExportPartEntry entry(part->name, commit_id, getStoragePolicy()->getAnyDisk(), relative_data_path, dest_storage->getStorageID()); + entry.write(); + export_part_entries.emplace(entry.commit_id, std::move(entry)); + + // do I need the lock if I already did `stopMergesAndWait`? + // std::lock_guard lock(currently_processing_in_background_mutex); + currently_merging_mutating_parts.insert(part); + } + + exportPartsImpl(src_parts, commit_id, dest_storage); - /// Perhaps this is a good way to trigger re-tries? return true; }; @@ -1098,6 +1162,136 @@ void StorageMergeTree::loadMutations() increment.value = std::max(increment.value.load(), current_mutations_by_version.rbegin()->first); } +void StorageMergeTree::loadExportPartition() +{ + std::lock_guard lock(currently_processing_in_background_mutex); + + for (const auto & disk : getDisks()) + { + for (auto it = disk->iterateDirectory(relative_data_path); it->isValid(); it->next()) + { + if (startsWith(it->name(), "export_part_")) + { + MergeTreeExportPartEntry entry(disk, it->path()); + + /// Check if part still exists + auto part = getPartIfExists(entry.part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + + if (!part) + { + LOG_WARNING(log, "Part {} not found during export partition load, removing export entry", entry.part_name); + entry.remove(); + continue; + } + + // Lock the part to prevent merging + auto currently_mutating_parts_insert_result = currently_merging_mutating_parts.insert(part); + if (!currently_mutating_parts_insert_result.second) + { + LOG_WARNING(log, "Part {} already exists in currently_merging_mutating_parts during export load", entry.part_name); + continue; + } + + LOG_INFO(log, "Loaded export part entry: {} -> {}", entry.part_name, entry.destination_storage_id.getNameForLogs()); + + // Add to export entries + export_part_entries.emplace(entry.commit_id, std::move(entry)); + } + } + } + + // Restart export process for any remaining entries + if (!export_part_entries.empty()) + { + restartExportProcess(); + } +} + +void StorageMergeTree::restartExportProcess() +{ + /// no need for locking because currently_processing_in_background_mutex is already locked and won't allow those parts to be merged + + // Group parts by commit_id to restart exports + std::map> parts_by_commit; + + for (const auto & entry : export_part_entries) + { + auto part = getPartIfExists(entry.second.part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + if (part) + { + parts_by_commit[entry.second.commit_id].push_back(part); + } + } + + // Restart export for each commit group + for (const auto & [commit_id, parts] : parts_by_commit) + { + if (!parts.empty()) + { + // Get destination storage from the first part's entry + auto entry_it = export_part_entries.find(commit_id); + if (entry_it != export_part_entries.end()) + { + StoragePtr destination_storage = reconstructDestinationStorage(entry_it->second.destination_storage_id); + if (destination_storage) + { + LOG_INFO(log, "Restarting export for {} parts with commit_id: {} -> {}", + parts.size(), commit_id, entry_it->second.destination_storage_id.getNameForLogs()); + + background_moves_assignee.scheduleMoveTask( + std::make_shared( + [this, parts, commit_id, destination_storage]() + { + exportPartsImpl(parts, commit_id, destination_storage); + return true; + }, + moves_assignee_trigger, + getStorageID() + ) + ); + } + else + { + LOG_ERROR(log, "Failed to reconstruct destination storage for commit_id: {}", commit_id); + } + } + } + } +} + +StoragePtr StorageMergeTree::reconstructDestinationStorage(const StorageID & storage_id) const +{ + try + { + if (storage_id.empty()) + { + LOG_WARNING(log, "Empty StorageID provided for destination storage reconstruction"); + return nullptr; + } + + // Resolve the StorageID to get the actual storage + auto resolved_storage_id = getContext()->resolveStorageID(storage_id); + + // Get the storage from DatabaseCatalog + auto storage = DatabaseCatalog::instance().tryGetTable(resolved_storage_id, getContext()); + + if (!storage) + { + LOG_ERROR(log, "Failed to find destination storage: {}", storage_id.getNameForLogs()); + return nullptr; + } + + LOG_DEBUG(log, "Successfully reconstructed destination storage: {}", storage_id.getNameForLogs()); + return storage; + } + catch (const Exception & e) + { + LOG_ERROR(log, "Exception while reconstructing destination storage {}: {}", + storage_id.getNameForLogs(), e.message()); + return nullptr; + } +} + std::expected StorageMergeTree::selectPartsToMerge( const StorageMetadataPtr & metadata_snapshot, bool aggressive, diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index c28f965f88c7..95d36738e17e 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -17,6 +17,7 @@ #include #include +#include "Storages/MergeTree/MergeTreeExportPartEntry.h" namespace DB @@ -120,6 +121,13 @@ class StorageMergeTree final : public MergeTreeData MergeTreeDeduplicationLog * getDeduplicationLog() { return deduplication_log.get(); } + /// Begin an export on a partition: persist a lock marker and acquire a partition-level merges blocker. + /// Returns a generated commit id (string) that can be used to create a commit file in the destination. + String beginExportPartitionLock(const String & partition_id); + + /// Finish an export on a partition: remove the persisted lock marker and release the blocker. + void endExportPartitionLock(const String & partition_id); + private: /// Mutex and condvar for synchronous mutations wait @@ -152,6 +160,7 @@ class StorageMergeTree final : public MergeTreeData DataParts currently_merging_mutating_parts; std::map current_mutations_by_version; + std::multimap export_part_entries; /// Unfinished mutations that are required for AlterConversions. MutationCounters mutation_counters; @@ -167,6 +176,14 @@ class StorageMergeTree final : public MergeTreeData std::map mutation_prepared_sets_cache; void loadMutations(); + /// Load persisted export partition locks and re-apply partition-level merge blockers. + void loadExportPartition(); + + /// Restart export process for parts that were being exported before restart + void restartExportProcess(); + + /// Reconstruct destination storage from StorageID + StoragePtr reconstructDestinationStorage(const StorageID & storage_id) const; /// Load and initialize deduplication logs. Even if deduplication setting /// equals zero creates object with deduplication window equals zero. @@ -285,6 +302,8 @@ class StorageMergeTree final : public MergeTreeData void assertNotReadonly() const; + void exportPartsImpl(const std::vector & parts, const String & commit_id, const StoragePtr & dest_storage); + friend class MergeTreeSink; friend class MergeTreeData; friend class MergePlainMergeTreeTask; @@ -382,19 +401,19 @@ class StorageMergeTree final : public MergeTreeData void resetMutationFailures() { - std::unique_lock _lock(parts_info_lock); + std::unique_lock parts_lock(parts_info_lock); failed_mutation_parts.clear(); } void removePartFromFailed(const String & part_name) { - std::unique_lock _lock(parts_info_lock); + std::unique_lock parts_lock(parts_info_lock); failed_mutation_parts.erase(part_name); } void addPartMutationFailure (const String& part_name, size_t max_postpone_time_ms_) { - std::unique_lock _lock(parts_info_lock); + std::unique_lock parts_lock(parts_info_lock); auto part_info_it = failed_mutation_parts.find(part_name); if (part_info_it == failed_mutation_parts.end()) { @@ -407,8 +426,7 @@ class StorageMergeTree final : public MergeTreeData bool partCanBeMutated(const String& part_name) { - - std::unique_lock _lock(parts_info_lock); + std::unique_lock parts_lock(parts_info_lock); auto iter = failed_mutation_parts.find(part_name); if (iter == failed_mutation_parts.end()) return true; @@ -419,6 +437,9 @@ class StorageMergeTree final : public MergeTreeData PartMutationBackoffPolicy mutation_backoff_policy; MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const override; + + /// Holds active ActionLocks that block merges per-partition for in-progress exports. + std::unordered_map export_partition_locks; }; } From 45bf82b4899abe2517e0c9293fa5e4138be340e7 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Fri, 22 Aug 2025 07:32:05 -0300 Subject: [PATCH 14/48] good for a demo --- src/Storages/IStorage.h | 3 + src/Storages/MergeTree/MergeTreeData.h | 26 +- .../MergeTree/MergeTreeExportManifest.h | 186 ++++++++++++ .../MergeTree/MergeTreeExportPartEntry.h | 104 ------- .../ObjectStorage/StorageObjectStorage.cpp | 20 +- .../ObjectStorage/StorageObjectStorage.h | 3 + src/Storages/StorageMergeTree.cpp | 268 ++++++++++-------- src/Storages/StorageMergeTree.h | 10 +- 8 files changed, 377 insertions(+), 243 deletions(-) create mode 100644 src/Storages/MergeTree/MergeTreeExportManifest.h delete mode 100644 src/Storages/MergeTree/MergeTreeExportPartEntry.h diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 47cb9eeace1b..bc99b398f2c1 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -214,6 +214,9 @@ class IStorage : public std::enable_shared_from_this, public TypePromo virtuals.set(std::make_unique(std::move(virtuals_))); } + /// Write an export commit file containing the list of exported remote paths + virtual void writeExportCommit(const String &, const Strings &, ContextPtr) {throw Exception(ErrorCodes::NOT_IMPLEMENTED, "writeExportCommit is not implemented for storage type {}", getName());} + /// Return list of virtual columns (like _part, _table, etc). In the vast /// majority of cases virtual columns are static constant part of Storage /// class and don't depend on Storage object. But sometimes we have fake diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 5c524a2f4007..e63baf8ab430 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1239,6 +1239,20 @@ class MergeTreeData : public IStorage, public WithMutableContext /// under lockForShare if rename is possible. String relative_data_path; + /// RAII Wrapper for atomic work with currently moving parts + /// Acquire them in constructor and remove them in destructor + /// Uses data.currently_moving_parts_mutex + struct CurrentlyMovingPartsTagger + { + MergeTreeMovingParts parts_to_move; + MergeTreeData & data; + CurrentlyMovingPartsTagger(MergeTreeMovingParts && moving_parts_, MergeTreeData & data_); + + ~CurrentlyMovingPartsTagger(); + }; + + using CurrentlyMovingPartsTaggerPtr = std::shared_ptr; + private: /// Columns and secondary indices sizes can be calculated lazily. mutable std::mutex columns_and_secondary_indices_sizes_mutex; @@ -1673,19 +1687,7 @@ class MergeTreeData : public IStorage, public WithMutableContext DataPartsVector * out_covered_parts, bool rename_in_transaction); - /// RAII Wrapper for atomic work with currently moving parts - /// Acquire them in constructor and remove them in destructor - /// Uses data.currently_moving_parts_mutex - struct CurrentlyMovingPartsTagger - { - MergeTreeMovingParts parts_to_move; - MergeTreeData & data; - CurrentlyMovingPartsTagger(MergeTreeMovingParts && moving_parts_, MergeTreeData & data_); - - ~CurrentlyMovingPartsTagger(); - }; - using CurrentlyMovingPartsTaggerPtr = std::shared_ptr; /// Moves part to specified space, used in ALTER ... MOVE ... queries std::future movePartsToSpace(const CurrentlyMovingPartsTaggerPtr & moving_tagger, const ReadSettings & read_settings, const WriteSettings & write_settings, bool async); diff --git a/src/Storages/MergeTree/MergeTreeExportManifest.h b/src/Storages/MergeTree/MergeTreeExportManifest.h new file mode 100644 index 000000000000..f45e88d0bba5 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeExportManifest.h @@ -0,0 +1,186 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include "Storages/MergeTree/RangesInDataPart.h" +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +/** + * JSON manifest for exporting a set of parts to object storage. + * Layout on disk (pretty-printed JSON): + * { + * "commit_id": "", + * "destination": ".", + * "parts": [ {"part_name": "name", "remote_path": "path-or-empty"}, ... ] + * } + */ +struct MergeTreeExportManifest +{ + MergeTreeExportManifest() + : destination_storage_id(StorageID::createEmpty()) + {} + + struct Item + { + String part_name; + String remote_path; // empty until uploaded + }; + + String commit_id; + StorageID destination_storage_id; + std::vector items; + + std::filesystem::path file_path; + DiskPtr disk; + + static std::shared_ptr create( + const DiskPtr & disk_, + const String & path_prefix, + const String & commit_id_, + const StorageID & destination_storage_id_, + const std::vector & data_parts) + { + auto manifest = std::make_shared(); + manifest->disk = disk_; + manifest->commit_id = commit_id_; + manifest->destination_storage_id = destination_storage_id_; + manifest->file_path = std::filesystem::path(path_prefix) / ("export_" + commit_id_ + ".json"); + manifest->items.reserve(data_parts.size()); + for (const auto & data_part : data_parts) + manifest->items.push_back({data_part->name, {}}); + manifest->write(); + return manifest; + } + + static std::shared_ptr read(const DiskPtr & disk_, const String & file_path_) + { + auto manifest = std::make_shared(); + manifest->disk = disk_; + manifest->file_path = file_path_; + + auto in = disk_->readFile(file_path_, ReadSettings{}); + + String json_str; + readStringUntilEOF(json_str, *in); + + Poco::JSON::Parser parser; + Poco::Dynamic::Var json = parser.parse(json_str); + const Poco::JSON::Object::Ptr & root = json.extract(); + if (!root) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid export manifest JSON: {}", file_path_); + + if (root->has("commit_id")) + manifest->commit_id = root->getValue("commit_id"); + if (root->has("destination")) + { + const auto destination = root->getValue("destination"); + if (!destination.empty()) + manifest->destination_storage_id = StorageID(QualifiedTableName::parseFromString(destination)); + } + + manifest->items.clear(); + if (root->has("parts")) + { + auto parts = root->get("parts").extract(); + if (parts) + { + for (unsigned int i = 0; i < parts->size(); ++i) + { + const auto part_obj = parts->getObject(i); + if (!part_obj) + continue; + Item item; + if (part_obj->has("part_name")) + item.part_name = part_obj->getValue("part_name"); + if (part_obj->has("remote_path")) + item.remote_path = part_obj->getValue("remote_path"); + if (!item.part_name.empty()) + manifest->items.push_back(std::move(item)); + } + } + } + + return manifest; + } + + void write() const + { + auto out = disk->writeFile(file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite); + + Poco::JSON::Object::Ptr root(new Poco::JSON::Object()); + root->set("commit_id", commit_id); + root->set("destination", destination_storage_id.getQualifiedName().getFullName()); + + Poco::JSON::Array::Ptr parts(new Poco::JSON::Array()); + for (const auto & i : items) + { + Poco::JSON::Object::Ptr obj(new Poco::JSON::Object()); + obj->set("part_name", i.part_name); + obj->set("remote_path", i.remote_path); + parts->add(obj); + } + root->set("parts", parts); + + std::ostringstream oss; + Poco::JSON::Stringifier::stringify(root, oss, 2); + const std::string s = oss.str(); + out->write(s.data(), s.size()); + out->finalize(); + out->sync(); + } + + void updateRemotePath(const String & part_name, const String & remote_path) + { + for (auto & i : items) + { + if (i.part_name == part_name) + { + i.remote_path = remote_path; + write(); + return; + } + } + } + + std::vector pendingParts() const + { + std::vector res; + for (const auto & i : items) + if (i.remote_path.empty()) + res.push_back(i.part_name); + return res; + } + + std::vector exportedPaths() const + { + std::vector res; + for (const auto & i : items) + if (!i.remote_path.empty()) + res.push_back(i.remote_path); + return res; + } + + void remove() const + { + disk->removeFileIfExists(file_path); + } +}; + +} + + diff --git a/src/Storages/MergeTree/MergeTreeExportPartEntry.h b/src/Storages/MergeTree/MergeTreeExportPartEntry.h deleted file mode 100644 index f71e6682cc2f..000000000000 --- a/src/Storages/MergeTree/MergeTreeExportPartEntry.h +++ /dev/null @@ -1,104 +0,0 @@ -#pragma once - -#include -#include -#include -#include "IO/WriteBufferFromFileBase.h" -#include "IO/ReadBuffer.h" -#include "IO/WriteHelpers.h" -#include "IO/ReadHelpers.h" -#include "Common/Exception.h" -#include - -namespace DB -{ - -struct MergeTreeExportPartEntry -{ - String part_name; - String commit_id; - std::filesystem::path file_path; - DiskPtr disk; - StorageID destination_storage_id; // Added StorageID field - - MergeTreeExportPartEntry(const String & part_name_, const String & commit_id_, DiskPtr disk_, - const String & path_prefix, const StorageID & destination_storage_id_) - : part_name(part_name_) - , commit_id(commit_id_) - , file_path(std::filesystem::path(path_prefix) / ("export_part_" + part_name + "_" + commit_id)) - , disk(std::move(disk_)) - , destination_storage_id(destination_storage_id_) - {} - - MergeTreeExportPartEntry(DiskPtr disk_, const String & file_path_) - : file_path(file_path_) - , disk(std::move(disk_)) - , destination_storage_id(StorageID::createEmpty()) // Initialize with empty StorageID - { - constexpr std::string_view prefix = "export_part_"; - - const auto & filename = file_path.filename().string(); - - if (!filename.starts_with(prefix)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Export part entry file path does not start with 'export_part_': {}", file_path_); - - auto rest = filename.substr(prefix.size()); - - auto pos = rest.rfind('_'); - if (pos == std::string::npos) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Export part entry file name does not contain commit id: {}", filename); - - part_name = rest.substr(0, pos); - commit_id = rest.substr(pos + 1); - - // Parse StorageID from the file content - destination_storage_id = parseStorageIDFromFile(); - } - - void write() const - { - try - { - auto out = disk->writeFile(file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite); - - // Serialize the QualifiedTableName (database.table format) - String serialized_qualified_name = destination_storage_id.getQualifiedName().getFullName(); - writeString(serialized_qualified_name, *out); - - out->finalize(); - out->sync(); - } - catch (const Exception &) - { - remove(); - throw; - } - } - - void remove() const - { - disk->removeFileIfExists(file_path); - } - -private: - StorageID parseStorageIDFromFile() const - { - try - { - auto in = disk->readFile(file_path, ReadSettings{}); - - String serialized_qualified_name; - readString(serialized_qualified_name, *in); - - return StorageID(QualifiedTableName::parseFromString(serialized_qualified_name)); - } - catch (const Exception &) - { - // If file doesn't exist or is corrupted, return empty StorageID - // This handles backward compatibility with existing files - return StorageID::createEmpty(); - } - } -}; - -} diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index f1d5a61e09e7..e4f41755e17a 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -655,18 +655,20 @@ void StorageObjectStorage::importMergeTreePartition( CompletedPipelineExecutor exec(export_pipeline); exec.execute(); - Strings exported_paths; - exported_paths.reserve(export_list_entries.size()); - for (const auto & entry : export_list_entries) + // NOTE: Do not write commit file here. The caller manages commit via JSON manifest. +} + +void StorageObjectStorage::writeExportCommit(const String & commit_id, const Strings & exported_paths, ContextPtr local_context) +{ + const String commit_object = configuration->getRawPath().path + "/commit_" + commit_id; + + /// if file already exists, nothing to be done + if (object_storage->exists(StoredObject(commit_object))) { - const auto info = entry->operator->()->getInfo(); - exported_paths.emplace_back(info.destination_path); + LOG_DEBUG(getLogger("StorageObjectStorage"), "Commit file already exists, nothing to be done: {}", commit_object); + return; } - // Generate commit id and write commit file - pcg64_fast rng(randomSeed()); - const String commit_id = toString(rng()); - const String commit_object = configuration->getRawPath().path + "/commit_" + commit_id; auto out = object_storage->writeObject(StoredObject(commit_object), WriteMode::Rewrite, /* attributes= */ {}, DBMS_DEFAULT_BUFFER_SIZE, local_context->getWriteSettings()); for (const auto & p : exported_paths) { diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index acf11cb6dbcd..3d14af9fe934 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -104,6 +104,9 @@ class StorageObjectStorage : public IStorage ContextPtr /*context*/, std::function part_log) override; + /// Write an export commit file containing the list of exported remote paths + void writeExportCommit(const String & commit_id, const Strings & exported_paths, ContextPtr local_context) override; + void truncate( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a36afe31337a..bcd79e7263e8 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -49,7 +49,9 @@ #include "Core/BackgroundSchedulePool.h" #include "Core/Names.h" #include "ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.h" -#include "Storages/MergeTree/MergeTreeExportPartEntry.h" +#include "Storages/MergeTree/MergeMutateSelectedEntry.h" +#include "Storages/MergeTree/MergeTreeExportManifest.h" +#include "Storages/MergeTree/MergeTreePartInfo.h" namespace DB { @@ -491,15 +493,21 @@ void StorageMergeTree::alter( } void StorageMergeTree::exportPartsImpl( - const std::vector & src_parts, - const String & /*commit_id*/, + const CurrentlyMovingPartsTaggerPtr & moving_tagger, + const String & commit_id, const StoragePtr & dest_storage) { - /// todo should stopMergesAndWait be accounted as well? const auto start_time = std::chrono::system_clock::now(); + std::vector src_parts; + for (const auto & part : moving_tagger->parts_to_move) + { + src_parts.push_back(part.part); + } + dest_storage->importMergeTreePartition(*this, src_parts, getContext(), [&](MergeTreePartImportStats stats) { + std::lock_guard lock(export_partition_commit_id_to_manifest_mutex); auto table_id = getStorageID(); if (stats.status.code != 0) @@ -508,23 +516,14 @@ void StorageMergeTree::exportPartsImpl( return; } - auto export_entry_it = export_part_entries.end(); - for (auto it = export_part_entries.begin(); it != export_part_entries.end(); ++it) - { - if (it->second.part_name == stats.part->name) - { - export_entry_it = it; - break; - } - } - - if (export_entry_it == export_part_entries.end()) + auto manifest = export_partition_commit_id_to_manifest[commit_id]; + if (!manifest) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Export entry not found for part {}", stats.part->name); + LOG_ERROR(log, "Export manifest not found for commit_id: {}", commit_id); + return; } - export_entry_it->second.remove(); - export_part_entries.erase(export_entry_it); + manifest->updateRemotePath(stats.part->name, stats.file_path); auto part_log = getContext()->getPartLog(table_id.database_name); if (!part_log) @@ -560,17 +559,6 @@ void StorageMergeTree::exportPartsImpl( part_log->add(std::move(part_log_elem)); }); - - for (const auto & part : src_parts) - { - if (!currently_merging_mutating_parts.contains(part)) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} is not in currently_merging_mutating_parts", part->name); - } - - currently_merging_mutating_parts.erase(part); - dropPart(part->name, false, getContext()); - } } /* @@ -596,40 +584,78 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, String partition_id = getPartitionIDFromQuery(command.partition, getContext()); - auto export_partition_function = [this, query_context, partition_id, dest_storage] () mutable + auto lock1 = lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef()[Setting::lock_acquire_timeout]); + auto merges_blocker = stopMergesAndWait(); + std::lock_guard lock(currently_processing_in_background_mutex); + std::lock_guard lock2(export_partition_commit_id_to_manifest_mutex); + + const auto all_parts = getVisibleDataPartsVectorInPartition(getContext(), partition_id); + + if (all_parts.empty()) { - auto lock1 = lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef()[Setting::lock_acquire_timeout]); - auto merges_blocker = stopMergesAndWait(); - const auto src_parts = getVisibleDataPartsVectorInPartition(getContext(), partition_id); + return; + } - if (src_parts.empty()) - { - return true; - } + std::vector src_parts; + + std::vector move_part_entries; - std::string commit_id; + for (const auto & part : all_parts) + { + bool part_being_exported = false; + /// check all manifests to see if the part is already being exported + /// if so, skip it + for (const auto & [commit_id, manifest] : export_partition_commit_id_to_manifest) { - std::random_device rd; - std::mt19937_64 gen(rd()); - std::uniform_int_distribution dis; - uint64_t random_value = dis(gen); - std::ostringstream oss; - oss << std::hex << random_value; - commit_id = oss.str(); + auto parts_being_exported_in_this_commit = manifest->items; + + + for (const auto & item : parts_being_exported_in_this_commit) + { + if (item.part_name == part->name) + { + part_being_exported = true; + break; + } + } } - for (const auto & part : src_parts) + if (part_being_exported) { - MergeTreeExportPartEntry entry(part->name, commit_id, getStoragePolicy()->getAnyDisk(), relative_data_path, dest_storage->getStorageID()); - entry.write(); - export_part_entries.emplace(entry.commit_id, std::move(entry)); - - // do I need the lock if I already did `stopMergesAndWait`? - // std::lock_guard lock(currently_processing_in_background_mutex); - currently_merging_mutating_parts.insert(part); + continue; } - exportPartsImpl(src_parts, commit_id, dest_storage); + src_parts.emplace_back(part); + move_part_entries.emplace_back(part, nullptr); + } + + if (move_part_entries.empty()) + { + return; + } + + auto moving_tagger = std::make_shared(std::move(move_part_entries), *this); + + std::string commit_id; + { + std::random_device rd; + std::mt19937_64 gen(rd()); + std::uniform_int_distribution dis; + uint64_t random_value = dis(gen); + std::ostringstream oss; + oss << std::hex << random_value; + commit_id = oss.str(); + } + + const auto manifest = MergeTreeExportManifest::create(getStoragePolicy()->getAnyDisk(), relative_data_path, commit_id, dest_storage->getStorageID(), src_parts); + + export_partition_commit_id_to_manifest[commit_id] = manifest; + + auto export_partition_function = [this, query_context, partition_id, dest_storage, commit_id, manifest, moving_tagger] () mutable + { + exportPartsImpl(moving_tagger, commit_id, dest_storage); + + commitExportPartition(manifest, dest_storage, getContext()); return true; }; @@ -641,7 +667,9 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, } else { - export_partition_function(); + /// async always + background_moves_assignee.scheduleMoveTask( + std::make_shared(export_partition_function, moves_assignee_trigger, getStorageID())); } } @@ -1164,96 +1192,106 @@ void StorageMergeTree::loadMutations() void StorageMergeTree::loadExportPartition() { - std::lock_guard lock(currently_processing_in_background_mutex); - for (const auto & disk : getDisks()) { for (auto it = disk->iterateDirectory(relative_data_path); it->isValid(); it->next()) { - if (startsWith(it->name(), "export_part_")) + const auto & name = it->name(); + if (startsWith(name, "export_") && endsWith(name, ".json")) { - MergeTreeExportPartEntry entry(disk, it->path()); - - /// Check if part still exists - auto part = getPartIfExists(entry.part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); - - if (!part) + try { - LOG_WARNING(log, "Part {} not found during export partition load, removing export entry", entry.part_name); - entry.remove(); - continue; + auto manifest = MergeTreeExportManifest::read(disk, fs::path(relative_data_path) / name); + export_partition_commit_id_to_manifest[manifest->commit_id] = manifest; + LOG_DEBUG(log, "Loaded export commit manifest: {} (commit_id: {})", name, manifest->commit_id); } - - // Lock the part to prevent merging - auto currently_mutating_parts_insert_result = currently_merging_mutating_parts.insert(part); - if (!currently_mutating_parts_insert_result.second) + catch (const std::exception & ex) { - LOG_WARNING(log, "Part {} already exists in currently_merging_mutating_parts during export load", entry.part_name); - continue; + LOG_ERROR(log, "Failed to load export commit manifest {}: {}", name, ex.what()); } - - LOG_INFO(log, "Loaded export part entry: {} -> {}", entry.part_name, entry.destination_storage_id.getNameForLogs()); - - // Add to export entries - export_part_entries.emplace(entry.commit_id, std::move(entry)); } } } - // Restart export process for any remaining entries - if (!export_part_entries.empty()) + if (!export_partition_commit_id_to_manifest.empty()) { restartExportProcess(); } } -void StorageMergeTree::restartExportProcess() +void StorageMergeTree::commitExportPartition(const std::shared_ptr & manifest, const StoragePtr & dest_storage, ContextPtr local_context) { - /// no need for locking because currently_processing_in_background_mutex is already locked and won't allow those parts to be merged - - // Group parts by commit_id to restart exports - std::map> parts_by_commit; + std::lock_guard lock(export_partition_commit_id_to_manifest_mutex); + dest_storage->writeExportCommit(manifest->commit_id, manifest->exportedPaths(), local_context); + auto parts_lock = lockParts(); - for (const auto & entry : export_part_entries) + std::vector parts_to_remove; + for (const auto & manifest_item : manifest->items) { - auto part = getPartIfExists(entry.second.part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); - if (part) + /// todo fix this, maybe ther eis a better way to get a data part object. Or maybe there is a better way to drop parts + /// actually, it is not safe to do this because I am not holding the lock on currently_merging_mutating_parts + auto part = getPartIfExistsUnlocked(manifest_item.part_name, {MergeTreeDataPartState::Active}, parts_lock); + if (!part) { - parts_by_commit[entry.second.commit_id].push_back(part); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} not found when committing export", manifest_item.part_name); } + + parts_to_remove.push_back(part); } - // Restart export for each commit group - for (const auto & [commit_id, parts] : parts_by_commit) + removePartsFromWorkingSet(nullptr, parts_to_remove, true, &parts_lock); + manifest->remove(); + export_partition_commit_id_to_manifest.erase(manifest->commit_id); +} + + +void StorageMergeTree::restartExportProcess() +{ + /// I suppose I don't need to lock it here because it is during the startup + for (const auto & [commit_id, manifest] : export_partition_commit_id_to_manifest) { - if (!parts.empty()) + auto destination_storage = reconstructDestinationStorage(manifest->destination_storage_id); + if (!destination_storage) + { + LOG_ERROR(log, "Failed to reconstruct destination storage: {}", manifest->destination_storage_id.getNameForLogs()); + continue; + } + + auto pending_part_names = manifest->pendingParts(); + + if (pending_part_names.empty()) + { + LOG_DEBUG(log, "No pending parts found for commit_id: {}, uploading commit file", commit_id); + commitExportPartition(manifest, destination_storage, getContext()); + } + else { - // Get destination storage from the first part's entry - auto entry_it = export_part_entries.find(commit_id); - if (entry_it != export_part_entries.end()) + std::vector move_part_entries; + + for (const auto & part_name : pending_part_names) { - StoragePtr destination_storage = reconstructDestinationStorage(entry_it->second.destination_storage_id); - if (destination_storage) - { - LOG_INFO(log, "Restarting export for {} parts with commit_id: {} -> {}", - parts.size(), commit_id, entry_it->second.destination_storage_id.getNameForLogs()); - - background_moves_assignee.scheduleMoveTask( - std::make_shared( - [this, parts, commit_id, destination_storage]() - { - exportPartsImpl(parts, commit_id, destination_storage); - return true; - }, - moves_assignee_trigger, - getStorageID() - ) - ); - } - else + auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active}); + + if (!part) { - LOG_ERROR(log, "Failed to reconstruct destination storage for commit_id: {}", commit_id); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} not found", part_name); } + + move_part_entries.emplace_back(part, nullptr); + } + + if (!move_part_entries.empty()) + { + auto moving_tagger = std::make_shared(std::move(move_part_entries), *this); + background_moves_assignee.scheduleMoveTask( + std::make_shared( + [this, moving_tagger, commit_id, destination_storage, manifest](){ + exportPartsImpl(moving_tagger, commit_id, destination_storage); + commitExportPartition(manifest, destination_storage, getContext()); + return true; + }, + moves_assignee_trigger, + getStorageID())); } } } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 95d36738e17e..58b87bf81eaa 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -17,7 +17,7 @@ #include #include -#include "Storages/MergeTree/MergeTreeExportPartEntry.h" +#include "Storages/MergeTree/MergeTreeExportManifest.h" namespace DB @@ -160,7 +160,9 @@ class StorageMergeTree final : public MergeTreeData DataParts currently_merging_mutating_parts; std::map current_mutations_by_version; - std::multimap export_part_entries; + + std::mutex export_partition_commit_id_to_manifest_mutex; + std::map> export_partition_commit_id_to_manifest; /// Unfinished mutations that are required for AlterConversions. MutationCounters mutation_counters; @@ -182,6 +184,8 @@ class StorageMergeTree final : public MergeTreeData /// Restart export process for parts that were being exported before restart void restartExportProcess(); + void commitExportPartition(const std::shared_ptr & manifest, const StoragePtr & dest_storage, ContextPtr context); + /// Reconstruct destination storage from StorageID StoragePtr reconstructDestinationStorage(const StorageID & storage_id) const; @@ -302,7 +306,7 @@ class StorageMergeTree final : public MergeTreeData void assertNotReadonly() const; - void exportPartsImpl(const std::vector & parts, const String & commit_id, const StoragePtr & dest_storage); + void exportPartsImpl(const CurrentlyMovingPartsTaggerPtr & moving_tagger, const String & commit_id, const StoragePtr & dest_storage); friend class MergeTreeSink; friend class MergeTreeData; From 41020a11194a904a059ecd588473cf52d84eeeaf Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Sat, 23 Aug 2025 16:52:01 -0300 Subject: [PATCH 15/48] do not drop parts, lock partition for further exports --- .../MergeTree/MergeMutateSelectedEntry.h | 10 ++ .../MergeTree/MergeTreeExportManifest.h | 30 ++-- src/Storages/StorageMergeTree.cpp | 141 ++++++++++-------- src/Storages/StorageMergeTree.h | 18 +-- 4 files changed, 119 insertions(+), 80 deletions(-) diff --git a/src/Storages/MergeTree/MergeMutateSelectedEntry.h b/src/Storages/MergeTree/MergeMutateSelectedEntry.h index c420cbca12b5..bb44d350d3ba 100644 --- a/src/Storages/MergeTree/MergeMutateSelectedEntry.h +++ b/src/Storages/MergeTree/MergeMutateSelectedEntry.h @@ -51,4 +51,14 @@ struct MergeMutateSelectedEntry using MergeMutateSelectedEntryPtr = std::shared_ptr; +struct CurrentlyExportingPartsTagger +{ + std::vector parts_to_export; + StorageMergeTree & storage; + CurrentlyExportingPartsTagger(std::vector && parts_to_export_, StorageMergeTree & storage_); + ~CurrentlyExportingPartsTagger(); +}; + +using CurrentlyExportingPartsTaggerPtr = std::shared_ptr; + } diff --git a/src/Storages/MergeTree/MergeTreeExportManifest.h b/src/Storages/MergeTree/MergeTreeExportManifest.h index f45e88d0bba5..d3f992545148 100644 --- a/src/Storages/MergeTree/MergeTreeExportManifest.h +++ b/src/Storages/MergeTree/MergeTreeExportManifest.h @@ -25,6 +25,7 @@ namespace DB * Layout on disk (pretty-printed JSON): * { * "commit_id": "", + * "partition_id": "", * "destination": ".
", * "parts": [ {"part_name": "name", "remote_path": "path-or-empty"}, ... ] * } @@ -42,8 +43,10 @@ struct MergeTreeExportManifest }; String commit_id; + String partition_id; StorageID destination_storage_id; std::vector items; + bool completed = false; std::filesystem::path file_path; DiskPtr disk; @@ -52,14 +55,16 @@ struct MergeTreeExportManifest const DiskPtr & disk_, const String & path_prefix, const String & commit_id_, + const String & partition_id_, const StorageID & destination_storage_id_, const std::vector & data_parts) { auto manifest = std::make_shared(); manifest->disk = disk_; manifest->commit_id = commit_id_; + manifest->partition_id = partition_id_; manifest->destination_storage_id = destination_storage_id_; - manifest->file_path = std::filesystem::path(path_prefix) / ("export_" + commit_id_ + ".json"); + manifest->file_path = std::filesystem::path(path_prefix) / ("export_partition_" + partition_id_ + "_commit_" + commit_id_ + ".json"); manifest->items.reserve(data_parts.size()); for (const auto & data_part : data_parts) manifest->items.push_back({data_part->name, {}}); @@ -86,12 +91,16 @@ struct MergeTreeExportManifest if (root->has("commit_id")) manifest->commit_id = root->getValue("commit_id"); + if (root->has("partition_id")) + manifest->partition_id = root->getValue("partition_id"); if (root->has("destination")) { const auto destination = root->getValue("destination"); if (!destination.empty()) manifest->destination_storage_id = StorageID(QualifiedTableName::parseFromString(destination)); } + if (root->has("completed")) + manifest->completed = root->getValue("completed"); manifest->items.clear(); if (root->has("parts")) @@ -124,7 +133,9 @@ struct MergeTreeExportManifest Poco::JSON::Object::Ptr root(new Poco::JSON::Object()); root->set("commit_id", commit_id); + root->set("partition_id", partition_id); root->set("destination", destination_storage_id.getQualifiedName().getFullName()); + root->set("completed", completed); Poco::JSON::Array::Ptr parts(new Poco::JSON::Array()); for (const auto & i : items) @@ -144,17 +155,23 @@ struct MergeTreeExportManifest out->sync(); } - void updateRemotePath(const String & part_name, const String & remote_path) + void updateRemotePathAndWrite(const String & part_name, const String & remote_path) { for (auto & i : items) { if (i.part_name == part_name) { i.remote_path = remote_path; - write(); - return; + break; } } + write(); + } + + void updateCompleted(bool completed_) + { + completed = completed_; + write(); } std::vector pendingParts() const @@ -174,11 +191,6 @@ struct MergeTreeExportManifest res.push_back(i.remote_path); return res; } - - void remove() const - { - disk->removeFileIfExists(file_path); - } }; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index bcd79e7263e8..6f2c9da820a1 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -493,16 +493,22 @@ void StorageMergeTree::alter( } void StorageMergeTree::exportPartsImpl( - const CurrentlyMovingPartsTaggerPtr & moving_tagger, + const CurrentlyExportingPartsTaggerPtr & exports_tagger, const String & commit_id, const StoragePtr & dest_storage) { + if (exports_tagger->parts_to_export.empty()) + { + LOG_INFO(log, "No parts to export for commit {}, skipping", commit_id); + return; + } + const auto start_time = std::chrono::system_clock::now(); std::vector src_parts; - for (const auto & part : moving_tagger->parts_to_move) + for (const auto & part : exports_tagger->parts_to_export) { - src_parts.push_back(part.part); + src_parts.push_back(part); } dest_storage->importMergeTreePartition(*this, src_parts, getContext(), [&](MergeTreePartImportStats stats) @@ -523,7 +529,7 @@ void StorageMergeTree::exportPartsImpl( return; } - manifest->updateRemotePath(stats.part->name, stats.file_path); + manifest->updateRemotePathAndWrite(stats.part->name, stats.file_path); auto part_log = getContext()->getPartLog(table_id.database_name); if (!part_log) @@ -585,8 +591,14 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, String partition_id = getPartitionIDFromQuery(command.partition, getContext()); auto lock1 = lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef()[Setting::lock_acquire_timeout]); - auto merges_blocker = stopMergesAndWait(); + auto merges_blocker = stopMergesAndWaitForPartition(partition_id); std::lock_guard lock(currently_processing_in_background_mutex); + + if (!already_exported_partition_ids.emplace(partition_id).second) + { + throw Exception(ErrorCodes::PART_IS_LOCKED, "Partition {} has already been exported", partition_id); + } + std::lock_guard lock2(export_partition_commit_id_to_manifest_mutex); const auto all_parts = getVisibleDataPartsVectorInPartition(getContext(), partition_id); @@ -598,7 +610,7 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, std::vector src_parts; - std::vector move_part_entries; + std::vector parts_to_export; for (const auto & part : all_parts) { @@ -626,15 +638,15 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, } src_parts.emplace_back(part); - move_part_entries.emplace_back(part, nullptr); + parts_to_export.emplace_back(part); } - if (move_part_entries.empty()) + if (parts_to_export.empty()) { return; } - auto moving_tagger = std::make_shared(std::move(move_part_entries), *this); + auto exports_tagger = std::make_shared(std::move(parts_to_export), *this); std::string commit_id; { @@ -647,13 +659,13 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, commit_id = oss.str(); } - const auto manifest = MergeTreeExportManifest::create(getStoragePolicy()->getAnyDisk(), relative_data_path, commit_id, dest_storage->getStorageID(), src_parts); + const auto manifest = MergeTreeExportManifest::create(getStoragePolicy()->getAnyDisk(), relative_data_path, commit_id, partition_id, dest_storage->getStorageID(), src_parts); export_partition_commit_id_to_manifest[commit_id] = manifest; - auto export_partition_function = [this, query_context, partition_id, dest_storage, commit_id, manifest, moving_tagger] () mutable + auto export_partition_function = [this, query_context, partition_id, dest_storage, commit_id, manifest, exports_tagger] () mutable { - exportPartsImpl(moving_tagger, commit_id, dest_storage); + exportPartsImpl(exports_tagger, commit_id, dest_storage); commitExportPartition(manifest, dest_storage, getContext()); @@ -747,6 +759,30 @@ CurrentlyMergingPartsTagger::~CurrentlyMergingPartsTagger() storage.currently_processing_in_background_condition.notify_all(); } +CurrentlyExportingPartsTagger::CurrentlyExportingPartsTagger(std::vector && parts_to_export_, StorageMergeTree & storage_) + : parts_to_export(std::move(parts_to_export_)), storage(storage_) +{ + /// assume it is already locked + for (const auto & part : parts_to_export) + if (!storage.currently_merging_mutating_parts.emplace(part).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot export part '{}'. It's already exporting.", part->name); +} + +CurrentlyExportingPartsTagger::~CurrentlyExportingPartsTagger() +{ + std::lock_guard lock(storage.currently_processing_in_background_mutex); + + for (const auto & part : parts_to_export) + { + if (!storage.currently_merging_mutating_parts.contains(part)) + std::terminate(); + storage.currently_merging_mutating_parts.erase(part); + } + + storage.currently_processing_in_background_condition.notify_all(); +} + + Int64 StorageMergeTree::startMutation(const MutationCommands & commands, ContextPtr query_context) { /// Choose any disk, because when we load mutations we search them at each disk @@ -1203,6 +1239,7 @@ void StorageMergeTree::loadExportPartition() { auto manifest = MergeTreeExportManifest::read(disk, fs::path(relative_data_path) / name); export_partition_commit_id_to_manifest[manifest->commit_id] = manifest; + already_exported_partition_ids.insert(manifest->partition_id); LOG_DEBUG(log, "Loaded export commit manifest: {} (commit_id: {})", name, manifest->commit_id); } catch (const std::exception & ex) @@ -1223,24 +1260,8 @@ void StorageMergeTree::commitExportPartition(const std::shared_ptrwriteExportCommit(manifest->commit_id, manifest->exportedPaths(), local_context); - auto parts_lock = lockParts(); - - std::vector parts_to_remove; - for (const auto & manifest_item : manifest->items) - { - /// todo fix this, maybe ther eis a better way to get a data part object. Or maybe there is a better way to drop parts - /// actually, it is not safe to do this because I am not holding the lock on currently_merging_mutating_parts - auto part = getPartIfExistsUnlocked(manifest_item.part_name, {MergeTreeDataPartState::Active}, parts_lock); - if (!part) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} not found when committing export", manifest_item.part_name); - } - - parts_to_remove.push_back(part); - } - - removePartsFromWorkingSet(nullptr, parts_to_remove, true, &parts_lock); - manifest->remove(); + manifest->completed = true; + manifest->write(); export_partition_commit_id_to_manifest.erase(manifest->commit_id); } @@ -1250,6 +1271,17 @@ void StorageMergeTree::restartExportProcess() /// I suppose I don't need to lock it here because it is during the startup for (const auto & [commit_id, manifest] : export_partition_commit_id_to_manifest) { + if (manifest->completed) + { + LOG_INFO( + log, + "Export commit {} of partition {} to destination storage {} already completed, skipping", + commit_id, + manifest->partition_id, + manifest->destination_storage_id.getNameForLogs()); + continue; + } + auto destination_storage = reconstructDestinationStorage(manifest->destination_storage_id); if (!destination_storage) { @@ -1259,41 +1291,30 @@ void StorageMergeTree::restartExportProcess() auto pending_part_names = manifest->pendingParts(); - if (pending_part_names.empty()) - { - LOG_DEBUG(log, "No pending parts found for commit_id: {}, uploading commit file", commit_id); - commitExportPartition(manifest, destination_storage, getContext()); - } - else - { - std::vector move_part_entries; - - for (const auto & part_name : pending_part_names) - { - auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active}); + std::vector parts_to_export; - if (!part) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} not found", part_name); - } + for (const auto & part_name : pending_part_names) + { + auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active}); - move_part_entries.emplace_back(part, nullptr); - } - - if (!move_part_entries.empty()) + if (!part) { - auto moving_tagger = std::make_shared(std::move(move_part_entries), *this); - background_moves_assignee.scheduleMoveTask( - std::make_shared( - [this, moving_tagger, commit_id, destination_storage, manifest](){ - exportPartsImpl(moving_tagger, commit_id, destination_storage); - commitExportPartition(manifest, destination_storage, getContext()); - return true; - }, - moves_assignee_trigger, - getStorageID())); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} not found", part_name); } + + parts_to_export.emplace_back(part); } + + auto exports_tagger = std::make_shared(std::move(parts_to_export), *this); + background_moves_assignee.scheduleMoveTask( + std::make_shared( + [this, exports_tagger, commit_id, destination_storage, manifest](){ + exportPartsImpl(exports_tagger, commit_id, destination_storage); + commitExportPartition(manifest, destination_storage, getContext()); + return true; + }, + moves_assignee_trigger, + getStorageID())); } } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 58b87bf81eaa..3ca922ca8bce 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -121,13 +121,6 @@ class StorageMergeTree final : public MergeTreeData MergeTreeDeduplicationLog * getDeduplicationLog() { return deduplication_log.get(); } - /// Begin an export on a partition: persist a lock marker and acquire a partition-level merges blocker. - /// Returns a generated commit id (string) that can be used to create a commit file in the destination. - String beginExportPartitionLock(const String & partition_id); - - /// Finish an export on a partition: remove the persisted lock marker and release the blocker. - void endExportPartitionLock(const String & partition_id); - private: /// Mutex and condvar for synchronous mutations wait @@ -159,6 +152,11 @@ class StorageMergeTree final : public MergeTreeData /// This set have to be used with `currently_processing_in_background_mutex`. DataParts currently_merging_mutating_parts; + /// Should be used with `currently_processing_in_background_mutex`. + /// Holds partition ids that have already been exported. + /// A partition can be exported only once. + std::unordered_set already_exported_partition_ids; + std::map current_mutations_by_version; std::mutex export_partition_commit_id_to_manifest_mutex; @@ -226,6 +224,7 @@ class StorageMergeTree final : public MergeTreeData void setMutationCSN(const String & mutation_id, CSN csn) override; friend struct CurrentlyMergingPartsTagger; + friend struct CurrentlyExportingPartsTagger; friend class MergeTreeMergePredicate; std::expected selectPartsToMerge( @@ -306,7 +305,7 @@ class StorageMergeTree final : public MergeTreeData void assertNotReadonly() const; - void exportPartsImpl(const CurrentlyMovingPartsTaggerPtr & moving_tagger, const String & commit_id, const StoragePtr & dest_storage); + void exportPartsImpl(const CurrentlyExportingPartsTaggerPtr & exports_tagger, const String & commit_id, const StoragePtr & dest_storage); friend class MergeTreeSink; friend class MergeTreeData; @@ -441,9 +440,6 @@ class StorageMergeTree final : public MergeTreeData PartMutationBackoffPolicy mutation_backoff_policy; MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const override; - - /// Holds active ActionLocks that block merges per-partition for in-progress exports. - std::unordered_map export_partition_locks; }; } From 61928e45a1a61119ad5cb95bf73340adba6fae38 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 25 Aug 2025 11:19:21 -0300 Subject: [PATCH 16/48] add partition_id to commit filename, remove unused code and refactor silly tests --- src/Storages/IStorage.h | 9 ++- .../ObjectStorage/StorageObjectStorage.cpp | 8 +-- .../ObjectStorage/StorageObjectStorage.h | 2 +- src/Storages/PartitionedSink.cpp | 53 +---------------- src/Storages/PartitionedSink.h | 11 ---- src/Storages/StorageMergeTree.cpp | 3 +- ...export_mt_part_to_object_storage.reference | 59 ++++++------------- .../03572_export_mt_part_to_object_storage.sh | 43 +++----------- 8 files changed, 44 insertions(+), 144 deletions(-) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index bc99b398f2c1..3cbe52f11ffe 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -215,7 +215,14 @@ class IStorage : public std::enable_shared_from_this, public TypePromo } /// Write an export commit file containing the list of exported remote paths - virtual void writeExportCommit(const String &, const Strings &, ContextPtr) {throw Exception(ErrorCodes::NOT_IMPLEMENTED, "writeExportCommit is not implemented for storage type {}", getName());} + virtual void writeExportCommit( + const String & /* commit_id */, + const String & /* partition_id */, + const Strings & /* exported_paths */, + ContextPtr /* local_context */) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "writeExportCommit is not implemented for storage type {}", getName()); + } /// Return list of virtual columns (like _part, _table, etc). In the vast /// majority of cases virtual columns are static constant part of Storage diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index e4f41755e17a..c7ca8f014550 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -565,8 +565,8 @@ void StorageObjectStorage::importMergeTreePartition( QueryPlan plan; - // todo arthur - MergeTreeSequentialSourceType read_type = MergeTreeSequentialSourceType::Merge; + /// using the mutations type for now + MergeTreeSequentialSourceType read_type = MergeTreeSequentialSourceType::Mutation; bool apply_deleted_mask = true; bool read_with_direct_io = false; @@ -658,9 +658,9 @@ void StorageObjectStorage::importMergeTreePartition( // NOTE: Do not write commit file here. The caller manages commit via JSON manifest. } -void StorageObjectStorage::writeExportCommit(const String & commit_id, const Strings & exported_paths, ContextPtr local_context) +void StorageObjectStorage::writeExportCommit(const String & commit_id, const String & partition_id, const Strings & exported_paths, ContextPtr local_context) { - const String commit_object = configuration->getRawPath().path + "/commit_" + commit_id; + const String commit_object = configuration->getRawPath().path + "/commit_" + partition_id + "_" + commit_id; /// if file already exists, nothing to be done if (object_storage->exists(StoredObject(commit_object))) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 3d14af9fe934..6c712e0c59d2 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -105,7 +105,7 @@ class StorageObjectStorage : public IStorage std::function part_log) override; /// Write an export commit file containing the list of exported remote paths - void writeExportCommit(const String & commit_id, const Strings & exported_paths, ContextPtr local_context) override; + void writeExportCommit(const String & commit_id, const String & partition_id, const Strings & exported_paths, ContextPtr local_context) override; void truncate( const ASTPtr & query, diff --git a/src/Storages/PartitionedSink.cpp b/src/Storages/PartitionedSink.cpp index 940f11333a58..1a852975e424 100644 --- a/src/Storages/PartitionedSink.cpp +++ b/src/Storages/PartitionedSink.cpp @@ -9,7 +9,6 @@ #include #include #include -#include "boost/iostreams/concepts.hpp" namespace DB @@ -46,42 +45,13 @@ SinkPtr PartitionedSink::getSinkForPartitionKey(StringRef partition_key) return it->second; } -PartitionedSink::ChunkSplitStatistics PartitionedSink::getPartitioningStats() const -{ - return partitioning_stats; -} - -void PartitionedSink::consumeAssumeSamePartition(Chunk & source_chunk) -{ - if (!sink_to_storage) - { - const ColumnPtr partition_by_result_column = partition_strategy->computePartitionKey(source_chunk); - auto partition_key = partition_by_result_column->getDataAt(0); - sink_to_storage = sink_creator->createSinkForPartition(partition_key.toString()); - } - - auto format_chunk = partition_strategy->getFormatChunk(source_chunk); - sink_to_storage->consume(format_chunk); -} - void PartitionedSink::consume(Chunk & source_chunk) { - if (assume_same_partition) - { - consumeAssumeSamePartition(source_chunk); - return; - } - - auto start_calc = std::chrono::system_clock::now(); const ColumnPtr partition_by_result_column = partition_strategy->computePartitionKey(source_chunk); - auto end_calc = std::chrono::system_clock::now(); - - partitioning_stats.time_spent_on_partition_calculation += -std::chrono::duration_cast(end_calc - start_calc).count(); /// Not all columns are serialized using the format writer (e.g, hive partitioning stores partition columns in the file path) auto format_chunk = partition_strategy->getFormatChunk(source_chunk); - const auto columns_to_consume = format_chunk.getColumns(); + const auto & columns_to_consume = format_chunk.getColumns(); if (columns_to_consume.empty()) { @@ -90,8 +60,6 @@ std::chrono::duration_cast(end_calc - start_calc).cou "Consider setting `partition_columns_in_data_file=1`"); } - auto start_split = std::chrono::system_clock::now(); - size_t chunk_rows = source_chunk.getNumRows(); chunk_row_index_to_partition_index.resize(chunk_rows); @@ -132,10 +100,6 @@ std::chrono::duration_cast(end_calc - start_calc).cou } } - auto end_split = std::chrono::system_clock::now(); - - partitioning_stats.time_spent_on_chunk_split += std::chrono::duration_cast(end_split - start_split).count(); - for (const auto & [partition_key, partition_index] : partition_id_to_chunk_index) { auto sink = getSinkForPartitionKey(partition_key); @@ -145,11 +109,6 @@ std::chrono::duration_cast(end_calc - start_calc).cou void PartitionedSink::onException(std::exception_ptr exception) { - if (assume_same_partition && sink_to_storage) - { - sink_to_storage->onException(exception); - return; - } for (auto & [_, sink] : partition_id_to_sink) { sink->onException(exception); @@ -158,11 +117,6 @@ void PartitionedSink::onException(std::exception_ptr exception) void PartitionedSink::onFinish() { - if (assume_same_partition && sink_to_storage) - { - sink_to_storage->onFinish(); - return; - } for (auto & [_, sink] : partition_id_to_sink) { sink->onFinish(); @@ -195,11 +149,6 @@ PartitionedSink::~PartitionedSink() { if (isCancelled()) { - if (assume_same_partition && sink_to_storage) - { - sink_to_storage->cancel(); - return; - } for (auto &item: partition_id_to_sink) item.second->cancel(); } diff --git a/src/Storages/PartitionedSink.h b/src/Storages/PartitionedSink.h index 231276c9bfa1..9ee120ad101d 100644 --- a/src/Storages/PartitionedSink.h +++ b/src/Storages/PartitionedSink.h @@ -48,16 +48,10 @@ class PartitionedSink : public SinkToStorage void onFinish() override; - void assumeSamePartition() { - assume_same_partition = true; - } - static void validatePartitionKey(const String & str, bool allow_slash); static String replaceWildcards(const String & haystack, const String & partition_id); - ChunkSplitStatistics getPartitioningStats() const; - private: std::shared_ptr partition_strategy; std::shared_ptr sink_creator; @@ -70,11 +64,6 @@ class PartitionedSink : public SinkToStorage Arena partition_keys_arena; SinkPtr getSinkForPartitionKey(StringRef partition_key); - ChunkSplitStatistics partitioning_stats; - bool assume_same_partition = false; - std::shared_ptr sink_to_storage; - - void consumeAssumeSamePartition(Chunk & chunk); }; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 6f2c9da820a1..fdaa190cee5b 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -532,6 +532,7 @@ void StorageMergeTree::exportPartsImpl( manifest->updateRemotePathAndWrite(stats.part->name, stats.file_path); auto part_log = getContext()->getPartLog(table_id.database_name); + if (!part_log) return; @@ -1259,7 +1260,7 @@ void StorageMergeTree::loadExportPartition() void StorageMergeTree::commitExportPartition(const std::shared_ptr & manifest, const StoragePtr & dest_storage, ContextPtr local_context) { std::lock_guard lock(export_partition_commit_id_to_manifest_mutex); - dest_storage->writeExportCommit(manifest->commit_id, manifest->exportedPaths(), local_context); + dest_storage->writeExportCommit(manifest->commit_id, manifest->partition_id, manifest->exportedPaths(), local_context); manifest->completed = true; manifest->write(); export_partition_commit_id_to_manifest.erase(manifest->commit_id); diff --git a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.reference b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.reference index 09f6b30f7552..d7e137f36cba 100644 --- a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.reference +++ b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.reference @@ -1,46 +1,25 @@ ----- Insert some more data in the already exported partition -test/s3_table_NAME/year=2020/2020_1_1_0.parquet -test/s3_table_NAME/year=2020/2020_1_1_0.parquet -test/s3_table_NAME/year=2020/2020_1_1_0.parquet ----- Export the parts, only the diff should be exported ----- New data part should appear (2020_2_2_0.parquet) with id 5 and 6 -test/s3_table_NAME/year=2020/2020_1_1_0.parquet -test/s3_table_NAME/year=2020/2020_1_1_0.parquet -test/s3_table_NAME/year=2020/2020_1_1_0.parquet -test/s3_table_NAME/year=2020/2020_3_3_0.parquet -test/s3_table_NAME/year=2020/2020_3_3_0.parquet ----- Merge all parts ----- Nothing should be exported even though the parts in the merge tree table have been merged ----- Check the parts in the remote storage have not been touched -test/s3_table_NAME/year=2020/2020_1_3_1.parquet -test/s3_table_NAME/year=2020/2020_1_3_1.parquet -test/s3_table_NAME/year=2020/2020_1_3_1.parquet -test/s3_table_NAME/year=2020/2020_1_3_1.parquet -test/s3_table_NAME/year=2020/2020_1_3_1.parquet ----- Yet another part ----- Merge the new part ----- The new part that cover everything should be exported ----- Assert there is only one data part -test/s3_table_NAME/year=2020/2020_1_4_2.parquet -test/s3_table_NAME/year=2020/2020_1_4_2.parquet -test/s3_table_NAME/year=2020/2020_1_4_2.parquet -test/s3_table_NAME/year=2020/2020_1_4_2.parquet -test/s3_table_NAME/year=2020/2020_1_4_2.parquet -test/s3_table_NAME/year=2020/2020_1_4_2.parquet ----- Finally, export the other partition (2021) ----- Assert both partitions are there -test/s3_table_NAME/year=2020/2020_1_4_2.parquet -test/s3_table_NAME/year=2020/2020_1_4_2.parquet -test/s3_table_NAME/year=2020/2020_1_4_2.parquet -test/s3_table_NAME/year=2021/2021_2_2_2.parquet -test/s3_table_NAME/year=2020/2020_1_4_2.parquet -test/s3_table_NAME/year=2020/2020_1_4_2.parquet -test/s3_table_NAME/year=2020/2020_1_4_2.parquet ----- Selecting from merge tree to have a sample of what the data in remote storage should look like +---- Querying merge tree for comparison. It should include both partitions (2020 and 2021) 1 2020 2 2020 3 2020 4 2021 5 2020 6 2020 -7 2020 +---- Make sure only the partition 2020 has been exported +test/s3_table_NAME/year=2020/SNOWFLAKE_ID.parquet 1 2020 +test/s3_table_NAME/year=2020/SNOWFLAKE_ID.parquet 2 2020 +test/s3_table_NAME/year=2020/SNOWFLAKE_ID.parquet 3 2020 +test/s3_table_NAME/year=2020/SNOWFLAKE_ID.parquet 5 2020 +test/s3_table_NAME/year=2020/SNOWFLAKE_ID.parquet 6 2020 +---- It should not be allowed to export the same partition twice +---- Check for commit file for partition 2020 +s3_table_NAME/year=2020//SNOWFLAKE_ID.parquet +s3_table_NAME/year=2020//SNOWFLAKE_ID.parquet +---- Finally, export the other partition (2021) +---- Assert both partitions are there +test/s3_table_NAME/year=2020/SNOWFLAKE_ID.parquet 1 2020 +test/s3_table_NAME/year=2020/SNOWFLAKE_ID.parquet 2 2020 +test/s3_table_NAME/year=2020/SNOWFLAKE_ID.parquet 3 2020 +test/s3_table_NAME/year=2021/SNOWFLAKE_ID.parquet 4 2021 +test/s3_table_NAME/year=2020/SNOWFLAKE_ID.parquet 5 2020 +test/s3_table_NAME/year=2020/SNOWFLAKE_ID.parquet 6 2020 diff --git a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh index 2bf6f6ac3704..f55893268074 100755 --- a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh +++ b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh @@ -19,52 +19,27 @@ query "CREATE TABLE $s3_table (id UInt64, year UInt16) ENGINE = S3(s3_conn, file query "SYSTEM STOP MERGES" query "INSERT INTO $mt_table VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)" -query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_partition = 1, export_merge_tree_partition_background_execution = 0" - -echo "---- Insert some more data in the already exported partition" -query "SELECT DISTINCT ON (id) replaceRegexpAll(_path, '$s3_table', 's3_table_NAME') FROM $s3_table ORDER BY id" - -echo "---- Export the parts, only the diff should be exported" query "INSERT INTO $mt_table VALUES (5, 2020), (6, 2020)" -query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_partition = 1, export_merge_tree_partition_background_execution = 0" - -echo "---- New data part should appear (2020_2_2_0.parquet) with id 5 and 6" -query "SELECT DISTINCT ON (id) replaceRegexpAll(_path, '$s3_table', 's3_table_NAME') FROM $s3_table ORDER BY id" - -echo "---- Merge all parts" -query "SYSTEM START MERGES" -query "OPTIMIZE TABLE $mt_table FINAL" -query "SYSTEM STOP MERGES" -echo "---- Nothing should be exported even though the parts in the merge tree table have been merged" query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_partition = 1, export_merge_tree_partition_background_execution = 0" -echo "---- Check the parts in the remote storage have not been touched" -query "SELECT DISTINCT ON (id) replaceRegexpAll(_path, '$s3_table', 's3_table_NAME') FROM $s3_table ORDER BY id" +echo "---- Querying merge tree for comparison. It should include both partitions (2020 and 2021)" +query "SELECT * FROM $mt_table ORDER BY id" -echo "---- Yet another part" -query "INSERT INTO $mt_table VALUES (7, 2020)" +echo "---- Make sure only the partition 2020 has been exported" +query "SELECT DISTINCT ON (id) replaceRegexpAll(replaceRegexpAll(_path, '$s3_table', 's3_table_NAME'), '[^/]+\\.parquet', 'SNOWFLAKE_ID.parquet'), * FROM $s3_table ORDER BY id" -echo "---- Merge the new part" -query "SYSTEM START MERGES" -query "OPTIMIZE TABLE $mt_table FINAL" -query "SYSTEM STOP MERGES" +echo "---- It should not be allowed to export the same partition twice" +query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_partition = 1 -- {serverError PART_IS_LOCKED}" -echo "---- The new part that cover everything should be exported" -query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_partition = 1, export_merge_tree_partition_background_execution = 0" - -echo "---- Assert there is only one data part" -query "SELECT DISTINCT ON (id) replaceRegexpAll(_path, '$s3_table', 's3_table_NAME') FROM $s3_table ORDER BY id" +echo "---- Check for commit file for partition 2020" +$CLICKHOUSE_CLIENT --query "SELECT replaceRegexpAll(replaceRegexpAll(remote_file_path, '$s3_table', 's3_table_NAME'), '[^/]+\\.parquet', 'SNOWFLAKE_ID.parquet') FROM s3(s3_conn, filename='$s3_table/commit_2020_*', format='LineAsString', structure='remote_file_path String')" echo "---- Finally, export the other partition (2021)" query "ALTER TABLE $mt_table EXPORT PARTITION ID '2021' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_partition = 1, export_merge_tree_partition_background_execution = 0" echo "---- Assert both partitions are there" -query "SELECT DISTINCT ON (id) replaceRegexpAll(_path, '$s3_table', 's3_table_NAME') FROM $s3_table ORDER BY id" - -echo "---- Selecting from merge tree to have a sample of what the data in remote storage should look like" -query "SELECT DISTINCT ON (id) id, year FROM $mt_table ORDER BY id" +query "SELECT DISTINCT ON (id) replaceRegexpAll(replaceRegexpAll(_path, '$s3_table', 's3_table_NAME'), '[^/]+\\.parquet', 'SNOWFLAKE_ID.parquet'), * FROM $s3_table ORDER BY id" query "SYSTEM START MERGES" query "DROP TABLE IF EXISTS $mt_table, $s3_table" - From f8ad06fd8908d37e5694760bed495c7df5048d5a Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 25 Aug 2025 11:49:55 -0300 Subject: [PATCH 17/48] simplify the code a bit --- .../MergeTree/MergeTreeExportManifest.h | 57 +++----- src/Storages/StorageMergeTree.cpp | 122 ++++++------------ 2 files changed, 58 insertions(+), 121 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeExportManifest.h b/src/Storages/MergeTree/MergeTreeExportManifest.h index d3f992545148..0f488f022e51 100644 --- a/src/Storages/MergeTree/MergeTreeExportManifest.h +++ b/src/Storages/MergeTree/MergeTreeExportManifest.h @@ -67,7 +67,7 @@ struct MergeTreeExportManifest manifest->file_path = std::filesystem::path(path_prefix) / ("export_partition_" + partition_id_ + "_commit_" + commit_id_ + ".json"); manifest->items.reserve(data_parts.size()); for (const auto & data_part : data_parts) - manifest->items.push_back({data_part->name, {}}); + manifest->items.push_back({data_part->name, ""}); manifest->write(); return manifest; } @@ -89,39 +89,21 @@ struct MergeTreeExportManifest if (!root) throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid export manifest JSON: {}", file_path_); - if (root->has("commit_id")) - manifest->commit_id = root->getValue("commit_id"); - if (root->has("partition_id")) - manifest->partition_id = root->getValue("partition_id"); - if (root->has("destination")) - { - const auto destination = root->getValue("destination"); - if (!destination.empty()) - manifest->destination_storage_id = StorageID(QualifiedTableName::parseFromString(destination)); - } - if (root->has("completed")) - manifest->completed = root->getValue("completed"); + manifest->commit_id = root->getValue("commit_id"); + manifest->partition_id = root->getValue("partition_id"); + const auto destination = root->getValue("destination"); + manifest->destination_storage_id = StorageID(QualifiedTableName::parseFromString(destination)); + manifest->completed = root->getValue("completed"); manifest->items.clear(); - if (root->has("parts")) + auto parts = root->get("parts").extract(); + for (unsigned int i = 0; i < parts->size(); ++i) { - auto parts = root->get("parts").extract(); - if (parts) - { - for (unsigned int i = 0; i < parts->size(); ++i) - { - const auto part_obj = parts->getObject(i); - if (!part_obj) - continue; - Item item; - if (part_obj->has("part_name")) - item.part_name = part_obj->getValue("part_name"); - if (part_obj->has("remote_path")) - item.remote_path = part_obj->getValue("remote_path"); - if (!item.part_name.empty()) - manifest->items.push_back(std::move(item)); - } - } + const auto part_obj = parts->getObject(i); + Item item; + item.part_name = part_obj->getValue("part_name"); + item.remote_path = part_obj->getValue("remote_path"); + manifest->items.push_back(std::move(item)); } return manifest; @@ -168,12 +150,6 @@ struct MergeTreeExportManifest write(); } - void updateCompleted(bool completed_) - { - completed = completed_; - write(); - } - std::vector pendingParts() const { std::vector res; @@ -186,9 +162,16 @@ struct MergeTreeExportManifest std::vector exportedPaths() const { std::vector res; + res.reserve(items.size()); + for (const auto & i : items) + { if (!i.remote_path.empty()) + { res.push_back(i.remote_path); + } + } + return res; } }; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index fdaa190cee5b..cccee58cba66 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -505,13 +505,7 @@ void StorageMergeTree::exportPartsImpl( const auto start_time = std::chrono::system_clock::now(); - std::vector src_parts; - for (const auto & part : exports_tagger->parts_to_export) - { - src_parts.push_back(part); - } - - dest_storage->importMergeTreePartition(*this, src_parts, getContext(), [&](MergeTreePartImportStats stats) + dest_storage->importMergeTreePartition(*this, exports_tagger->parts_to_export, getContext(), [&](MergeTreePartImportStats stats) { std::lock_guard lock(export_partition_commit_id_to_manifest_mutex); auto table_id = getStorageID(); @@ -522,14 +516,7 @@ void StorageMergeTree::exportPartsImpl( return; } - auto manifest = export_partition_commit_id_to_manifest[commit_id]; - if (!manifest) - { - LOG_ERROR(log, "Export manifest not found for commit_id: {}", commit_id); - return; - } - - manifest->updateRemotePathAndWrite(stats.part->name, stats.file_path); + export_partition_commit_id_to_manifest[commit_id]->updateRemotePathAndWrite(stats.part->name, stats.file_path); auto part_log = getContext()->getPartLog(table_id.database_name); @@ -583,71 +570,32 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, String dest_database = query_context->resolveDatabase(command.to_database); auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, command.to_table}, query_context); - /// The target table and the source table are the same. if (dest_storage->getStorageID() == this->getStorageID()) - return; - - auto query = std::make_shared(); + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Exporting to the same table is not allowed"); + } String partition_id = getPartitionIDFromQuery(command.partition, getContext()); auto lock1 = lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef()[Setting::lock_acquire_timeout]); auto merges_blocker = stopMergesAndWaitForPartition(partition_id); - std::lock_guard lock(currently_processing_in_background_mutex); - - if (!already_exported_partition_ids.emplace(partition_id).second) - { - throw Exception(ErrorCodes::PART_IS_LOCKED, "Partition {} has already been exported", partition_id); - } - - std::lock_guard lock2(export_partition_commit_id_to_manifest_mutex); - const auto all_parts = getVisibleDataPartsVectorInPartition(getContext(), partition_id); + auto all_parts = getVisibleDataPartsVectorInPartition(getContext(), partition_id); if (all_parts.empty()) { + LOG_INFO(log, "No parts to export for partition {}, skipping", partition_id); return; } - std::vector src_parts; - - std::vector parts_to_export; - - for (const auto & part : all_parts) - { - bool part_being_exported = false; - /// check all manifests to see if the part is already being exported - /// if so, skip it - for (const auto & [commit_id, manifest] : export_partition_commit_id_to_manifest) - { - auto parts_being_exported_in_this_commit = manifest->items; - - - for (const auto & item : parts_being_exported_in_this_commit) - { - if (item.part_name == part->name) - { - part_being_exported = true; - break; - } - } - } - - if (part_being_exported) - { - continue; - } - - src_parts.emplace_back(part); - parts_to_export.emplace_back(part); - } + std::lock_guard lock(currently_processing_in_background_mutex); - if (parts_to_export.empty()) + if (!already_exported_partition_ids.emplace(partition_id).second) { - return; + throw Exception(ErrorCodes::PART_IS_LOCKED, "Partition {} has already been exported", partition_id); } - auto exports_tagger = std::make_shared(std::move(parts_to_export), *this); + auto exports_tagger = std::make_shared(std::move(all_parts), *this); std::string commit_id; { @@ -660,9 +608,19 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, commit_id = oss.str(); } - const auto manifest = MergeTreeExportManifest::create(getStoragePolicy()->getAnyDisk(), relative_data_path, commit_id, partition_id, dest_storage->getStorageID(), src_parts); + const auto manifest = MergeTreeExportManifest::create( + getStoragePolicy()->getAnyDisk(), + relative_data_path, + commit_id, + partition_id, + dest_storage->getStorageID(), + exports_tagger->parts_to_export); - export_partition_commit_id_to_manifest[commit_id] = manifest; + { + std::lock_guard lock2(export_partition_commit_id_to_manifest_mutex); + + export_partition_commit_id_to_manifest[commit_id] = manifest; + } auto export_partition_function = [this, query_context, partition_id, dest_storage, commit_id, manifest, exports_tagger] () mutable { @@ -1234,13 +1192,25 @@ void StorageMergeTree::loadExportPartition() for (auto it = disk->iterateDirectory(relative_data_path); it->isValid(); it->next()) { const auto & name = it->name(); - if (startsWith(name, "export_") && endsWith(name, ".json")) + if (startsWith(name, "export_")) { try { auto manifest = MergeTreeExportManifest::read(disk, fs::path(relative_data_path) / name); - export_partition_commit_id_to_manifest[manifest->commit_id] = manifest; + already_exported_partition_ids.insert(manifest->partition_id); + + if (manifest->completed) + { + LOG_INFO( + log, + "Export commit {} of partition {} to destination storage {} already completed, skipping", + manifest->commit_id, + manifest->partition_id, + manifest->destination_storage_id.getNameForLogs()); + continue; + } + LOG_DEBUG(log, "Loaded export commit manifest: {} (commit_id: {})", name, manifest->commit_id); } catch (const std::exception & ex) @@ -1251,10 +1221,7 @@ void StorageMergeTree::loadExportPartition() } } - if (!export_partition_commit_id_to_manifest.empty()) - { - restartExportProcess(); - } + restartExportProcess(); } void StorageMergeTree::commitExportPartition(const std::shared_ptr & manifest, const StoragePtr & dest_storage, ContextPtr local_context) @@ -1266,23 +1233,10 @@ void StorageMergeTree::commitExportPartition(const std::shared_ptrcommit_id); } - void StorageMergeTree::restartExportProcess() { - /// I suppose I don't need to lock it here because it is during the startup for (const auto & [commit_id, manifest] : export_partition_commit_id_to_manifest) { - if (manifest->completed) - { - LOG_INFO( - log, - "Export commit {} of partition {} to destination storage {} already completed, skipping", - commit_id, - manifest->partition_id, - manifest->destination_storage_id.getNameForLogs()); - continue; - } - auto destination_storage = reconstructDestinationStorage(manifest->destination_storage_id); if (!destination_storage) { From 185924434046b4c820d4eba98e542ca77beb0601 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 25 Aug 2025 14:27:24 -0300 Subject: [PATCH 18/48] rename from commit id to transaction id --- src/Storages/IStorage.h | 5 +- .../MergeTree/MergeTreeExportManifest.h | 10 +- .../ObjectStorage/StorageObjectStorage.cpp | 4 +- .../ObjectStorage/StorageObjectStorage.h | 2 +- src/Storages/StorageMergeTree.cpp | 103 +++++++----------- src/Storages/StorageMergeTree.h | 16 +-- 6 files changed, 57 insertions(+), 83 deletions(-) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 3cbe52f11ffe..a49631b0fe66 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -214,9 +214,8 @@ class IStorage : public std::enable_shared_from_this, public TypePromo virtuals.set(std::make_unique(std::move(virtuals_))); } - /// Write an export commit file containing the list of exported remote paths - virtual void writeExportCommit( - const String & /* commit_id */, + virtual void commitExportPartitionTransaction( + const String & /* transaction_id */, const String & /* partition_id */, const Strings & /* exported_paths */, ContextPtr /* local_context */) diff --git a/src/Storages/MergeTree/MergeTreeExportManifest.h b/src/Storages/MergeTree/MergeTreeExportManifest.h index 0f488f022e51..8e805f51124e 100644 --- a/src/Storages/MergeTree/MergeTreeExportManifest.h +++ b/src/Storages/MergeTree/MergeTreeExportManifest.h @@ -24,7 +24,7 @@ namespace DB * JSON manifest for exporting a set of parts to object storage. * Layout on disk (pretty-printed JSON): * { - * "commit_id": "", + * "transaction_id": "", * "partition_id": "", * "destination": ".
", * "parts": [ {"part_name": "name", "remote_path": "path-or-empty"}, ... ] @@ -42,7 +42,7 @@ struct MergeTreeExportManifest String remote_path; // empty until uploaded }; - String commit_id; + String transaction_id; String partition_id; StorageID destination_storage_id; std::vector items; @@ -61,7 +61,7 @@ struct MergeTreeExportManifest { auto manifest = std::make_shared(); manifest->disk = disk_; - manifest->commit_id = commit_id_; + manifest->transaction_id = commit_id_; manifest->partition_id = partition_id_; manifest->destination_storage_id = destination_storage_id_; manifest->file_path = std::filesystem::path(path_prefix) / ("export_partition_" + partition_id_ + "_commit_" + commit_id_ + ".json"); @@ -89,7 +89,7 @@ struct MergeTreeExportManifest if (!root) throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid export manifest JSON: {}", file_path_); - manifest->commit_id = root->getValue("commit_id"); + manifest->transaction_id = root->getValue("transaction_id"); manifest->partition_id = root->getValue("partition_id"); const auto destination = root->getValue("destination"); manifest->destination_storage_id = StorageID(QualifiedTableName::parseFromString(destination)); @@ -114,7 +114,7 @@ struct MergeTreeExportManifest auto out = disk->writeFile(file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite); Poco::JSON::Object::Ptr root(new Poco::JSON::Object()); - root->set("commit_id", commit_id); + root->set("transaction_id", transaction_id); root->set("partition_id", partition_id); root->set("destination", destination_storage_id.getQualifiedName().getFullName()); root->set("completed", completed); diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index c7ca8f014550..f9d39e453571 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -658,9 +658,9 @@ void StorageObjectStorage::importMergeTreePartition( // NOTE: Do not write commit file here. The caller manages commit via JSON manifest. } -void StorageObjectStorage::writeExportCommit(const String & commit_id, const String & partition_id, const Strings & exported_paths, ContextPtr 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 + "_" + commit_id; + 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))) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 6c712e0c59d2..db2d1277abca 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -105,7 +105,7 @@ class StorageObjectStorage : public IStorage std::function part_log) override; /// Write an export commit file containing the list of exported remote paths - void writeExportCommit(const String & commit_id, const String & partition_id, const Strings & exported_paths, ContextPtr local_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, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index cccee58cba66..743cb4b17d0d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -494,12 +494,12 @@ void StorageMergeTree::alter( void StorageMergeTree::exportPartsImpl( const CurrentlyExportingPartsTaggerPtr & exports_tagger, - const String & commit_id, + const String & transaction_id, const StoragePtr & dest_storage) { if (exports_tagger->parts_to_export.empty()) { - LOG_INFO(log, "No parts to export for commit {}, skipping", commit_id); + LOG_INFO(log, "No parts to export for transaction {}, skipping", transaction_id); return; } @@ -507,7 +507,7 @@ void StorageMergeTree::exportPartsImpl( dest_storage->importMergeTreePartition(*this, exports_tagger->parts_to_export, getContext(), [&](MergeTreePartImportStats stats) { - std::lock_guard lock(export_partition_commit_id_to_manifest_mutex); + std::lock_guard lock(export_partition_transaction_id_to_manifest_mutex); auto table_id = getStorageID(); if (stats.status.code != 0) @@ -516,7 +516,7 @@ void StorageMergeTree::exportPartsImpl( return; } - export_partition_commit_id_to_manifest[commit_id]->updateRemotePathAndWrite(stats.part->name, stats.file_path); + export_partition_transaction_id_to_manifest[transaction_id]->updateRemotePathAndWrite(stats.part->name, stats.file_path); auto part_log = getContext()->getPartLog(table_id.database_name); @@ -597,7 +597,7 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, auto exports_tagger = std::make_shared(std::move(all_parts), *this); - std::string commit_id; + std::string transaction_id; { std::random_device rd; std::mt19937_64 gen(rd()); @@ -605,28 +605,28 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, uint64_t random_value = dis(gen); std::ostringstream oss; oss << std::hex << random_value; - commit_id = oss.str(); + transaction_id = oss.str(); } const auto manifest = MergeTreeExportManifest::create( getStoragePolicy()->getAnyDisk(), relative_data_path, - commit_id, + transaction_id, partition_id, dest_storage->getStorageID(), exports_tagger->parts_to_export); { - std::lock_guard lock2(export_partition_commit_id_to_manifest_mutex); + std::lock_guard lock2(export_partition_transaction_id_to_manifest_mutex); - export_partition_commit_id_to_manifest[commit_id] = manifest; + export_partition_transaction_id_to_manifest[transaction_id] = manifest; } - auto export_partition_function = [this, query_context, partition_id, dest_storage, commit_id, manifest, exports_tagger] () mutable + auto export_partition_function = [this, query_context, partition_id, dest_storage, transaction_id, manifest, exports_tagger] () mutable { - exportPartsImpl(exports_tagger, commit_id, dest_storage); + exportPartsImpl(exports_tagger, transaction_id, dest_storage); - commitExportPartition(manifest, dest_storage, getContext()); + commitExportPartitionTask(manifest, dest_storage, getContext()); return true; }; @@ -1185,7 +1185,7 @@ void StorageMergeTree::loadMutations() increment.value = std::max(increment.value.load(), current_mutations_by_version.rbegin()->first); } -void StorageMergeTree::loadExportPartition() +void StorageMergeTree::readExportPartitionManifests() { for (const auto & disk : getDisks()) { @@ -1204,40 +1204,29 @@ void StorageMergeTree::loadExportPartition() { LOG_INFO( log, - "Export commit {} of partition {} to destination storage {} already completed, skipping", - manifest->commit_id, + "Export transaction {} of partition {} to destination storage {} already completed, skipping", + manifest->transaction_id, manifest->partition_id, manifest->destination_storage_id.getNameForLogs()); continue; } - LOG_DEBUG(log, "Loaded export commit manifest: {} (commit_id: {})", name, manifest->commit_id); + LOG_DEBUG(log, "Loaded export transaction manifest: {} (transaction_id: {})", name, manifest->transaction_id); } catch (const std::exception & ex) { - LOG_ERROR(log, "Failed to load export commit manifest {}: {}", name, ex.what()); + LOG_ERROR(log, "Failed to load export transaction manifest {}: {}", name, ex.what()); } } } } - - restartExportProcess(); -} - -void StorageMergeTree::commitExportPartition(const std::shared_ptr & manifest, const StoragePtr & dest_storage, ContextPtr local_context) -{ - std::lock_guard lock(export_partition_commit_id_to_manifest_mutex); - dest_storage->writeExportCommit(manifest->commit_id, manifest->partition_id, manifest->exportedPaths(), local_context); - manifest->completed = true; - manifest->write(); - export_partition_commit_id_to_manifest.erase(manifest->commit_id); } -void StorageMergeTree::restartExportProcess() +void StorageMergeTree::resumeExportPartitionTasks() { - for (const auto & [commit_id, manifest] : export_partition_commit_id_to_manifest) + for (const auto & [transaction_id, manifest] : export_partition_transaction_id_to_manifest) { - auto destination_storage = reconstructDestinationStorage(manifest->destination_storage_id); + auto destination_storage = DatabaseCatalog::instance().tryGetTable(manifest->destination_storage_id, getContext()); if (!destination_storage) { LOG_ERROR(log, "Failed to reconstruct destination storage: {}", manifest->destination_storage_id.getNameForLogs()); @@ -1254,7 +1243,11 @@ void StorageMergeTree::restartExportProcess() if (!part) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} not found", part_name); + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Part {} is present in the manifest file {}, but not found in the storage {}", + part_name, + manifest->transaction_id, + getStorageID().getNameForLogs()); } parts_to_export.emplace_back(part); @@ -1263,9 +1256,9 @@ void StorageMergeTree::restartExportProcess() auto exports_tagger = std::make_shared(std::move(parts_to_export), *this); background_moves_assignee.scheduleMoveTask( std::make_shared( - [this, exports_tagger, commit_id, destination_storage, manifest](){ - exportPartsImpl(exports_tagger, commit_id, destination_storage); - commitExportPartition(manifest, destination_storage, getContext()); + [this, exports_tagger, transaction_id, destination_storage, manifest](){ + exportPartsImpl(exports_tagger, transaction_id, destination_storage); + commitExportPartitionTask(manifest, destination_storage, getContext()); return true; }, moves_assignee_trigger, @@ -1273,37 +1266,19 @@ void StorageMergeTree::restartExportProcess() } } -StoragePtr StorageMergeTree::reconstructDestinationStorage(const StorageID & storage_id) const +void StorageMergeTree::loadExportPartition() { - try - { - if (storage_id.empty()) - { - LOG_WARNING(log, "Empty StorageID provided for destination storage reconstruction"); - return nullptr; - } - - // Resolve the StorageID to get the actual storage - auto resolved_storage_id = getContext()->resolveStorageID(storage_id); - - // Get the storage from DatabaseCatalog - auto storage = DatabaseCatalog::instance().tryGetTable(resolved_storage_id, getContext()); - - if (!storage) - { - LOG_ERROR(log, "Failed to find destination storage: {}", storage_id.getNameForLogs()); - return nullptr; - } + readExportPartitionManifests(); + resumeExportPartitionTasks(); +} - LOG_DEBUG(log, "Successfully reconstructed destination storage: {}", storage_id.getNameForLogs()); - return storage; - } - catch (const Exception & e) - { - LOG_ERROR(log, "Exception while reconstructing destination storage {}: {}", - storage_id.getNameForLogs(), e.message()); - return nullptr; - } +void StorageMergeTree::commitExportPartitionTask(const std::shared_ptr & manifest, const StoragePtr & dest_storage, ContextPtr local_context) +{ + std::lock_guard lock(export_partition_transaction_id_to_manifest_mutex); + dest_storage->commitExportPartitionTransaction(manifest->transaction_id, manifest->partition_id, manifest->exportedPaths(), local_context); + manifest->completed = true; + manifest->write(); + export_partition_transaction_id_to_manifest.erase(manifest->transaction_id); } std::expected StorageMergeTree::selectPartsToMerge( diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 3ca922ca8bce..6a05eabfb44a 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -159,8 +159,8 @@ class StorageMergeTree final : public MergeTreeData std::map current_mutations_by_version; - std::mutex export_partition_commit_id_to_manifest_mutex; - std::map> export_partition_commit_id_to_manifest; + std::map> export_partition_transaction_id_to_manifest; + std::mutex export_partition_transaction_id_to_manifest_mutex; /// Unfinished mutations that are required for AlterConversions. MutationCounters mutation_counters; @@ -179,13 +179,13 @@ class StorageMergeTree final : public MergeTreeData /// Load persisted export partition locks and re-apply partition-level merge blockers. void loadExportPartition(); - /// Restart export process for parts that were being exported before restart - void restartExportProcess(); + /// Read export partition manifests from disk and populate internal data structures + void readExportPartitionManifests(); - void commitExportPartition(const std::shared_ptr & manifest, const StoragePtr & dest_storage, ContextPtr context); + /// Restart export process for parts that were being exported before restart + void resumeExportPartitionTasks(); - /// Reconstruct destination storage from StorageID - StoragePtr reconstructDestinationStorage(const StorageID & storage_id) const; + void commitExportPartitionTask(const std::shared_ptr & manifest, const StoragePtr & dest_storage, ContextPtr context); /// Load and initialize deduplication logs. Even if deduplication setting /// equals zero creates object with deduplication window equals zero. @@ -305,7 +305,7 @@ class StorageMergeTree final : public MergeTreeData void assertNotReadonly() const; - void exportPartsImpl(const CurrentlyExportingPartsTaggerPtr & exports_tagger, const String & commit_id, const StoragePtr & dest_storage); + void exportPartsImpl(const CurrentlyExportingPartsTaggerPtr & exports_tagger, const String & transaction_id, const StoragePtr & dest_storage); friend class MergeTreeSink; friend class MergeTreeData; From cdfa5abb54c63c324e8a13942abde0a1a95c06c6 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 25 Aug 2025 14:59:48 -0300 Subject: [PATCH 19/48] use snowflakeid as transaction id --- .../MergeTree/MergeTreeExportManifest.h | 11 +++------- src/Storages/StorageMergeTree.cpp | 22 ++++++------------- src/Storages/StorageMergeTree.h | 2 +- 3 files changed, 11 insertions(+), 24 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeExportManifest.h b/src/Storages/MergeTree/MergeTreeExportManifest.h index 8e805f51124e..08c86685daeb 100644 --- a/src/Storages/MergeTree/MergeTreeExportManifest.h +++ b/src/Storages/MergeTree/MergeTreeExportManifest.h @@ -8,14 +8,11 @@ #include #include #include -#include "Storages/MergeTree/RangesInDataPart.h" #include #include #include #include #include -#include -#include namespace DB { @@ -54,17 +51,17 @@ struct MergeTreeExportManifest static std::shared_ptr create( const DiskPtr & disk_, const String & path_prefix, - const String & commit_id_, + const String & transaction_id_, const String & partition_id_, const StorageID & destination_storage_id_, const std::vector & data_parts) { auto manifest = std::make_shared(); manifest->disk = disk_; - manifest->transaction_id = commit_id_; + manifest->transaction_id = transaction_id_; manifest->partition_id = partition_id_; manifest->destination_storage_id = destination_storage_id_; - manifest->file_path = std::filesystem::path(path_prefix) / ("export_partition_" + partition_id_ + "_commit_" + commit_id_ + ".json"); + manifest->file_path = std::filesystem::path(path_prefix) / ("export_partition_" + partition_id_ + "_transaction_" + transaction_id_ + ".json"); manifest->items.reserve(data_parts.size()); for (const auto & data_part : data_parts) manifest->items.push_back({data_part->name, ""}); @@ -177,5 +174,3 @@ struct MergeTreeExportManifest }; } - - diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 743cb4b17d0d..e4eeab072f14 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -47,11 +47,12 @@ #include #include #include "Core/BackgroundSchedulePool.h" -#include "Core/Names.h" -#include "ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.h" -#include "Storages/MergeTree/MergeMutateSelectedEntry.h" -#include "Storages/MergeTree/MergeTreeExportManifest.h" -#include "Storages/MergeTree/MergeTreePartInfo.h" +#include +#include +#include +#include +#include +#include namespace DB { @@ -597,16 +598,7 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, auto exports_tagger = std::make_shared(std::move(all_parts), *this); - std::string transaction_id; - { - std::random_device rd; - std::mt19937_64 gen(rd()); - std::uniform_int_distribution dis; - uint64_t random_value = dis(gen); - std::ostringstream oss; - oss << std::hex << random_value; - transaction_id = oss.str(); - } + const auto transaction_id = std::to_string(generateSnowflakeID()); const auto manifest = MergeTreeExportManifest::create( getStoragePolicy()->getAnyDisk(), diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 6a05eabfb44a..842522206469 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -17,7 +17,7 @@ #include #include -#include "Storages/MergeTree/MergeTreeExportManifest.h" +#include namespace DB From 9f9fcb2c200f54b4f1115cea2b65200ea3936278 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 25 Aug 2025 15:11:35 -0300 Subject: [PATCH 20/48] add back the sync behavior --- src/Storages/StorageMergeTree.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index e4eeab072f14..efa2e721dcb2 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -630,9 +630,9 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, } else { - /// async always - background_moves_assignee.scheduleMoveTask( - std::make_shared(export_partition_function, moves_assignee_trigger, getStorageID())); + exportPartsImpl(exports_tagger, transaction_id, dest_storage); + + commitExportPartitionTask(manifest, dest_storage, getContext()); } } From bfb72aea0629438e317d22cb578a48d5f86cec6e Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 25 Aug 2025 16:00:06 -0300 Subject: [PATCH 21/48] minor changes --- src/Interpreters/Context.h | 3 +-- src/Storages/MergeTree/MergeTreeData.h | 26 +++++++++---------- .../MergeTree/MergeTreeExportManifest.h | 1 + .../ObjectStorage/StorageObjectStorage.cpp | 1 + 4 files changed, 15 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 5f5ed675112c..344e86ceb14f 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -32,8 +32,6 @@ #include #include -#include "Storages/ExportsList.h" - namespace Poco::Net { @@ -90,6 +88,7 @@ class AsynchronousMetrics; class BackgroundSchedulePool; class MergeList; class MovesList; +class ExportsList; class ReplicatedFetchList; class RefreshSet; class Cluster; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index e63baf8ab430..5c524a2f4007 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1239,20 +1239,6 @@ class MergeTreeData : public IStorage, public WithMutableContext /// under lockForShare if rename is possible. String relative_data_path; - /// RAII Wrapper for atomic work with currently moving parts - /// Acquire them in constructor and remove them in destructor - /// Uses data.currently_moving_parts_mutex - struct CurrentlyMovingPartsTagger - { - MergeTreeMovingParts parts_to_move; - MergeTreeData & data; - CurrentlyMovingPartsTagger(MergeTreeMovingParts && moving_parts_, MergeTreeData & data_); - - ~CurrentlyMovingPartsTagger(); - }; - - using CurrentlyMovingPartsTaggerPtr = std::shared_ptr; - private: /// Columns and secondary indices sizes can be calculated lazily. mutable std::mutex columns_and_secondary_indices_sizes_mutex; @@ -1687,7 +1673,19 @@ class MergeTreeData : public IStorage, public WithMutableContext DataPartsVector * out_covered_parts, bool rename_in_transaction); + /// RAII Wrapper for atomic work with currently moving parts + /// Acquire them in constructor and remove them in destructor + /// Uses data.currently_moving_parts_mutex + struct CurrentlyMovingPartsTagger + { + MergeTreeMovingParts parts_to_move; + MergeTreeData & data; + CurrentlyMovingPartsTagger(MergeTreeMovingParts && moving_parts_, MergeTreeData & data_); + + ~CurrentlyMovingPartsTagger(); + }; + using CurrentlyMovingPartsTaggerPtr = std::shared_ptr; /// Moves part to specified space, used in ALTER ... MOVE ... queries std::future movePartsToSpace(const CurrentlyMovingPartsTaggerPtr & moving_tagger, const ReadSettings & read_settings, const WriteSettings & write_settings, bool async); diff --git a/src/Storages/MergeTree/MergeTreeExportManifest.h b/src/Storages/MergeTree/MergeTreeExportManifest.h index 08c86685daeb..6117d220c9da 100644 --- a/src/Storages/MergeTree/MergeTreeExportManifest.h +++ b/src/Storages/MergeTree/MergeTreeExportManifest.h @@ -24,6 +24,7 @@ namespace DB * "transaction_id": "", * "partition_id": "", * "destination": ".
", + * "completed": , * "parts": [ {"part_name": "name", "remote_path": "path-or-empty"}, ... ] * } */ diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index f9d39e453571..8eea586fc68a 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -19,6 +19,7 @@ #include #include +#include #include #include #include From 7dbb53f51a293fddf2fc1e57bbe28e968d4cfdd2 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 26 Aug 2025 08:11:24 -0300 Subject: [PATCH 22/48] add missing include for build --- src/Interpreters/Context.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index efec76e59ee5..b1dc2dd595de 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -34,6 +34,7 @@ #include #include #include +#include #include #include #include From 25066632cadfeccebd3c7284d74677cb25c8363f Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 26 Aug 2025 08:11:59 -0300 Subject: [PATCH 23/48] freakin ai code suggestions.. --- src/Interpreters/Context.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b1dc2dd595de..caf1831e114e 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -34,7 +34,7 @@ #include #include #include -#include +#include #include #include #include From 6bc7c09f4462f7d87d02b75001066cfe8b00c176 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 26 Aug 2025 16:53:51 -0300 Subject: [PATCH 24/48] add roundtrip check --- src/Storages/StorageMergeTree.cpp | 6 +++--- .../03572_export_mt_part_to_object_storage.reference | 8 ++++++++ .../03572_export_mt_part_to_object_storage.sh | 12 ++++++++++-- 3 files changed, 21 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index efa2e721dcb2..a9f2255e560f 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -630,9 +630,9 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, } else { - exportPartsImpl(exports_tagger, transaction_id, dest_storage); - - commitExportPartitionTask(manifest, dest_storage, getContext()); + /// always in the background for now. I need to sort out the locks + background_moves_assignee.scheduleMoveTask( + std::make_shared(export_partition_function, moves_assignee_trigger, getStorageID())); } } diff --git a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.reference b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.reference index d7e137f36cba..760fd247967a 100644 --- a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.reference +++ b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.reference @@ -23,3 +23,11 @@ test/s3_table_NAME/year=2020/SNOWFLAKE_ID.parquet 3 2020 test/s3_table_NAME/year=2021/SNOWFLAKE_ID.parquet 4 2021 test/s3_table_NAME/year=2020/SNOWFLAKE_ID.parquet 5 2020 test/s3_table_NAME/year=2020/SNOWFLAKE_ID.parquet 6 2020 +---- Round trip check: create a new MergeTree table as SELECT * from s3_table +---- Data in roundtrip MergeTree table (should match s3_table) +1 2020 +2 2020 +3 2020 +4 2021 +5 2020 +6 2020 diff --git a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh index f55893268074..8de10a66119b 100755 --- a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh +++ b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh @@ -6,12 +6,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) mt_table="mt_table_${RANDOM}" s3_table="s3_table_${RANDOM}" +mt_table_roundtrip="mt_table_roundtrip_${RANDOM}" query() { $CLICKHOUSE_CLIENT --query "$1" } -query "DROP TABLE IF EXISTS $mt_table, $s3_table" +query "DROP TABLE IF EXISTS $mt_table, $s3_table, $mt_table_roundtrip" query "CREATE TABLE $mt_table (id UInt64, year UInt16) ENGINE = MergeTree() 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" @@ -41,5 +42,12 @@ query "ALTER TABLE $mt_table EXPORT PARTITION ID '2021' TO TABLE $s3_table SETTI echo "---- Assert both partitions are there" query "SELECT DISTINCT ON (id) replaceRegexpAll(replaceRegexpAll(_path, '$s3_table', 's3_table_NAME'), '[^/]+\\.parquet', 'SNOWFLAKE_ID.parquet'), * FROM $s3_table ORDER BY id" +echo "---- Round trip check: create a new MergeTree table as SELECT * from s3_table" + +query "CREATE TABLE $mt_table_roundtrip ENGINE = MergeTree() PARTITION BY year ORDER BY tuple() AS SELECT * FROM $s3_table" + +echo "---- Data in roundtrip MergeTree table (should match s3_table)" +query "SELECT DISTINCT ON (id) * FROM $mt_table_roundtrip ORDER BY id" + query "SYSTEM START MERGES" -query "DROP TABLE IF EXISTS $mt_table, $s3_table" +query "DROP TABLE IF EXISTS $mt_table, $s3_table, $mt_table_roundtrip" From 8e08991f99223ed43775eab0331f625d7e8fc732 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 27 Aug 2025 10:02:54 -0300 Subject: [PATCH 25/48] opsy --- src/Storages/StorageMergeTree.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a9f2255e560f..7352cff02226 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1202,7 +1202,9 @@ void StorageMergeTree::readExportPartitionManifests() manifest->destination_storage_id.getNameForLogs()); continue; } - + + export_partition_transaction_id_to_manifest.emplace(manifest->transaction_id, manifest); + LOG_DEBUG(log, "Loaded export transaction manifest: {} (transaction_id: {})", name, manifest->transaction_id); } catch (const std::exception & ex) From 54bf67872b0b322a50aeb4c0cac23abdc40ec562 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 27 Aug 2025 14:12:20 -0300 Subject: [PATCH 26/48] remove export part, add some partition exp sanity checking, change exception message --- src/Parsers/CommonParsers.h | 1 - src/Parsers/ParserAlterQuery.cpp | 18 ----------------- src/Storages/IStorage.h | 2 ++ .../ObjectStorage/StorageObjectStorage.cpp | 5 +++++ .../ObjectStorage/StorageObjectStorage.h | 2 ++ src/Storages/StorageMergeTree.cpp | 20 +++++++++++++++++-- 6 files changed, 27 insertions(+), 21 deletions(-) diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index a0a59288b4a3..a634df26603a 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -331,7 +331,6 @@ namespace DB MR_MACROS(MONTHS, "MONTHS") \ 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") \ diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index cb9c541f3a9f..7278a9152787 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -82,7 +82,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_forget_partition(Keyword::FORGET_PARTITION); 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); @@ -582,23 +581,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected return false; command->move_destination_type = DataDestinationType::TABLE; } - else if (s_export_part.ignore(pos, expected)) - { - if (!parser_partition.parse(pos, command_partition, expected)) - return false; - - command->type = ASTAlterCommand::EXPORT_PARTITION; - command->part = true; - - 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_add_constraint.ignore(pos, expected)) { if (s_if_not_exists.ignore(pos, expected)) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index a49631b0fe66..8998a6ba69ac 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -454,6 +454,8 @@ class IStorage : public std::enable_shared_from_this, public TypePromo ContextPtr /*context*/, bool /*async_insert*/); + virtual bool supportsImportMergeTreePartition() const { return false; } + virtual void importMergeTreePartition( const MergeTreeData &, const std::vector &, diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 8eea586fc68a..c394ab80e87c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -548,6 +548,11 @@ void StorageObjectStorage::read( query_plan.addStep(std::move(read_step)); } +bool StorageObjectStorage::supportsImportMergeTreePartition() const +{ + return configuration->partition_strategy != nullptr && configuration->partition_strategy_type == PartitionStrategyFactory::StrategyType::HIVE; +} + void StorageObjectStorage::importMergeTreePartition( const MergeTreeData & merge_tree_data, const std::vector & data_parts, diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index db2d1277abca..511d3d79c9e2 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -98,6 +98,8 @@ class StorageObjectStorage : public IStorage ContextPtr context, bool async_insert) override; + bool supportsImportMergeTreePartition() const override; + void importMergeTreePartition( const MergeTreeData & merge_tree_data, const std::vector & data_parts, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 7352cff02226..83f53705681e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -576,9 +576,25 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, throw Exception(ErrorCodes::BAD_ARGUMENTS, "Exporting to the same table is not allowed"); } + if (!dest_storage->supportsImportMergeTreePartition()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Destination storage {} does not support importing merge tree partitions", 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 (query_to_string(src_snapshot->getPartitionKeyAST()) != query_to_string(destination_snapshot->getPartitionKeyAST())) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key"); + String partition_id = getPartitionIDFromQuery(command.partition, getContext()); - auto lock1 = lockForShare(query_context->getCurrentQueryId(), query_context->getSettingsRef()[Setting::lock_acquire_timeout]); + auto lock1 = lockForShare( + query_context->getCurrentQueryId(), + query_context->getSettingsRef()[Setting::lock_acquire_timeout]); auto merges_blocker = stopMergesAndWaitForPartition(partition_id); auto all_parts = getVisibleDataPartsVectorInPartition(getContext(), partition_id); @@ -716,7 +732,7 @@ CurrentlyExportingPartsTagger::CurrentlyExportingPartsTagger(std::vectorname); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Tagging already tagged part {}. This is a bug.", part->name); } CurrentlyExportingPartsTagger::~CurrentlyExportingPartsTagger() From 71bc26f5702a0ca4da8943dc939b5fa111b520ee Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 27 Aug 2025 15:27:28 -0300 Subject: [PATCH 27/48] add tests --- .../03572_export_mt_part_to_object_storage.sh | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh index 8de10a66119b..593f6e5d22f3 100755 --- a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh +++ b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh @@ -7,12 +7,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) mt_table="mt_table_${RANDOM}" s3_table="s3_table_${RANDOM}" mt_table_roundtrip="mt_table_roundtrip_${RANDOM}" +invalid_schema_table="invalid_schema_table_${RANDOM}" query() { $CLICKHOUSE_CLIENT --query "$1" } -query "DROP TABLE IF EXISTS $mt_table, $s3_table, $mt_table_roundtrip" +query "DROP TABLE IF EXISTS $mt_table, $s3_table, $mt_table_roundtrip, $invalid_schema_table" query "CREATE TABLE $mt_table (id UInt64, year UInt16) ENGINE = MergeTree() 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" @@ -49,5 +50,15 @@ query "CREATE TABLE $mt_table_roundtrip ENGINE = MergeTree() PARTITION BY year O echo "---- Data in roundtrip MergeTree table (should match s3_table)" query "SELECT DISTINCT ON (id) * FROM $mt_table_roundtrip ORDER BY id" +# Create a table with a different partition key and export a partition to it. It should throw +query "CREATE TABLE $invalid_schema_table (id UInt64, x UInt16) ENGINE = S3(s3_conn, filename='$invalid_schema_table', format=Parquet, partition_strategy='hive') PARTITION BY x" +query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $invalid_schema_table SETTINGS allow_experimental_export_merge_tree_partition = 1, export_merge_tree_partition_background_execution = 0 -- {serverError BAD_ARGUMENTS}" + +query "DROP TABLE $invalid_schema_table" + +# The only partition strategy that supports exports is hive. Wildcard should throw +query "CREATE TABLE $invalid_schema_table (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$invalid_schema_table/{_partition_id}', format=Parquet, partition_strategy='wildcard') PARTITION BY (id, year)" +query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $invalid_schema_table SETTINGS allow_experimental_export_merge_tree_partition = 1, export_merge_tree_partition_background_execution = 0 -- {serverError NOT_IMPLEMENTED}" + query "SYSTEM START MERGES" query "DROP TABLE IF EXISTS $mt_table, $s3_table, $mt_table_roundtrip" From b489f83c2f4efba2a2ba2cb42e8e00a9075e9d8c Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 1 Sep 2025 08:53:28 -0300 Subject: [PATCH 28/48] Refactor to use a background task instead of inline code --- src/Storages/MergeTree/MergeTreeData.h | 1 + .../MergeTree/MergeTreeExportManifest.h | 3 + .../ExportPartitionPlainMergeTreeTask.cpp | 215 ++++++++++++++++++ .../ExportPartitionPlainMergeTreeTask.h | 70 ++++++ ...ObjectStorageMergeTreePartImporterSink.cpp | 29 ++- src/Storages/StorageMergeTree.cpp | 119 ++-------- src/Storages/StorageMergeTree.h | 5 +- .../README.md | 48 ++++ .../__init__.py | 0 .../configs/named_collections.xml | 9 + .../test.py | 104 +++++++++ 11 files changed, 493 insertions(+), 110 deletions(-) create mode 100644 src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.cpp create mode 100644 src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.h create mode 100644 tests/integration/test_export_partition_disaster_recovery/README.md create mode 100644 tests/integration/test_export_partition_disaster_recovery/__init__.py create mode 100644 tests/integration/test_export_partition_disaster_recovery/configs/named_collections.xml create mode 100644 tests/integration/test_export_partition_disaster_recovery/test.py diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 5c524a2f4007..53ebe9dfda45 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1232,6 +1232,7 @@ class MergeTreeData : public IStorage, public WithMutableContext friend class MergeTask; friend class IPartMetadataManager; friend class IMergedBlockOutputStream; // for access to log + friend class ExportPartitionPlainMergeTreeTask; bool require_part_metadata; diff --git a/src/Storages/MergeTree/MergeTreeExportManifest.h b/src/Storages/MergeTree/MergeTreeExportManifest.h index 6117d220c9da..c546daf58e69 100644 --- a/src/Storages/MergeTree/MergeTreeExportManifest.h +++ b/src/Storages/MergeTree/MergeTreeExportManifest.h @@ -13,6 +13,7 @@ #include #include #include +#include namespace DB { @@ -30,6 +31,8 @@ namespace DB */ struct MergeTreeExportManifest { + using DataPartPtr = std::shared_ptr; + MergeTreeExportManifest() : destination_storage_id(StorageID::createEmpty()) {} diff --git a/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.cpp b/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.cpp new file mode 100644 index 000000000000..3b668c30f1d9 --- /dev/null +++ b/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.cpp @@ -0,0 +1,215 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +ExportPartitionPlainMergeTreeTask::ExportPartitionPlainMergeTreeTask( + StorageMergeTree & storage_, + const std::shared_ptr & exports_tagger_, + const StoragePtr & destination_storage_, + ContextPtr context_, + std::shared_ptr manifest_, + IExecutableTask::TaskResultCallback & task_result_callback_, + size_t max_retries_) + : storage(storage_) + , exports_tagger(exports_tagger_) + , destination_storage(destination_storage_) + , context(std::move(context_)) + , manifest(std::move(manifest_)) + , task_result_callback(task_result_callback_) + , max_retries(max_retries_) +{ + UInt64 transaction_id = std::stoull(manifest->transaction_id); + priority.value = transaction_id; +} + +StorageID ExportPartitionPlainMergeTreeTask::getStorageID() const +{ + return storage.getStorageID(); +} + +String ExportPartitionPlainMergeTreeTask::getQueryId() const +{ + return getStorageID().getShortName() + "::export_partition::" + manifest->transaction_id; +} + +bool ExportPartitionPlainMergeTreeTask::executeStep() +{ + if (cancelled) + return false; + + switch (state) + { + case State::NEED_PREPARE: + { + prepare(); + state = State::NEED_EXECUTE; + return true; + } + case State::NEED_EXECUTE: + { + executeExport(); + state = State::NEED_ANALYZE; + return true; + } + case State::NEED_ANALYZE: + { + if (exportedAllIndividualParts()) + { + state = State::NEED_COMMIT; + return true; + } + else + { + if (max_retries > retry_count) + { + LOG_INFO(getLogger("ExportMergeTreePartitionToObjectStorageTask"), + "Retrying export attempt {} for partition {}", + retry_count, manifest->partition_id); + state = State::NEED_EXECUTE; + retry_count++; + return true; + } + + /// do we need to update the state here? + return false; + } + } + case State::NEED_COMMIT: + { + if (commitExport()) + { + state = State::SUCCESS; + } + else + { + state = State::NEED_EXECUTE; + retry_count++; + LOG_INFO(getLogger("ExportMergeTreePartitionToObjectStorageTask"), + "Retrying export attempt {} for partition {}", + retry_count, manifest->partition_id); + } + + return true; + } + case State::SUCCESS: + { + return false; + } + } + + return false; +} + +void ExportPartitionPlainMergeTreeTask::prepare() +{ + stopwatch_ptr = std::make_unique(); +} + +void ExportPartitionPlainMergeTreeTask::executeExport() +{ + if (cancelled) + return; + + try + { + // Build a vector of parts that have not been exported yet (i.e., not present in manifest->exportedPaths) + std::vector parts_to_export; + const auto & items = manifest->items; + for (const auto & part : exports_tagger->parts_to_export) + { + if (std::find_if(items.begin(), items.end(), [&part](const auto & item) { + return item.part_name == part->name && item.remote_path.empty(); + }) != items.end()) + parts_to_export.push_back(part); + } + + std::function part_log_wrapper = [this](MergeTreePartImportStats stats) { + + std::lock_guard lock(storage.export_partition_transaction_id_to_manifest_mutex); + auto table_id = storage.getStorageID(); + + if (stats.status.code != 0) + { + LOG_INFO(getLogger("ExportMergeTreePartitionToObjectStorageTask"), "Error importing part {}: {}", stats.part->name, stats.status.message); + return; + } + + storage.export_partition_transaction_id_to_manifest[manifest->transaction_id]->updateRemotePathAndWrite( + stats.part->name, + stats.file_path); + + UInt64 elapsed_ns = stopwatch_ptr->elapsedNanoseconds(); + + storage.writePartLog( + PartLogElement::Type::EXPORT_PART, + stats.status, + elapsed_ns, + stats.part->name, + stats.part, + {stats.part}, + nullptr, + nullptr); + }; + + destination_storage->importMergeTreePartition( + storage, + parts_to_export, + context, + part_log_wrapper); + } + catch (...) + { + LOG_ERROR(getLogger("ExportMergeTreePartitionToObjectStorageTask"), + "Export attempt failed completely: {}", getCurrentExceptionMessage(true)); + + throw; + } +} + +bool ExportPartitionPlainMergeTreeTask::commitExport() +{ + std::lock_guard lock(storage.export_partition_transaction_id_to_manifest_mutex); + + destination_storage->commitExportPartitionTransaction( + manifest->transaction_id, + manifest->partition_id, + manifest->exportedPaths(), + context); + + manifest->completed = true; + manifest->write(); + + storage.export_partition_transaction_id_to_manifest.erase(manifest->transaction_id); + + LOG_INFO(getLogger("ExportMergeTreePartitionToObjectStorageTask"), + "Successfully committed export transaction {} for partition {}", + manifest->transaction_id, manifest->partition_id); + + return true; +} + +void ExportPartitionPlainMergeTreeTask::onCompleted() +{ + std::lock_guard lock(storage.export_partition_transaction_id_to_manifest_mutex); + task_result_callback(manifest->completed); +} + +void ExportPartitionPlainMergeTreeTask::cancel() noexcept +{ + cancelled = true; +} + +bool ExportPartitionPlainMergeTreeTask::exportedAllIndividualParts() const +{ + std::lock_guard lock(storage.export_partition_transaction_id_to_manifest_mutex); + return manifest->exportedPaths().size() == manifest->items.size(); +} + +} diff --git a/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.h b/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.h new file mode 100644 index 000000000000..abb1948295ee --- /dev/null +++ b/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.h @@ -0,0 +1,70 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class MergeTreeData; + +class ExportPartitionPlainMergeTreeTask : public IExecutableTask +{ +public: + ExportPartitionPlainMergeTreeTask( + StorageMergeTree & storage_, + const std::shared_ptr & exports_tagger_, + const StoragePtr & destination_storage_, + ContextPtr context_, + std::shared_ptr manifest_, + IExecutableTask::TaskResultCallback & task_result_callback_, + size_t max_retries_ = 3); + + void onCompleted() override; + bool executeStep() override; + void cancel() noexcept override; + StorageID getStorageID() const override; + Priority getPriority() const override { return priority; } + String getQueryId() const override; + +private: + void prepare(); + void executeExport(); + bool commitExport(); + bool exportedAllIndividualParts() const; + + enum class State : uint8_t + { + NEED_PREPARE, + NEED_EXECUTE, + NEED_ANALYZE, + NEED_COMMIT, + SUCCESS + }; + + State state{State::NEED_PREPARE}; + + StorageMergeTree & storage; + std::shared_ptr exports_tagger; + StoragePtr destination_storage; + ContextPtr context; + std::shared_ptr manifest; + IExecutableTask::TaskResultCallback task_result_callback; + + size_t max_retries; + size_t retry_count = 0; + Priority priority; + std::unique_ptr stopwatch_ptr; + + bool cancelled = false; + std::exception_ptr current_exception; +}; + +using ExportPartitionPlainMergeTreeTaskPtr = std::shared_ptr; + +} diff --git a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.cpp b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.cpp index fc7943b467b0..f027e4ac950c 100644 --- a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.cpp +++ b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.cpp @@ -38,14 +38,30 @@ String StorageObjectStorageMergeTreePartImporterSink::getName() const void StorageObjectStorageMergeTreePartImporterSink::consume(Chunk & chunk) { - sink->consume(chunk); + if (stats.status.code != 0) + return; - stats.read_bytes += chunk.bytes(); - stats.read_rows += chunk.getNumRows(); + try + { + sink->consume(chunk); + stats.read_bytes += chunk.bytes(); + stats.read_rows += chunk.getNumRows(); + + stats.status = ExecutionStatus(0, "Success"); + } catch (...) { + stats.status = ExecutionStatus(-1, "Error importing part"); + part_log(stats); + } } void StorageObjectStorageMergeTreePartImporterSink::onFinish() { + if (stats.status.code != 0) + { + sink->cancel(); + return; + } + sink->onFinish(); if (const auto object_metadata = object_storage->tryGetObjectMetadata(stats.file_path)) { @@ -54,11 +70,10 @@ void StorageObjectStorageMergeTreePartImporterSink::onFinish() part_log(stats); } -void StorageObjectStorageMergeTreePartImporterSink::onException(std::exception_ptr exception) +void StorageObjectStorageMergeTreePartImporterSink::onException(std::exception_ptr) { - sink->onException(exception); - stats.status = ExecutionStatus(-1, "Error importing part"); - part_log(stats); + /// we should not reach here + std::terminate(); } } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 83f53705681e..06370851d10c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -48,6 +48,8 @@ #include #include "Core/BackgroundSchedulePool.h" #include +#include +#include #include #include #include @@ -493,69 +495,6 @@ void StorageMergeTree::alter( } } -void StorageMergeTree::exportPartsImpl( - const CurrentlyExportingPartsTaggerPtr & exports_tagger, - const String & transaction_id, - const StoragePtr & dest_storage) -{ - if (exports_tagger->parts_to_export.empty()) - { - LOG_INFO(log, "No parts to export for transaction {}, skipping", transaction_id); - return; - } - - const auto start_time = std::chrono::system_clock::now(); - - dest_storage->importMergeTreePartition(*this, exports_tagger->parts_to_export, getContext(), [&](MergeTreePartImportStats stats) - { - std::lock_guard lock(export_partition_transaction_id_to_manifest_mutex); - auto table_id = getStorageID(); - - if (stats.status.code != 0) - { - LOG_ERROR(log, "Error importing part {}: {}", stats.part->name, stats.status.message); - return; - } - - export_partition_transaction_id_to_manifest[transaction_id]->updateRemotePathAndWrite(stats.part->name, stats.file_path); - - auto part_log = getContext()->getPartLog(table_id.database_name); - - if (!part_log) - return; - - PartLogElement part_log_elem; - part_log_elem.event_type = PartLogElement::Type::EXPORT_PART; - part_log_elem.merge_algorithm = PartLogElement::PartMergeAlgorithm::UNDECIDED; - part_log_elem.merge_reason = PartLogElement::MergeReasonType::NOT_A_MERGE; - - part_log_elem.database_name = table_id.database_name; - part_log_elem.table_name = table_id.table_name; - part_log_elem.table_uuid = table_id.uuid; - part_log_elem.partition_id = MergeTreePartInfo::fromPartName(stats.part->name, format_version).getPartitionId(); - // construct event_time and event_time_microseconds using the same time point - // so that the two times will always be equal up to a precision of a second. - const auto time_now = std::chrono::system_clock::now(); - part_log_elem.event_time = timeInSeconds(time_now); - part_log_elem.event_time_microseconds = timeInMicroseconds(time_now); - - part_log_elem.duration_ms = std::chrono::duration_cast(start_time - time_now).count() / 1000000; - part_log_elem.error = static_cast(stats.status.code); - part_log_elem.exception = stats.status.message; - part_log_elem.path_on_disk = stats.file_path; - part_log_elem.part_name = stats.part->name; - part_log_elem.bytes_compressed_on_disk = stats.bytes_on_disk; - part_log_elem.rows = stats.part->rows_count; - part_log_elem.disk_name = dest_storage->getName(); - part_log_elem.part_type = stats.part->getType(); - part_log_elem.source_part_names = {stats.part->name}; - part_log_elem.rows_read = stats.read_rows; - part_log_elem.bytes_read_uncompressed = stats.read_bytes; - - part_log->add(std::move(part_log_elem)); - }); -} - /* * For now, this function is meant to be used when exporting to different formats (i.e, the case where data needs to be re-encoded / serialized) * For the cases where this is not necessary, there are way more optimal ways of doing that, such as hard links implemented by `movePartitionToTable` @@ -630,26 +569,15 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, export_partition_transaction_id_to_manifest[transaction_id] = manifest; } - auto export_partition_function = [this, query_context, partition_id, dest_storage, transaction_id, manifest, exports_tagger] () mutable - { - exportPartsImpl(exports_tagger, transaction_id, dest_storage); - - commitExportPartitionTask(manifest, dest_storage, getContext()); + auto task = std::make_shared( + *this, + exports_tagger, + dest_storage, + getContext(), + manifest, + moves_assignee_trigger); - return true; - }; - - if (query_context->getSettingsRef()[Setting::export_merge_tree_partition_background_execution]) - { - background_moves_assignee.scheduleMoveTask( - std::make_shared(export_partition_function, moves_assignee_trigger, getStorageID())); - } - else - { - /// always in the background for now. I need to sort out the locks - background_moves_assignee.scheduleMoveTask( - std::make_shared(export_partition_function, moves_assignee_trigger, getStorageID())); - } + background_moves_assignee.scheduleMoveTask(task); } /// While exists, marks parts as 'currently_merging_mutating_parts' and reserves free space on filesystem. @@ -1264,15 +1192,16 @@ void StorageMergeTree::resumeExportPartitionTasks() } auto exports_tagger = std::make_shared(std::move(parts_to_export), *this); - background_moves_assignee.scheduleMoveTask( - std::make_shared( - [this, exports_tagger, transaction_id, destination_storage, manifest](){ - exportPartsImpl(exports_tagger, transaction_id, destination_storage); - commitExportPartitionTask(manifest, destination_storage, getContext()); - return true; - }, - moves_assignee_trigger, - getStorageID())); + + auto task = std::make_shared( + *this, + exports_tagger, + destination_storage, + getContext(), + manifest, + moves_assignee_trigger); + + background_moves_assignee.scheduleMoveTask(task); } } @@ -1282,14 +1211,6 @@ void StorageMergeTree::loadExportPartition() resumeExportPartitionTasks(); } -void StorageMergeTree::commitExportPartitionTask(const std::shared_ptr & manifest, const StoragePtr & dest_storage, ContextPtr local_context) -{ - std::lock_guard lock(export_partition_transaction_id_to_manifest_mutex); - dest_storage->commitExportPartitionTransaction(manifest->transaction_id, manifest->partition_id, manifest->exportedPaths(), local_context); - manifest->completed = true; - manifest->write(); - export_partition_transaction_id_to_manifest.erase(manifest->transaction_id); -} std::expected StorageMergeTree::selectPartsToMerge( const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 842522206469..f54679bdde33 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -185,8 +185,6 @@ class StorageMergeTree final : public MergeTreeData /// Restart export process for parts that were being exported before restart void resumeExportPartitionTasks(); - void commitExportPartitionTask(const std::shared_ptr & manifest, const StoragePtr & dest_storage, ContextPtr context); - /// Load and initialize deduplication logs. Even if deduplication setting /// equals zero creates object with deduplication window equals zero. void loadDeduplicationLog(); @@ -305,12 +303,11 @@ class StorageMergeTree final : public MergeTreeData void assertNotReadonly() const; - void exportPartsImpl(const CurrentlyExportingPartsTaggerPtr & exports_tagger, const String & transaction_id, const StoragePtr & dest_storage); - friend class MergeTreeSink; friend class MergeTreeData; friend class MergePlainMergeTreeTask; friend class MutatePlainMergeTreeTask; + friend class ExportPartitionPlainMergeTreeTask; struct DataValidationTasks : public IStorage::DataValidationTasksBase { diff --git a/tests/integration/test_export_partition_disaster_recovery/README.md b/tests/integration/test_export_partition_disaster_recovery/README.md new file mode 100644 index 000000000000..3a107185a7e3 --- /dev/null +++ b/tests/integration/test_export_partition_disaster_recovery/README.md @@ -0,0 +1,48 @@ +# Export Partition Disaster Recovery Tests + +This test suite validates the disaster recovery capabilities of the EXPORT PARTITION operation in ClickHouse. + +## Test Scenarios + +1. **Basic Server Kill** - Tests simple crash during export with recovery +2. **Network Delays** - Tests crash during network-delayed operations +3. **Multiple Crashes** - Tests resilience across multiple crash/restart cycles +4. **Large Datasets** - Tests recovery with substantial data that ensures slow operations +5. **State Persistence** - Tests that recovery state/checkpoints survive crashes +6. **Concurrent Operations** - Tests crashes during multiple simultaneous exports + +## Running Tests + +```bash +# Run all disaster recovery tests +pytest -v test_export_partition_disaster_recovery/ + +# Run specific test +pytest -v test_export_partition_disaster_recovery/test.py::TestExportPartitionDisasterRecovery::test_export_partition_with_server_kill_basic + +# Run with detailed logging +pytest -v -s test_export_partition_disaster_recovery/ +``` + +## Test Infrastructure + +- Uses `BrokenS3` mock to simulate slow S3 operations +- Uses `PartitionManager` for network delay simulation +- Uses `kill -9` to simulate hard server crashes +- Validates data integrity and operation resumability after recovery +- Tests various data sizes and partition configurations + +## Requirements + +- ClickHouse with S3 support +- MinIO integration test environment +- `stay_alive=True` cluster configuration for crash testing + +## Why No Zookeeper? + +These tests focus on **single-node disaster recovery** for export partition operations on plain MergeTree tables. Zookeeper is only needed for: +- ReplicatedMergeTree coordination +- Multi-node distributed operations +- Cross-replica synchronization + +Since we're testing crash recovery of individual export operations, no coordination is required. diff --git a/tests/integration/test_export_partition_disaster_recovery/__init__.py b/tests/integration/test_export_partition_disaster_recovery/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_export_partition_disaster_recovery/configs/named_collections.xml b/tests/integration/test_export_partition_disaster_recovery/configs/named_collections.xml new file mode 100644 index 000000000000..573822539c50 --- /dev/null +++ b/tests/integration/test_export_partition_disaster_recovery/configs/named_collections.xml @@ -0,0 +1,9 @@ + + + + http://minio1:9001/root/data + minio + ClickHouse_Minio_P@ssw0rd + + + diff --git a/tests/integration/test_export_partition_disaster_recovery/test.py b/tests/integration/test_export_partition_disaster_recovery/test.py new file mode 100644 index 000000000000..c4699410657a --- /dev/null +++ b/tests/integration/test_export_partition_disaster_recovery/test.py @@ -0,0 +1,104 @@ +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( + "node", + main_configs=["configs/named_collections.xml"], + user_configs=[], + with_minio=True, + stay_alive=True, + ) + cluster.add_instance( + "node2", + main_configs=["configs/named_collections.xml"], + user_configs=[], + with_minio=True, + ) + logging.info("Starting cluster...") + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_export_partition_with_network_delays(cluster): + """Test server kill during export with network delays.""" + node = cluster.instances["node"] + table_name = "disaster_test_network" + s3_table = "destination_s3_network" + + node.query(f""" + CREATE TABLE {table_name} ( + id UInt64, + year UInt16, + data String + ) ENGINE = MergeTree() + PARTITION BY year + ORDER BY id + """) + + node.query(f"INSERT INTO {table_name} VALUES (1, 2020, 'a'), (2, 2020, 'b'), (3, 2021, 'c')") + + node.query(f""" + CREATE TABLE {s3_table} ( + id UInt64, + year UInt16, + data String, + ) ENGINE = S3(s3_conn, filename='disaster-recovery-network', format=Parquet, partition_strategy='hive') + PARTITION BY year + """) + + node2 = cluster.instances["node2"] + node2.query(f""" + CREATE TABLE {s3_table} ( + id UInt64, + year UInt16, + data String, + ) ENGINE = S3(s3_conn, filename='disaster-recovery-network', format=Parquet, partition_strategy='hive') + PARTITION BY year + """) + + with PartitionManager() as pm: + pm.add_network_delay(node, delay_ms=1000) # 5 second delays + + export_queries = f""" + ALTER TABLE {table_name} + EXPORT PARTITION 2020 TO TABLE {s3_table} + SETTINGS allow_experimental_export_merge_tree_partition=1; + ALTER TABLE {table_name} + EXPORT PARTITION 2021 TO TABLE {s3_table} + SETTINGS allow_experimental_export_merge_tree_partition=1; + """ + + node.query(export_queries) + + # Kill server + logging.info("Killing server during network-delayed export") + node.stop_clickhouse(kill=True) + + # check s3 to make sure no data was written + assert node2.query(f"SELECT count() FROM {s3_table} where year = 2020") == '0\n', "Partition 2020 was written to S3 during network delay crash" + + assert node2.query(f"SELECT count() FROM {s3_table} where year = 2021") == '0\n', "Partition 2021 was written to S3 during network delay crash" + + node.start_clickhouse() + + # wait for the export to resume and complete + time.sleep(5) + + # verify that the export has been resumed and completed + 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 = 2020") != f'0\n', "Export of partition 2021 did not resume after crash" + From ff68ba99742291abee5b50b903faac9e8b801f4d Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 2 Sep 2025 09:09:17 -0300 Subject: [PATCH 29/48] small stuff --- src/Storages/StorageMergeTree.cpp | 17 ++++++- .../README.md | 48 ------------------- .../03572_export_mt_part_to_object_storage.sh | 10 ++-- ...mt_part_to_object_storage_simple.reference | 0 ...xport_mt_part_to_object_storage_simple.sql | 29 +++++++++++ 5 files changed, 47 insertions(+), 57 deletions(-) delete mode 100644 tests/integration/test_export_partition_disaster_recovery/README.md create mode 100644 tests/queries/0_stateless/03572_export_mt_part_to_object_storage_simple.reference create mode 100644 tests/queries/0_stateless/03572_export_mt_part_to_object_storage_simple.sql diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 06370851d10c..09d00bab5add 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -119,6 +119,7 @@ namespace ErrorCodes extern const int TABLE_IS_READ_ONLY; extern const int TOO_MANY_PARTS; extern const int PART_IS_LOCKED; + extern const int INCOMPATIBLE_COLUMNS; } namespace ActionLocks @@ -526,6 +527,9 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, 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"); @@ -536,6 +540,7 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, query_context->getSettingsRef()[Setting::lock_acquire_timeout]); auto merges_blocker = stopMergesAndWaitForPartition(partition_id); + /// todo is getVisible the right api? Shouldn't it be get parts for internal usage auto all_parts = getVisibleDataPartsVectorInPartition(getContext(), partition_id); if (all_parts.empty()) @@ -544,7 +549,9 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, return; } - std::lock_guard lock(currently_processing_in_background_mutex); + /// Do not put this in a scope because `CurrentlyExportingPartsTagger` instantiated above relies on this already being locked + /// shitty design I came up with huh + std::lock_guard lock_background_mutex(currently_processing_in_background_mutex); if (!already_exported_partition_ids.emplace(partition_id).second) { @@ -564,7 +571,7 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, exports_tagger->parts_to_export); { - std::lock_guard lock2(export_partition_transaction_id_to_manifest_mutex); + std::lock_guard lock(export_partition_transaction_id_to_manifest_mutex); export_partition_transaction_id_to_manifest[transaction_id] = manifest; } @@ -1162,6 +1169,8 @@ void StorageMergeTree::readExportPartitionManifests() void StorageMergeTree::resumeExportPartitionTasks() { + /// Initially I opted for having two separate methods: read and resume because I wanted to schedule the tasks in order + /// but it turns out the background executor schedules tasks based on their priority, so it is likely this is not needed anymore. for (const auto & [transaction_id, manifest] : export_partition_transaction_id_to_manifest) { auto destination_storage = DatabaseCatalog::instance().tryGetTable(manifest->destination_storage_id, getContext()); @@ -1191,6 +1200,9 @@ void StorageMergeTree::resumeExportPartitionTasks() parts_to_export.emplace_back(part); } + /// TODO: this locks the parts that have not been exported yet. Should we also lock the already exported parts as well? + /// There is some inconsistency with in-progress exports. The parts will not be unlocked until all parts have been exported OR a re-start happens + /// I just checked and mutations handle it slightly different. Tagger will actually contain a single part, which is released as soon as it finishes. auto exports_tagger = std::make_shared(std::move(parts_to_export), *this); auto task = std::make_shared( @@ -2707,6 +2719,7 @@ ActionLock StorageMergeTree::getActionLock(StorageActionBlockType action_type) return {}; } +/// TODO arthur do I need to do something about this? void StorageMergeTree::onActionLockRemove(StorageActionBlockType action_type) { if (action_type == ActionLocks::PartsMerge || action_type == ActionLocks::PartsTTLMerge) diff --git a/tests/integration/test_export_partition_disaster_recovery/README.md b/tests/integration/test_export_partition_disaster_recovery/README.md deleted file mode 100644 index 3a107185a7e3..000000000000 --- a/tests/integration/test_export_partition_disaster_recovery/README.md +++ /dev/null @@ -1,48 +0,0 @@ -# Export Partition Disaster Recovery Tests - -This test suite validates the disaster recovery capabilities of the EXPORT PARTITION operation in ClickHouse. - -## Test Scenarios - -1. **Basic Server Kill** - Tests simple crash during export with recovery -2. **Network Delays** - Tests crash during network-delayed operations -3. **Multiple Crashes** - Tests resilience across multiple crash/restart cycles -4. **Large Datasets** - Tests recovery with substantial data that ensures slow operations -5. **State Persistence** - Tests that recovery state/checkpoints survive crashes -6. **Concurrent Operations** - Tests crashes during multiple simultaneous exports - -## Running Tests - -```bash -# Run all disaster recovery tests -pytest -v test_export_partition_disaster_recovery/ - -# Run specific test -pytest -v test_export_partition_disaster_recovery/test.py::TestExportPartitionDisasterRecovery::test_export_partition_with_server_kill_basic - -# Run with detailed logging -pytest -v -s test_export_partition_disaster_recovery/ -``` - -## Test Infrastructure - -- Uses `BrokenS3` mock to simulate slow S3 operations -- Uses `PartitionManager` for network delay simulation -- Uses `kill -9` to simulate hard server crashes -- Validates data integrity and operation resumability after recovery -- Tests various data sizes and partition configurations - -## Requirements - -- ClickHouse with S3 support -- MinIO integration test environment -- `stay_alive=True` cluster configuration for crash testing - -## Why No Zookeeper? - -These tests focus on **single-node disaster recovery** for export partition operations on plain MergeTree tables. Zookeeper is only needed for: -- ReplicatedMergeTree coordination -- Multi-node distributed operations -- Cross-replica synchronization - -Since we're testing crash recovery of individual export operations, no coordination is required. diff --git a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh index 593f6e5d22f3..7793fbbd9c7b 100755 --- a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh +++ b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh @@ -23,7 +23,7 @@ query "SYSTEM STOP MERGES" query "INSERT INTO $mt_table VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)" query "INSERT INTO $mt_table VALUES (5, 2020), (6, 2020)" -query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_partition = 1, export_merge_tree_partition_background_execution = 0" +query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_partition = 1" echo "---- Querying merge tree for comparison. It should include both partitions (2020 and 2021)" query "SELECT * FROM $mt_table ORDER BY id" @@ -38,7 +38,7 @@ echo "---- Check for commit file for partition 2020" $CLICKHOUSE_CLIENT --query "SELECT replaceRegexpAll(replaceRegexpAll(remote_file_path, '$s3_table', 's3_table_NAME'), '[^/]+\\.parquet', 'SNOWFLAKE_ID.parquet') FROM s3(s3_conn, filename='$s3_table/commit_2020_*', format='LineAsString', structure='remote_file_path String')" echo "---- Finally, export the other partition (2021)" -query "ALTER TABLE $mt_table EXPORT PARTITION ID '2021' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_partition = 1, export_merge_tree_partition_background_execution = 0" +query "ALTER TABLE $mt_table EXPORT PARTITION ID '2021' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_partition = 1" echo "---- Assert both partitions are there" query "SELECT DISTINCT ON (id) replaceRegexpAll(replaceRegexpAll(_path, '$s3_table', 's3_table_NAME'), '[^/]+\\.parquet', 'SNOWFLAKE_ID.parquet'), * FROM $s3_table ORDER BY id" @@ -52,13 +52,9 @@ query "SELECT DISTINCT ON (id) * FROM $mt_table_roundtrip ORDER BY id" # Create a table with a different partition key and export a partition to it. It should throw query "CREATE TABLE $invalid_schema_table (id UInt64, x UInt16) ENGINE = S3(s3_conn, filename='$invalid_schema_table', format=Parquet, partition_strategy='hive') PARTITION BY x" -query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $invalid_schema_table SETTINGS allow_experimental_export_merge_tree_partition = 1, export_merge_tree_partition_background_execution = 0 -- {serverError BAD_ARGUMENTS}" +query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $invalid_schema_table SETTINGS allow_experimental_export_merge_tree_partition = 1 -- {serverError BAD_ARGUMENTS}" query "DROP TABLE $invalid_schema_table" -# The only partition strategy that supports exports is hive. Wildcard should throw -query "CREATE TABLE $invalid_schema_table (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$invalid_schema_table/{_partition_id}', format=Parquet, partition_strategy='wildcard') PARTITION BY (id, year)" -query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $invalid_schema_table SETTINGS allow_experimental_export_merge_tree_partition = 1, export_merge_tree_partition_background_execution = 0 -- {serverError NOT_IMPLEMENTED}" - query "SYSTEM START MERGES" query "DROP TABLE IF EXISTS $mt_table, $s3_table, $mt_table_roundtrip" diff --git a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage_simple.reference b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage_simple.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage_simple.sql b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage_simple.sql new file mode 100644 index 000000000000..151585c5eee0 --- /dev/null +++ b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage_simple.sql @@ -0,0 +1,29 @@ +-- Tags: no-parallel + +DROP TABLE IF EXISTS 03572_mt_table, 03572_invalid_schema_table; + +CREATE TABLE 03572_mt_table (id UInt64, year UInt16) ENGINE = MergeTree() PARTITION BY year ORDER BY tuple(); + +INSERT INTO 03572_mt_table VALUES (1, 2020); + +-- Create a table with a different partition key and export a partition to it. It should throw +CREATE TABLE 03572_invalid_schema_table (id UInt64, x UInt16) +ENGINE = S3( + s3_conn, + filename='03572_invalid_schema_table', + format=Parquet, partition_strategy='hive') PARTITION BY x; + +ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE 03572_invalid_schema_table +SETTINGS allow_experimental_export_merge_tree_partition = 1 -- {serverError BAD_ARGUMENTS} + + +-- The only partition strategy that supports exports is hive. Wildcard should throw +CREATE TABLE 03572_invalid_schema_table (id UInt64, year UInt16) +ENGINE = S3( + s3_conn, + filename='03572_invalid_schema_table/{_partition_id}', + format=Parquet, + partition_strategy='wildcard') PARTITION BY (id, year); + +ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE 03572_invalid_schema_table +SETTINGS allow_experimental_export_merge_tree_partition = 1 -- {serverError NOT_IMPLEMENTED} From 44c697c34709b2dd43fdf98a8738b7653fc3467b Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 2 Sep 2025 11:57:51 -0300 Subject: [PATCH 30/48] fix test --- .../0_stateless/03572_export_mt_part_to_object_storage.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh index 7793fbbd9c7b..3b84c83b72f5 100755 --- a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh +++ b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh @@ -52,7 +52,7 @@ query "SELECT DISTINCT ON (id) * FROM $mt_table_roundtrip ORDER BY id" # Create a table with a different partition key and export a partition to it. It should throw query "CREATE TABLE $invalid_schema_table (id UInt64, x UInt16) ENGINE = S3(s3_conn, filename='$invalid_schema_table', format=Parquet, partition_strategy='hive') PARTITION BY x" -query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $invalid_schema_table SETTINGS allow_experimental_export_merge_tree_partition = 1 -- {serverError BAD_ARGUMENTS}" +query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $invalid_schema_table SETTINGS allow_experimental_export_merge_tree_partition = 1 -- {serverError INCOMPATIBLE_COLUMNS}" query "DROP TABLE $invalid_schema_table" From 8f171b89cef0ebadbe203eb459783b9bd6eabdcd Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Tue, 2 Sep 2025 13:51:08 -0300 Subject: [PATCH 31/48] fiox tests --- .../03572_export_mt_part_to_object_storage.sh | 9 +------ ...xport_mt_part_to_object_storage_simple.sql | 24 +++++++------------ 2 files changed, 9 insertions(+), 24 deletions(-) diff --git a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh index 3b84c83b72f5..405faf2ddc98 100755 --- a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh +++ b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage.sh @@ -7,13 +7,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) mt_table="mt_table_${RANDOM}" s3_table="s3_table_${RANDOM}" mt_table_roundtrip="mt_table_roundtrip_${RANDOM}" -invalid_schema_table="invalid_schema_table_${RANDOM}" query() { $CLICKHOUSE_CLIENT --query "$1" } -query "DROP TABLE IF EXISTS $mt_table, $s3_table, $mt_table_roundtrip, $invalid_schema_table" +query "DROP TABLE IF EXISTS $mt_table, $s3_table, $mt_table_roundtrip" query "CREATE TABLE $mt_table (id UInt64, year UInt16) ENGINE = MergeTree() 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" @@ -50,11 +49,5 @@ query "CREATE TABLE $mt_table_roundtrip ENGINE = MergeTree() PARTITION BY year O echo "---- Data in roundtrip MergeTree table (should match s3_table)" query "SELECT DISTINCT ON (id) * FROM $mt_table_roundtrip ORDER BY id" -# Create a table with a different partition key and export a partition to it. It should throw -query "CREATE TABLE $invalid_schema_table (id UInt64, x UInt16) ENGINE = S3(s3_conn, filename='$invalid_schema_table', format=Parquet, partition_strategy='hive') PARTITION BY x" -query "ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE $invalid_schema_table SETTINGS allow_experimental_export_merge_tree_partition = 1 -- {serverError INCOMPATIBLE_COLUMNS}" - -query "DROP TABLE $invalid_schema_table" - query "SYSTEM START MERGES" query "DROP TABLE IF EXISTS $mt_table, $s3_table, $mt_table_roundtrip" diff --git a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage_simple.sql b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage_simple.sql index 151585c5eee0..7c84ff608c54 100644 --- a/tests/queries/0_stateless/03572_export_mt_part_to_object_storage_simple.sql +++ b/tests/queries/0_stateless/03572_export_mt_part_to_object_storage_simple.sql @@ -7,23 +7,15 @@ CREATE TABLE 03572_mt_table (id UInt64, year UInt16) ENGINE = MergeTree() PARTIT INSERT INTO 03572_mt_table VALUES (1, 2020); -- Create a table with a different partition key and export a partition to it. It should throw -CREATE TABLE 03572_invalid_schema_table (id UInt64, x UInt16) -ENGINE = S3( - s3_conn, - filename='03572_invalid_schema_table', - format=Parquet, partition_strategy='hive') PARTITION BY x; +CREATE TABLE 03572_invalid_schema_table (id UInt64, x UInt16) ENGINE = S3(s3_conn, filename='03572_invalid_schema_table', format='Parquet', partition_strategy='hive') PARTITION BY x; -ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE 03572_invalid_schema_table -SETTINGS allow_experimental_export_merge_tree_partition = 1 -- {serverError BAD_ARGUMENTS} +ALTER TABLE 03572_mt_table EXPORT PARTITION ID '2020' TO TABLE 03572_invalid_schema_table +SETTINGS allow_experimental_export_merge_tree_partition = 1; -- {serverError INCOMPATIBLE_COLUMNS} +DROP TABLE 03572_invalid_schema_table; -- The only partition strategy that supports exports is hive. Wildcard should throw -CREATE TABLE 03572_invalid_schema_table (id UInt64, year UInt16) -ENGINE = S3( - s3_conn, - filename='03572_invalid_schema_table/{_partition_id}', - format=Parquet, - partition_strategy='wildcard') PARTITION BY (id, year); - -ALTER TABLE $mt_table EXPORT PARTITION ID '2020' TO TABLE 03572_invalid_schema_table -SETTINGS allow_experimental_export_merge_tree_partition = 1 -- {serverError NOT_IMPLEMENTED} +CREATE TABLE 03572_invalid_schema_table (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='03572_invalid_schema_table/{_partition_id}', format='Parquet', partition_strategy='wildcard') PARTITION BY (id, year); + +ALTER TABLE 03572_mt_table EXPORT PARTITION ID '2020' TO TABLE 03572_invalid_schema_table +SETTINGS allow_experimental_export_merge_tree_partition = 1; -- {serverError NOT_IMPLEMENTED} From 8a51270078ab8cc9d896f08e67140365938a231b Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 3 Sep 2025 09:46:57 -0300 Subject: [PATCH 32/48] implement single part task --- src/Core/Settings.cpp | 4 +- src/Storages/IStorage.h | 7 + src/Storages/MergeTree/MergeTreeData.h | 1 + .../MergeTree/MergeTreeExportManifest.h | 5 + .../ExportPartPlainMergeTreeTask.cpp | 198 ++++++++++++++++++ .../MergeTree/ExportPartPlainMergeTreeTask.h | 69 ++++++ .../ExportPartitionPlainMergeTreeTask.cpp | 8 +- .../ObjectStorage/StorageObjectStorage.cpp | 84 ++++++++ .../ObjectStorage/StorageObjectStorage.h | 8 + src/Storages/StorageMergeTree.cpp | 77 ++++--- src/Storages/StorageMergeTree.h | 1 + 11 files changed, 435 insertions(+), 27 deletions(-) create mode 100644 src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp create mode 100644 src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index db3cfafe702f..4f621cc2f446 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -6877,8 +6877,8 @@ Experimental timeSeries* aggregate functions for Prometheus-like timeseries resa DECLARE_WITH_ALIAS(Bool, allow_experimental_export_merge_tree_partition, false, R"( Experimental export merge tree partition. )", EXPERIMENTAL, allow_experimental_export_merge_tree_partition) \ - DECLARE_WITH_ALIAS(Bool, export_merge_tree_partition_background_execution, true, R"( - Process exports asynchronously in background threads + DECLARE_WITH_ALIAS(Bool, export_merge_tree_partition_individual_part_executor, true, R"( + Use the part task instead of the partition task )", EXPERIMENTAL, export_merge_tree_partition_background_execution) \ \ /* ####################################################### */ \ diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 8998a6ba69ac..1ecb7ddb9510 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -462,6 +462,13 @@ class IStorage : public std::enable_shared_from_this, public TypePromo ContextPtr /*context*/, std::function) {} + virtual void importMergeTreePart( + const MergeTreeData &, + const DataPartPtr &, + ContextPtr, + std::function + ) {} + /** Writes the data to a table in distributed manner. * It is supposed that implementation looks into SELECT part of the query and executes distributed * INSERT SELECT if it is possible with current storage as a receiver and query SELECT part as a producer. diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 53ebe9dfda45..0c6438f29524 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1233,6 +1233,7 @@ class MergeTreeData : public IStorage, public WithMutableContext friend class IPartMetadataManager; friend class IMergedBlockOutputStream; // for access to log friend class ExportPartitionPlainMergeTreeTask; + friend class ExportPartPlainMergeTreeTask; bool require_part_metadata; diff --git a/src/Storages/MergeTree/MergeTreeExportManifest.h b/src/Storages/MergeTree/MergeTreeExportManifest.h index c546daf58e69..e4c168bd0560 100644 --- a/src/Storages/MergeTree/MergeTreeExportManifest.h +++ b/src/Storages/MergeTree/MergeTreeExportManifest.h @@ -138,6 +138,11 @@ struct MergeTreeExportManifest out->sync(); } + void deleteFile() + { + disk->removeFile(file_path); + } + void updateRemotePathAndWrite(const String & part_name, const String & remote_path) { for (auto & i : items) diff --git a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp new file mode 100644 index 000000000000..c5861e4f853b --- /dev/null +++ b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp @@ -0,0 +1,198 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +ExportPartPlainMergeTreeTask::ExportPartPlainMergeTreeTask( + StorageMergeTree & storage_, + const std::shared_ptr & exports_tagger_, + const StoragePtr & destination_storage_, + ContextPtr context_, + std::shared_ptr manifest_, + IExecutableTask::TaskResultCallback & task_result_callback_, + size_t max_retries_) + : storage(storage_) + , exports_tagger(exports_tagger_) + , destination_storage(destination_storage_) + , context(std::move(context_)) + , manifest(std::move(manifest_)) + , task_result_callback(task_result_callback_) + , max_retries(max_retries_) +{ + UInt64 transaction_id = std::stoull(manifest->transaction_id); + priority.value = transaction_id; +} + +StorageID ExportPartPlainMergeTreeTask::getStorageID() const +{ + return storage.getStorageID(); +} + +String ExportPartPlainMergeTreeTask::getQueryId() const +{ + return getStorageID().getShortName() + "::export_partition::" + manifest->transaction_id; +} + +bool ExportPartPlainMergeTreeTask::executeStep() +{ + if (cancelled) + return false; + + switch (state) + { + case State::NEED_PREPARE: + { + prepare(); + state = State::NEED_EXECUTE; + return true; + } + case State::NEED_EXECUTE: + { + if (executeExport()) + { + state = State::NEED_COMMIT; + return true; + } + + if (retry_count < max_retries) + { + retry_count++; + LOG_INFO(getLogger("ExportPartPlainMergeTreeTask"), + "Retrying export attempt {} for part {}", + retry_count, exports_tagger->parts_to_export[0]->name); + state = State::NEED_EXECUTE; + + return true; + } + + return false; + } + case State::NEED_COMMIT: + { + if (commitExport()) + { + state = State::SUCCESS; + return true; + } + + if (retry_count < max_retries) + { + retry_count++; + LOG_INFO(getLogger("ExportPartPlainMergeTreeTask"), + "Retrying export attempt {} for part {}", + retry_count, exports_tagger->parts_to_export[0]->name); + state = State::NEED_COMMIT; + + return true; + } + + return false; + } + case State::SUCCESS: + { + return false; + } + } + + return false; +} + + +void ExportPartPlainMergeTreeTask::prepare() +{ + stopwatch_ptr = std::make_unique(); +} + +bool ExportPartPlainMergeTreeTask::executeExport() +{ + if (cancelled) + return false; + + std::function part_log_wrapper = [this](MergeTreePartImportStats stats) { + + std::lock_guard lock(storage.export_partition_transaction_id_to_manifest_mutex); + auto table_id = storage.getStorageID(); + + if (stats.status.code != 0) + { + LOG_INFO(getLogger("ExportMergeTreePartitionToObjectStorageTask"), "Error importing part {}: {}", stats.part->name, stats.status.message); + return; + } + + storage.export_partition_transaction_id_to_manifest[manifest->transaction_id]->updateRemotePathAndWrite( + stats.part->name, + stats.file_path); + + UInt64 elapsed_ns = stopwatch_ptr->elapsedNanoseconds(); + + storage.writePartLog( + PartLogElement::Type::EXPORT_PART, + stats.status, + elapsed_ns, + stats.part->name, + stats.part, + {stats.part}, + nullptr, + nullptr); + }; + + try + { + destination_storage->importMergeTreePart( + storage, + exports_tagger->parts_to_export[0], + context, + part_log_wrapper); + + return true; + } + catch (...) + { + LOG_ERROR(getLogger("ExportPartPlainMergeTreeTask"), "Failed to export part {}", exports_tagger->parts_to_export[0]->name); + + return false; + } +} + +bool ExportPartPlainMergeTreeTask::commitExport() +{ + std::lock_guard lock(storage.export_partition_transaction_id_to_manifest_mutex); + + if (manifest->exportedPaths().size() == manifest->items.size()) + { + destination_storage->commitExportPartitionTransaction( + manifest->transaction_id, + manifest->partition_id, + manifest->exportedPaths(), + context); + manifest->completed = true; + manifest->write(); + storage.export_partition_transaction_id_to_manifest.erase(manifest->transaction_id); + LOG_INFO(getLogger("ExportMergeTreePartitionToObjectStorageTask"), + "Successfully committed export transaction {} for partition {}", + manifest->transaction_id, manifest->partition_id); + } + + LOG_INFO(getLogger("ExportPartPlainMergeTreeTask"), "Not all parts have been exported yet for transaction id {}, not comitting for this part", manifest->transaction_id); + + return true; +} + +void ExportPartPlainMergeTreeTask::onCompleted() +{ + bool success = (state == State::SUCCESS); + task_result_callback(success); +} + +void ExportPartPlainMergeTreeTask::cancel() noexcept +{ + cancelled = true; +} + +} diff --git a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h new file mode 100644 index 000000000000..1bc0d3deb9b5 --- /dev/null +++ b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h @@ -0,0 +1,69 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class MergeTreeData; + +class ExportPartPlainMergeTreeTask : public IExecutableTask +{ +public: + ExportPartPlainMergeTreeTask( + StorageMergeTree & storage_, + const std::shared_ptr & exports_tagger_, + const StoragePtr & destination_storage_, + ContextPtr context_, + std::shared_ptr manifest_, + IExecutableTask::TaskResultCallback & task_result_callback_, + size_t max_retries_ = 3); + + void onCompleted() override; + bool executeStep() override; + void cancel() noexcept override; + StorageID getStorageID() const override; + Priority getPriority() const override { return priority; } + String getQueryId() const override; + +private: + void prepare(); + bool executeExport(); + bool commitExport(); + bool exportedAllIndividualParts() const; + + enum class State : uint8_t + { + NEED_PREPARE, + NEED_EXECUTE, + NEED_COMMIT, + SUCCESS + }; + + State state{State::NEED_PREPARE}; + + StorageMergeTree & storage; + std::shared_ptr exports_tagger; + StoragePtr destination_storage; + ContextPtr context; + std::shared_ptr manifest; + IExecutableTask::TaskResultCallback task_result_callback; + + size_t max_retries; + size_t retry_count = 0; + Priority priority; + std::unique_ptr stopwatch_ptr; + + bool cancelled = false; + std::exception_ptr current_exception; +}; + +using ExportPartPlainMergeTreeTaskPtr = std::shared_ptr; + +} diff --git a/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.cpp b/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.cpp index 3b668c30f1d9..57b62336da81 100644 --- a/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.cpp +++ b/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.cpp @@ -168,7 +168,13 @@ void ExportPartitionPlainMergeTreeTask::executeExport() { LOG_ERROR(getLogger("ExportMergeTreePartitionToObjectStorageTask"), "Export attempt failed completely: {}", getCurrentExceptionMessage(true)); - + + std::lock_guard lock(storage.export_partition_transaction_id_to_manifest_mutex); + manifest->deleteFile(); + storage.already_exported_partition_ids.erase(manifest->partition_id); + storage.export_partition_transaction_id_to_manifest.erase(manifest->transaction_id); + + throw; } } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index c394ab80e87c..f29f1089d046 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -553,6 +553,90 @@ bool StorageObjectStorage::supportsImportMergeTreePartition() const return configuration->partition_strategy != nullptr && configuration->partition_strategy_type == PartitionStrategyFactory::StrategyType::HIVE; } +void StorageObjectStorage::importMergeTreePart( + const MergeTreeData & merge_tree_data, + const DataPartPtr & data_part, + ContextPtr local_context, + std::function part_log +) +{ + auto metadata_snapshot = merge_tree_data.getInMemoryMetadataPtr(); + Names columns_to_read = metadata_snapshot->getColumns().getNamesOfPhysical(); + StorageSnapshotPtr storage_snapshot = merge_tree_data.getStorageSnapshot(metadata_snapshot, local_context); + + QueryPlan plan; + + /// using the mutations type for now + MergeTreeSequentialSourceType read_type = MergeTreeSequentialSourceType::Mutation; + + bool apply_deleted_mask = true; + bool read_with_direct_io = false; + bool prefetch = false; + + const auto partition_columns = configuration->partition_strategy->getPartitionColumns(); + + auto block_with_partition_values = data_part->partition.getBlockWithPartitionValues(partition_columns); + + const auto column_with_partition_key = configuration->partition_strategy->computePartitionKey(block_with_partition_values); + + const auto file_path = configuration->file_path_generator->getWritingPath(column_with_partition_key->getDataAt(0).toString()); + + MergeTreeData::IMutationsSnapshot::Params params + { + .metadata_version = metadata_snapshot->getMetadataVersion(), + .min_part_metadata_version = data_part->getMetadataVersion(), + }; + + auto mutations_snapshot = merge_tree_data.getMutationsSnapshot(params); + + auto alter_conversions = MergeTreeData::getAlterConversionsForPart( + data_part, + mutations_snapshot, + local_context); + + QueryPlan plan_for_part; + + createReadFromPartStep( + read_type, + plan_for_part, + merge_tree_data, + storage_snapshot, + RangesInDataPart(data_part), + alter_conversions, + nullptr, + columns_to_read, + nullptr, + apply_deleted_mask, + std::nullopt, + read_with_direct_io, + prefetch, + local_context, + getLogger("ExportPartition")); + + QueryPlanOptimizationSettings optimization_settings(local_context); + auto pipeline_settings = BuildQueryPipelineSettings(local_context); + auto builder = plan_for_part.buildQueryPipeline(optimization_settings, pipeline_settings); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + + auto sink = std::make_shared( + data_part, + file_path, + object_storage, + configuration, + format_settings, + metadata_snapshot->getSampleBlock(), + part_log, + local_context + ); + + pipeline.complete(sink); + + pipeline.setNumThreads(local_context->getSettingsRef()[Setting::max_threads]); + + CompletedPipelineExecutor exec(pipeline); + exec.execute(); +} + void StorageObjectStorage::importMergeTreePartition( const MergeTreeData & merge_tree_data, const std::vector & data_parts, diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 511d3d79c9e2..aaf92a41d399 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -7,6 +7,7 @@ #include #include #include +#include "Storages/MergeTree/RangesInDataPart.h" #include #include #include @@ -106,6 +107,13 @@ class StorageObjectStorage : public IStorage ContextPtr /*context*/, std::function part_log) override; + void importMergeTreePart( + const MergeTreeData & merge_tree_data, + const DataPartPtr & data_part, + ContextPtr, + std::function part_log + ) override; + /// Write an export commit file containing the list of exported remote paths void commitExportPartitionTransaction(const String & transaction_id, const String & partition_id, const Strings & exported_paths, ContextPtr local_context) override; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 09d00bab5add..95de5f87d287 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -47,6 +47,7 @@ #include #include #include "Core/BackgroundSchedulePool.h" +#include "Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h" #include #include #include @@ -80,8 +81,8 @@ namespace Setting extern const SettingsBool parallel_replicas_for_non_replicated_merge_tree; extern const SettingsBool throw_on_unsupported_query_inside_transaction; extern const SettingsUInt64 max_parts_to_move; - extern const SettingsBool export_merge_tree_partition_background_execution; extern const SettingsBool allow_experimental_export_merge_tree_partition; + extern const SettingsBool export_merge_tree_partition_individual_part_executor; } namespace MergeTreeSetting @@ -558,17 +559,18 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, throw Exception(ErrorCodes::PART_IS_LOCKED, "Partition {} has already been exported", partition_id); } - auto exports_tagger = std::make_shared(std::move(all_parts), *this); + const auto transaction_id = std::to_string(generateSnowflakeID()); + /// TODO missing parts lock here with tagger const auto manifest = MergeTreeExportManifest::create( getStoragePolicy()->getAnyDisk(), relative_data_path, transaction_id, partition_id, dest_storage->getStorageID(), - exports_tagger->parts_to_export); + all_parts); { std::lock_guard lock(export_partition_transaction_id_to_manifest_mutex); @@ -576,15 +578,29 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, export_partition_transaction_id_to_manifest[transaction_id] = manifest; } - auto task = std::make_shared( - *this, - exports_tagger, - dest_storage, - getContext(), - manifest, - moves_assignee_trigger); + if (getContext()->getSettingsRef()[Setting::export_merge_tree_partition_individual_part_executor]) + { + for (const auto & part : all_parts) + { + auto tagger = std::make_shared(std::vector{part}, *this); + auto task = std::make_shared(*this, tagger, dest_storage, getContext(), manifest, moves_assignee_trigger); + background_moves_assignee.scheduleMoveTask(task); + } + } + else + { + auto exports_tagger = std::make_shared(std::move(all_parts), *this); - background_moves_assignee.scheduleMoveTask(task); + auto task = std::make_shared( + *this, + exports_tagger, + dest_storage, + getContext(), + manifest, + moves_assignee_trigger); + + background_moves_assignee.scheduleMoveTask(task); + } } /// While exists, marks parts as 'currently_merging_mutating_parts' and reserves free space on filesystem. @@ -1200,20 +1216,33 @@ void StorageMergeTree::resumeExportPartitionTasks() parts_to_export.emplace_back(part); } - /// TODO: this locks the parts that have not been exported yet. Should we also lock the already exported parts as well? - /// There is some inconsistency with in-progress exports. The parts will not be unlocked until all parts have been exported OR a re-start happens - /// I just checked and mutations handle it slightly different. Tagger will actually contain a single part, which is released as soon as it finishes. - auto exports_tagger = std::make_shared(std::move(parts_to_export), *this); - - auto task = std::make_shared( - *this, - exports_tagger, - destination_storage, - getContext(), - manifest, - moves_assignee_trigger); + if (getContext()->getSettingsRef()[Setting::export_merge_tree_partition_individual_part_executor]) + { + for (const auto & part : parts_to_export) + { + auto tagger = std::make_shared(std::vector{part}, *this); + auto task = std::make_shared(*this, tagger, destination_storage, getContext(), manifest, moves_assignee_trigger); + background_moves_assignee.scheduleMoveTask(task); + } + } + else + { + /// TODO: this locks the parts that have not been exported yet. Should we also lock the already exported parts as well? + /// There is some inconsistency with in-progress exports. The parts will not be unlocked until all parts have been exported OR a re-start happens + /// I just checked and mutations handle it slightly different. Tagger will actually contain a single part, which is released as soon as it finishes. + auto exports_tagger = std::make_shared(std::move(parts_to_export), *this); + + auto task = std::make_shared( + *this, + exports_tagger, + destination_storage, + getContext(), + manifest, + moves_assignee_trigger); + + background_moves_assignee.scheduleMoveTask(task); + } - background_moves_assignee.scheduleMoveTask(task); } } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index f54679bdde33..ceb3f347f361 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -308,6 +308,7 @@ class StorageMergeTree final : public MergeTreeData friend class MergePlainMergeTreeTask; friend class MutatePlainMergeTreeTask; friend class ExportPartitionPlainMergeTreeTask; + friend class ExportPartPlainMergeTreeTask; struct DataValidationTasks : public IStorage::DataValidationTasksBase { From af3352b71d7c83a76322c1194a9f666c8295f45b Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 3 Sep 2025 11:37:10 -0300 Subject: [PATCH 33/48] fix privileges test --- tests/queries/0_stateless/01271_show_privileges.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 268959bb444c..56f2e76ed3a1 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -43,6 +43,7 @@ ALTER TTL ['ALTER MODIFY TTL','MODIFY TTL'] TABLE ALTER TABLE 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 PARTITION ['ALTER EXPORT PART','EXPORT PARTITION','EXPORT PART'] 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 From 502b5018d5b318f579e90d4ba730cba881647a87 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 3 Sep 2025 13:29:20 -0300 Subject: [PATCH 34/48] improve system.exports, show failed exports --- src/Core/Settings.cpp | 4 +- src/Storages/MergeTree/MergeTreeData.h | 2 + .../MergeTree/MergeTreeExportManifest.h | 28 +++- .../MergeTree/MergeTreeExportStatus.cpp | 0 .../MergeTree/MergeTreeExportStatus.h | 25 ++++ .../ExportPartPlainMergeTreeTask.cpp | 35 +++-- .../MergeTree/ExportPartPlainMergeTreeTask.h | 1 + .../ExportPartitionPlainMergeTreeTask.cpp | 4 +- src/Storages/StorageMergeTree.cpp | 61 ++++++--- src/Storages/StorageMergeTree.h | 4 +- src/Storages/System/StorageSystemExports.cpp | 123 +++++++++++++++--- 11 files changed, 231 insertions(+), 56 deletions(-) create mode 100644 src/Storages/MergeTree/MergeTreeExportStatus.cpp create mode 100644 src/Storages/MergeTree/MergeTreeExportStatus.h diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 4f621cc2f446..f02fbf61726e 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -6877,9 +6877,9 @@ Experimental timeSeries* aggregate functions for Prometheus-like timeseries resa DECLARE_WITH_ALIAS(Bool, allow_experimental_export_merge_tree_partition, false, R"( Experimental export merge tree partition. )", EXPERIMENTAL, allow_experimental_export_merge_tree_partition) \ - DECLARE_WITH_ALIAS(Bool, export_merge_tree_partition_individual_part_executor, true, R"( + DECLARE_WITH_ALIAS(Bool, export_merge_tree_partition_executor, false, R"( Use the part task instead of the partition task -)", EXPERIMENTAL, export_merge_tree_partition_background_execution) \ +)", EXPERIMENTAL, export_merge_tree_partition_executor) \ \ /* ####################################################### */ \ /* ############ END OF EXPERIMENTAL FEATURES ############# */ \ diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 0c6438f29524..1f46fd1b50b9 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -966,6 +967,7 @@ class MergeTreeData : public IStorage, public WithMutableContext bool must_on_same_disk); virtual std::vector getMutationsStatus() const = 0; + virtual std::vector getExportsStatus() const { return {}; } /// Returns true if table can create new parts with adaptive granularity /// Has additional constraint in replicated version diff --git a/src/Storages/MergeTree/MergeTreeExportManifest.h b/src/Storages/MergeTree/MergeTreeExportManifest.h index e4c168bd0560..c1aec96cdbcb 100644 --- a/src/Storages/MergeTree/MergeTreeExportManifest.h +++ b/src/Storages/MergeTree/MergeTreeExportManifest.h @@ -14,6 +14,8 @@ #include #include #include +#include +#include namespace DB { @@ -25,16 +27,24 @@ namespace DB * "transaction_id": "", * "partition_id": "", * "destination": ".
", - * "completed": , + * "create_time": , + * "status": "", * "parts": [ {"part_name": "name", "remote_path": "path-or-empty"}, ... ] * } */ struct MergeTreeExportManifest { using DataPartPtr = std::shared_ptr; + + enum class Status { + pending, + completed, + failed + }; MergeTreeExportManifest() : destination_storage_id(StorageID::createEmpty()) + , status(Status::pending) {} struct Item @@ -42,12 +52,14 @@ struct MergeTreeExportManifest String part_name; String remote_path; // empty until uploaded }; + String transaction_id; String partition_id; StorageID destination_storage_id; + time_t create_time = 0; std::vector items; - bool completed = false; + Status status = Status::pending; std::filesystem::path file_path; DiskPtr disk; @@ -65,6 +77,7 @@ struct MergeTreeExportManifest manifest->transaction_id = transaction_id_; manifest->partition_id = partition_id_; manifest->destination_storage_id = destination_storage_id_; + manifest->create_time = std::time(nullptr); manifest->file_path = std::filesystem::path(path_prefix) / ("export_partition_" + partition_id_ + "_transaction_" + transaction_id_ + ".json"); manifest->items.reserve(data_parts.size()); for (const auto & data_part : data_parts) @@ -94,7 +107,11 @@ struct MergeTreeExportManifest manifest->partition_id = root->getValue("partition_id"); const auto destination = root->getValue("destination"); manifest->destination_storage_id = StorageID(QualifiedTableName::parseFromString(destination)); - manifest->completed = root->getValue("completed"); + + manifest->create_time = root->getValue("create_time"); + + String status_str = root->getValue("status"); + manifest->status = magic_enum::enum_cast(status_str).value(); manifest->items.clear(); auto parts = root->get("parts").extract(); @@ -118,7 +135,8 @@ struct MergeTreeExportManifest root->set("transaction_id", transaction_id); root->set("partition_id", partition_id); root->set("destination", destination_storage_id.getQualifiedName().getFullName()); - root->set("completed", completed); + root->set("create_time", static_cast(create_time)); + root->set("status", String(magic_enum::enum_name(status))); Poco::JSON::Array::Ptr parts(new Poco::JSON::Array()); for (const auto & i : items) @@ -138,7 +156,7 @@ struct MergeTreeExportManifest out->sync(); } - void deleteFile() + void deleteFile() const { disk->removeFile(file_path); } diff --git a/src/Storages/MergeTree/MergeTreeExportStatus.cpp b/src/Storages/MergeTree/MergeTreeExportStatus.cpp new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/src/Storages/MergeTree/MergeTreeExportStatus.h b/src/Storages/MergeTree/MergeTreeExportStatus.h new file mode 100644 index 000000000000..7f6905c2f1a4 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeExportStatus.h @@ -0,0 +1,25 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +struct MergeTreeExportStatus +{ + String source_database; + String source_table; + String destination_database; + String destination_table; + String transaction_id; + time_t create_time = 0; + std::vector parts_to_do_names; + MergeTreeExportManifest::Status status; +}; + +} + diff --git a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp index c5861e4f853b..93aa1abcff2e 100644 --- a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp +++ b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp @@ -57,40 +57,51 @@ bool ExportPartPlainMergeTreeTask::executeStep() if (executeExport()) { state = State::NEED_COMMIT; - return true; } - - if (retry_count < max_retries) + else if (retry_count < max_retries) { retry_count++; LOG_INFO(getLogger("ExportPartPlainMergeTreeTask"), "Retrying export attempt {} for part {}", retry_count, exports_tagger->parts_to_export[0]->name); state = State::NEED_EXECUTE; - - return true; + } + else + { + state = State::FAILED; } - return false; + return true; } case State::NEED_COMMIT: { if (commitExport()) { state = State::SUCCESS; - return true; } - - if (retry_count < max_retries) + else if (retry_count < max_retries) { retry_count++; LOG_INFO(getLogger("ExportPartPlainMergeTreeTask"), "Retrying export attempt {} for part {}", retry_count, exports_tagger->parts_to_export[0]->name); state = State::NEED_COMMIT; - - return true; } + else + { + state = State::FAILED; + } + + return true; + } + case State::FAILED: + { + std::lock_guard lock(storage.export_partition_transaction_id_to_manifest_mutex); + + manifest->status = MergeTreeExportManifest::Status::failed; + manifest->write(); + + storage.already_exported_partition_ids.erase(manifest->partition_id); return false; } @@ -171,7 +182,7 @@ bool ExportPartPlainMergeTreeTask::commitExport() manifest->partition_id, manifest->exportedPaths(), context); - manifest->completed = true; + manifest->status = MergeTreeExportManifest::Status::completed; manifest->write(); storage.export_partition_transaction_id_to_manifest.erase(manifest->transaction_id); LOG_INFO(getLogger("ExportMergeTreePartitionToObjectStorageTask"), diff --git a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h index 1bc0d3deb9b5..9c694b8fce6c 100644 --- a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h +++ b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h @@ -43,6 +43,7 @@ class ExportPartPlainMergeTreeTask : public IExecutableTask NEED_PREPARE, NEED_EXECUTE, NEED_COMMIT, + FAILED, SUCCESS }; diff --git a/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.cpp b/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.cpp index 57b62336da81..30bbfd5785a9 100644 --- a/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.cpp +++ b/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.cpp @@ -189,7 +189,7 @@ bool ExportPartitionPlainMergeTreeTask::commitExport() manifest->exportedPaths(), context); - manifest->completed = true; + manifest->status = MergeTreeExportManifest::Status::completed; manifest->write(); storage.export_partition_transaction_id_to_manifest.erase(manifest->transaction_id); @@ -204,7 +204,7 @@ bool ExportPartitionPlainMergeTreeTask::commitExport() void ExportPartitionPlainMergeTreeTask::onCompleted() { std::lock_guard lock(storage.export_partition_transaction_id_to_manifest_mutex); - task_result_callback(manifest->completed); + task_result_callback(manifest->status == MergeTreeExportManifest::Status::completed); } void ExportPartitionPlainMergeTreeTask::cancel() noexcept diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 95de5f87d287..ec216d2768b7 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -82,7 +82,7 @@ namespace Setting extern const SettingsBool throw_on_unsupported_query_inside_transaction; extern const SettingsUInt64 max_parts_to_move; extern const SettingsBool allow_experimental_export_merge_tree_partition; - extern const SettingsBool export_merge_tree_partition_individual_part_executor; + extern const SettingsBool export_merge_tree_partition_executor; } namespace MergeTreeSetting @@ -559,8 +559,6 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, throw Exception(ErrorCodes::PART_IS_LOCKED, "Partition {} has already been exported", partition_id); } - - const auto transaction_id = std::to_string(generateSnowflakeID()); /// TODO missing parts lock here with tagger @@ -578,7 +576,7 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, export_partition_transaction_id_to_manifest[transaction_id] = manifest; } - if (getContext()->getSettingsRef()[Setting::export_merge_tree_partition_individual_part_executor]) + if (!getContext()->getSettingsRef()[Setting::export_merge_tree_partition_executor]) { for (const auto & part : all_parts) { @@ -981,6 +979,33 @@ std::map StorageMergeTree::getUnfinishedMutationC return result; } + +std::vector StorageMergeTree::getExportsStatus() const +{ + std::lock_guard lock(export_partition_transaction_id_to_manifest_mutex); + std::vector result; + + auto source_database = getStorageID().database_name; + auto source_table = getStorageID().table_name; + + for (const auto & [transaction_id, manifest] : export_partition_transaction_id_to_manifest) + { + MergeTreeExportStatus status; + + status.transaction_id = transaction_id; + status.source_database = source_database; + status.source_table = source_table; + status.destination_database = manifest->destination_storage_id.database_name; + status.destination_table = manifest->destination_storage_id.table_name; + status.create_time = 0; + status.parts_to_do_names = manifest->pendingParts(); + status.status = manifest->status; + + result.emplace_back(std::move(status)); + } + return result; +} + std::vector StorageMergeTree::getMutationsStatus() const { std::lock_guard lock(currently_processing_in_background_mutex); @@ -1157,17 +1182,20 @@ void StorageMergeTree::readExportPartitionManifests() { auto manifest = MergeTreeExportManifest::read(disk, fs::path(relative_data_path) / name); - already_exported_partition_ids.insert(manifest->partition_id); - - if (manifest->completed) + if (manifest->status != MergeTreeExportManifest::Status::failed) { - LOG_INFO( - log, - "Export transaction {} of partition {} to destination storage {} already completed, skipping", - manifest->transaction_id, - manifest->partition_id, - manifest->destination_storage_id.getNameForLogs()); - continue; + already_exported_partition_ids.insert(manifest->partition_id); + + if (manifest->status == MergeTreeExportManifest::Status::completed) + { + LOG_INFO( + log, + "Export transaction {} of partition {} to destination storage {} already completed, skipping", + manifest->transaction_id, + manifest->partition_id, + manifest->destination_storage_id.getNameForLogs()); + continue; + } } export_partition_transaction_id_to_manifest.emplace(manifest->transaction_id, manifest); @@ -1189,6 +1217,9 @@ void StorageMergeTree::resumeExportPartitionTasks() /// but it turns out the background executor schedules tasks based on their priority, so it is likely this is not needed anymore. for (const auto & [transaction_id, manifest] : export_partition_transaction_id_to_manifest) { + if (manifest->status != MergeTreeExportManifest::Status::pending) + continue; + auto destination_storage = DatabaseCatalog::instance().tryGetTable(manifest->destination_storage_id, getContext()); if (!destination_storage) { @@ -1216,7 +1247,7 @@ void StorageMergeTree::resumeExportPartitionTasks() parts_to_export.emplace_back(part); } - if (getContext()->getSettingsRef()[Setting::export_merge_tree_partition_individual_part_executor]) + if (!getContext()->getSettingsRef()[Setting::export_merge_tree_partition_executor]) { for (const auto & part : parts_to_export) { diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index ceb3f347f361..a23f43849453 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -96,6 +96,8 @@ class StorageMergeTree final : public MergeTreeData /// Return introspection information about currently processing or recently processed mutations. std::vector getMutationsStatus() const override; + std::vector getExportsStatus() const override; + CancellationCode killMutation(const String & mutation_id) override; /// Makes backup entries to backup the data of the storage. @@ -160,7 +162,7 @@ class StorageMergeTree final : public MergeTreeData std::map current_mutations_by_version; std::map> export_partition_transaction_id_to_manifest; - std::mutex export_partition_transaction_id_to_manifest_mutex; + mutable std::mutex export_partition_transaction_id_to_manifest_mutex; /// mutable because of getExportsStatus /// Unfinished mutations that are required for AlterConversions. MutationCounters mutation_counters; diff --git a/src/Storages/System/StorageSystemExports.cpp b/src/Storages/System/StorageSystemExports.cpp index 3b4c938ac115..a919d7d9d88a 100644 --- a/src/Storages/System/StorageSystemExports.cpp +++ b/src/Storages/System/StorageSystemExports.cpp @@ -1,9 +1,16 @@ #include #include #include +#include #include +#include "Columns/ColumnString.h" #include "DataTypes/DataTypeString.h" #include "DataTypes/DataTypesNumber.h" +#include "Storages/MergeTree/MergeTreeExportStatus.h" +#include "Storages/VirtualColumnUtils.h" +#include +#include +#include namespace DB @@ -17,33 +24,111 @@ ColumnsDescription StorageSystemExports::getColumnsDescription() {"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."}, - {"elapsed", std::make_shared(), "Time elapsed (in seconds) since data part movement started."}, - {"destination_path", std::make_shared(), "Path to the destination file in the destination storage."}, - {"part_name", std::make_shared(), "Name of the data part being moved."}, - {"thread_id", std::make_shared(), "Identifier of a thread performing the movement."}, + {"transaction_id", std::make_shared(), "The ID of the export transaction."}, + {"create_time", std::make_shared(), "Date and time when the export command was submitted for execution."}, + {"parts_to_do_names", std::make_shared(std::make_shared()), "An array of names of data parts that need to be exported for the transaction to complete."}, + {"parts_to_do", std::make_shared(), "The number of data parts that need to be exported for the transaction to complete."}, + {"status", std::make_shared(), "The status of the export transaction."}, + // {"latest_failed_part_name", std::make_shared(), "The name of the most recent part that could not be exported."}, + // {"latest_fail_time", std::make_shared(), "The date and time of the most recent part export failure."}, + // {"latest_fail_reason", std::make_shared(), "The exception message that caused the most recent part export failure."}, }; } -void StorageSystemExports::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const +void StorageSystemExports::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const { const auto access = context->getAccess(); - const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_TABLES); - const auto & move_list = context->getExportsList(); - - for (const auto & move : move_list.get()) + const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); + + /// Collect a set of *MergeTree tables. + std::map> merge_tree_tables; + for (const auto & db : DatabaseCatalog::instance().getDatabases()) { - if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, move.source_database, move.source_table)) + /// Check if database can contain MergeTree tables + if (!db.second->canContainMergeTreeTables()) continue; - size_t i = 0; - res_columns[i++]->insert(move.source_database); - res_columns[i++]->insert(move.source_table); - res_columns[i++]->insert(move.destination_database); - res_columns[i++]->insert(move.destination_table); - res_columns[i++]->insert(move.elapsed); - res_columns[i++]->insert(move.destination_path); - res_columns[i++]->insert(move.part_name); - res_columns[i++]->insert(move.thread_id); + 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; + + if (!dynamic_cast(table.get())) + continue; + + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name())) + continue; + + merge_tree_tables[db.first][iterator->name()] = table; + } + } + + MutableColumnPtr col_source_database_export = ColumnString::create(); + MutableColumnPtr col_source_table_export = ColumnString::create(); + + for (auto & db : merge_tree_tables) + { + for (auto & table : db.second) + { + col_source_database_export->insert(db.first); + col_source_table_export->insert(table.first); + } + } + + ColumnPtr col_source_database = std::move(col_source_database_export); + ColumnPtr col_source_table = std::move(col_source_table_export); + + /// Determine what tables are needed by the conditions in the query. + { + Block filtered_block + { + { col_source_database, std::make_shared(), "source_database" }, + { col_source_table, std::make_shared(), "source_table" }, + }; + + VirtualColumnUtils::filterBlockWithPredicate(predicate, filtered_block, context); + + if (!filtered_block.rows()) + return; + + col_source_database = filtered_block.getByName("source_database").column; + col_source_table = filtered_block.getByName("source_table").column; + } + + for (size_t i_storage = 0; i_storage < col_source_database->size(); ++i_storage) + { + auto database = (*col_source_database)[i_storage].safeGet(); + auto table = (*col_source_table)[i_storage].safeGet(); + + std::vector statuses; + { + const IStorage * storage = merge_tree_tables[database][table].get(); + if (const auto * merge_tree = dynamic_cast(storage)) + statuses = merge_tree->getExportsStatus(); + } + + for (const MergeTreeExportStatus & status : statuses) + { + /// fill based on getColumnsDescription + Array parts_to_do_names; + parts_to_do_names.reserve(status.parts_to_do_names.size()); + for (const String & part_name : status.parts_to_do_names) + parts_to_do_names.emplace_back(part_name); + + size_t col_num = 0; + res_columns[col_num++]->insert(status.source_database); + res_columns[col_num++]->insert(status.source_table); + res_columns[col_num++]->insert(status.destination_database); + res_columns[col_num++]->insert(status.destination_table); + res_columns[col_num++]->insert(status.transaction_id); + res_columns[col_num++]->insert(status.create_time); + res_columns[col_num++]->insert(parts_to_do_names); + res_columns[col_num++]->insert(status.parts_to_do_names.size()); + res_columns[col_num++]->insert(String(magic_enum::enum_name(status.status))); + } } } From 34f71304c5c8c9097730eba14e0d2c24717906a3 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 3 Sep 2025 13:30:22 -0300 Subject: [PATCH 35/48] opsy --- src/Storages/StorageMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ec216d2768b7..2b42e369df65 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -997,7 +997,7 @@ std::vector StorageMergeTree::getExportsStatus() const status.source_table = source_table; status.destination_database = manifest->destination_storage_id.database_name; status.destination_table = manifest->destination_storage_id.table_name; - status.create_time = 0; + status.create_time = manifest->create_time; status.parts_to_do_names = manifest->pendingParts(); status.status = manifest->status; From a39d63ac920dce29c9f81180221b6367be5cccc9 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 3 Sep 2025 16:42:44 -0300 Subject: [PATCH 36/48] remove old partition task --- src/Core/Settings.cpp | 3 - .../ExportPartitionPlainMergeTreeTask.cpp | 221 ------------------ .../ExportPartitionPlainMergeTreeTask.h | 70 ------ src/Storages/StorageMergeTree.cpp | 52 +---- 4 files changed, 6 insertions(+), 340 deletions(-) delete mode 100644 src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.cpp delete mode 100644 src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.h diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index f02fbf61726e..3456e1824084 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -6877,9 +6877,6 @@ Experimental timeSeries* aggregate functions for Prometheus-like timeseries resa DECLARE_WITH_ALIAS(Bool, allow_experimental_export_merge_tree_partition, false, R"( Experimental export merge tree partition. )", EXPERIMENTAL, allow_experimental_export_merge_tree_partition) \ - DECLARE_WITH_ALIAS(Bool, export_merge_tree_partition_executor, false, R"( - Use the part task instead of the partition task -)", EXPERIMENTAL, export_merge_tree_partition_executor) \ \ /* ####################################################### */ \ /* ############ END OF EXPERIMENTAL FEATURES ############# */ \ diff --git a/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.cpp b/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.cpp deleted file mode 100644 index 30bbfd5785a9..000000000000 --- a/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.cpp +++ /dev/null @@ -1,221 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -ExportPartitionPlainMergeTreeTask::ExportPartitionPlainMergeTreeTask( - StorageMergeTree & storage_, - const std::shared_ptr & exports_tagger_, - const StoragePtr & destination_storage_, - ContextPtr context_, - std::shared_ptr manifest_, - IExecutableTask::TaskResultCallback & task_result_callback_, - size_t max_retries_) - : storage(storage_) - , exports_tagger(exports_tagger_) - , destination_storage(destination_storage_) - , context(std::move(context_)) - , manifest(std::move(manifest_)) - , task_result_callback(task_result_callback_) - , max_retries(max_retries_) -{ - UInt64 transaction_id = std::stoull(manifest->transaction_id); - priority.value = transaction_id; -} - -StorageID ExportPartitionPlainMergeTreeTask::getStorageID() const -{ - return storage.getStorageID(); -} - -String ExportPartitionPlainMergeTreeTask::getQueryId() const -{ - return getStorageID().getShortName() + "::export_partition::" + manifest->transaction_id; -} - -bool ExportPartitionPlainMergeTreeTask::executeStep() -{ - if (cancelled) - return false; - - switch (state) - { - case State::NEED_PREPARE: - { - prepare(); - state = State::NEED_EXECUTE; - return true; - } - case State::NEED_EXECUTE: - { - executeExport(); - state = State::NEED_ANALYZE; - return true; - } - case State::NEED_ANALYZE: - { - if (exportedAllIndividualParts()) - { - state = State::NEED_COMMIT; - return true; - } - else - { - if (max_retries > retry_count) - { - LOG_INFO(getLogger("ExportMergeTreePartitionToObjectStorageTask"), - "Retrying export attempt {} for partition {}", - retry_count, manifest->partition_id); - state = State::NEED_EXECUTE; - retry_count++; - return true; - } - - /// do we need to update the state here? - return false; - } - } - case State::NEED_COMMIT: - { - if (commitExport()) - { - state = State::SUCCESS; - } - else - { - state = State::NEED_EXECUTE; - retry_count++; - LOG_INFO(getLogger("ExportMergeTreePartitionToObjectStorageTask"), - "Retrying export attempt {} for partition {}", - retry_count, manifest->partition_id); - } - - return true; - } - case State::SUCCESS: - { - return false; - } - } - - return false; -} - -void ExportPartitionPlainMergeTreeTask::prepare() -{ - stopwatch_ptr = std::make_unique(); -} - -void ExportPartitionPlainMergeTreeTask::executeExport() -{ - if (cancelled) - return; - - try - { - // Build a vector of parts that have not been exported yet (i.e., not present in manifest->exportedPaths) - std::vector parts_to_export; - const auto & items = manifest->items; - for (const auto & part : exports_tagger->parts_to_export) - { - if (std::find_if(items.begin(), items.end(), [&part](const auto & item) { - return item.part_name == part->name && item.remote_path.empty(); - }) != items.end()) - parts_to_export.push_back(part); - } - - std::function part_log_wrapper = [this](MergeTreePartImportStats stats) { - - std::lock_guard lock(storage.export_partition_transaction_id_to_manifest_mutex); - auto table_id = storage.getStorageID(); - - if (stats.status.code != 0) - { - LOG_INFO(getLogger("ExportMergeTreePartitionToObjectStorageTask"), "Error importing part {}: {}", stats.part->name, stats.status.message); - return; - } - - storage.export_partition_transaction_id_to_manifest[manifest->transaction_id]->updateRemotePathAndWrite( - stats.part->name, - stats.file_path); - - UInt64 elapsed_ns = stopwatch_ptr->elapsedNanoseconds(); - - storage.writePartLog( - PartLogElement::Type::EXPORT_PART, - stats.status, - elapsed_ns, - stats.part->name, - stats.part, - {stats.part}, - nullptr, - nullptr); - }; - - destination_storage->importMergeTreePartition( - storage, - parts_to_export, - context, - part_log_wrapper); - } - catch (...) - { - LOG_ERROR(getLogger("ExportMergeTreePartitionToObjectStorageTask"), - "Export attempt failed completely: {}", getCurrentExceptionMessage(true)); - - std::lock_guard lock(storage.export_partition_transaction_id_to_manifest_mutex); - manifest->deleteFile(); - storage.already_exported_partition_ids.erase(manifest->partition_id); - storage.export_partition_transaction_id_to_manifest.erase(manifest->transaction_id); - - - throw; - } -} - -bool ExportPartitionPlainMergeTreeTask::commitExport() -{ - std::lock_guard lock(storage.export_partition_transaction_id_to_manifest_mutex); - - destination_storage->commitExportPartitionTransaction( - manifest->transaction_id, - manifest->partition_id, - manifest->exportedPaths(), - context); - - manifest->status = MergeTreeExportManifest::Status::completed; - manifest->write(); - - storage.export_partition_transaction_id_to_manifest.erase(manifest->transaction_id); - - LOG_INFO(getLogger("ExportMergeTreePartitionToObjectStorageTask"), - "Successfully committed export transaction {} for partition {}", - manifest->transaction_id, manifest->partition_id); - - return true; -} - -void ExportPartitionPlainMergeTreeTask::onCompleted() -{ - std::lock_guard lock(storage.export_partition_transaction_id_to_manifest_mutex); - task_result_callback(manifest->status == MergeTreeExportManifest::Status::completed); -} - -void ExportPartitionPlainMergeTreeTask::cancel() noexcept -{ - cancelled = true; -} - -bool ExportPartitionPlainMergeTreeTask::exportedAllIndividualParts() const -{ - std::lock_guard lock(storage.export_partition_transaction_id_to_manifest_mutex); - return manifest->exportedPaths().size() == manifest->items.size(); -} - -} diff --git a/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.h b/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.h deleted file mode 100644 index abb1948295ee..000000000000 --- a/src/Storages/ObjectStorage/MergeTree/ExportPartitionPlainMergeTreeTask.h +++ /dev/null @@ -1,70 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -class MergeTreeData; - -class ExportPartitionPlainMergeTreeTask : public IExecutableTask -{ -public: - ExportPartitionPlainMergeTreeTask( - StorageMergeTree & storage_, - const std::shared_ptr & exports_tagger_, - const StoragePtr & destination_storage_, - ContextPtr context_, - std::shared_ptr manifest_, - IExecutableTask::TaskResultCallback & task_result_callback_, - size_t max_retries_ = 3); - - void onCompleted() override; - bool executeStep() override; - void cancel() noexcept override; - StorageID getStorageID() const override; - Priority getPriority() const override { return priority; } - String getQueryId() const override; - -private: - void prepare(); - void executeExport(); - bool commitExport(); - bool exportedAllIndividualParts() const; - - enum class State : uint8_t - { - NEED_PREPARE, - NEED_EXECUTE, - NEED_ANALYZE, - NEED_COMMIT, - SUCCESS - }; - - State state{State::NEED_PREPARE}; - - StorageMergeTree & storage; - std::shared_ptr exports_tagger; - StoragePtr destination_storage; - ContextPtr context; - std::shared_ptr manifest; - IExecutableTask::TaskResultCallback task_result_callback; - - size_t max_retries; - size_t retry_count = 0; - Priority priority; - std::unique_ptr stopwatch_ptr; - - bool cancelled = false; - std::exception_ptr current_exception; -}; - -using ExportPartitionPlainMergeTreeTaskPtr = std::shared_ptr; - -} diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 2b42e369df65..6106583fb74e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -49,7 +49,6 @@ #include "Core/BackgroundSchedulePool.h" #include "Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h" #include -#include #include #include #include @@ -82,7 +81,6 @@ namespace Setting extern const SettingsBool throw_on_unsupported_query_inside_transaction; extern const SettingsUInt64 max_parts_to_move; extern const SettingsBool allow_experimental_export_merge_tree_partition; - extern const SettingsBool export_merge_tree_partition_executor; } namespace MergeTreeSetting @@ -576,27 +574,10 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, export_partition_transaction_id_to_manifest[transaction_id] = manifest; } - if (!getContext()->getSettingsRef()[Setting::export_merge_tree_partition_executor]) + for (const auto & part : all_parts) { - for (const auto & part : all_parts) - { - auto tagger = std::make_shared(std::vector{part}, *this); - auto task = std::make_shared(*this, tagger, dest_storage, getContext(), manifest, moves_assignee_trigger); - background_moves_assignee.scheduleMoveTask(task); - } - } - else - { - auto exports_tagger = std::make_shared(std::move(all_parts), *this); - - auto task = std::make_shared( - *this, - exports_tagger, - dest_storage, - getContext(), - manifest, - moves_assignee_trigger); - + auto tagger = std::make_shared(std::vector{part}, *this); + auto task = std::make_shared(*this, tagger, dest_storage, getContext(), manifest, moves_assignee_trigger); background_moves_assignee.scheduleMoveTask(task); } } @@ -1247,33 +1228,12 @@ void StorageMergeTree::resumeExportPartitionTasks() parts_to_export.emplace_back(part); } - if (!getContext()->getSettingsRef()[Setting::export_merge_tree_partition_executor]) + for (const auto & part : parts_to_export) { - for (const auto & part : parts_to_export) - { - auto tagger = std::make_shared(std::vector{part}, *this); - auto task = std::make_shared(*this, tagger, destination_storage, getContext(), manifest, moves_assignee_trigger); - background_moves_assignee.scheduleMoveTask(task); - } - } - else - { - /// TODO: this locks the parts that have not been exported yet. Should we also lock the already exported parts as well? - /// There is some inconsistency with in-progress exports. The parts will not be unlocked until all parts have been exported OR a re-start happens - /// I just checked and mutations handle it slightly different. Tagger will actually contain a single part, which is released as soon as it finishes. - auto exports_tagger = std::make_shared(std::move(parts_to_export), *this); - - auto task = std::make_shared( - *this, - exports_tagger, - destination_storage, - getContext(), - manifest, - moves_assignee_trigger); - + auto tagger = std::make_shared(std::vector{part}, *this); + auto task = std::make_shared(*this, tagger, destination_storage, getContext(), manifest, moves_assignee_trigger); background_moves_assignee.scheduleMoveTask(task); } - } } From 666394954abbee715fdc33975f39c3215cfd844f Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Wed, 3 Sep 2025 16:47:34 -0300 Subject: [PATCH 37/48] rmv no longer used method --- .../ObjectStorage/StorageObjectStorage.cpp | 113 ------------------ .../ObjectStorage/StorageObjectStorage.h | 6 - 2 files changed, 119 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index f29f1089d046..29b1b9aebefd 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -631,123 +631,10 @@ void StorageObjectStorage::importMergeTreePart( pipeline.complete(sink); - pipeline.setNumThreads(local_context->getSettingsRef()[Setting::max_threads]); - CompletedPipelineExecutor exec(pipeline); exec.execute(); } -void StorageObjectStorage::importMergeTreePartition( - const MergeTreeData & merge_tree_data, - const std::vector & data_parts, - ContextPtr local_context, - std::function part_log) -{ - if (data_parts.empty()) - return; - - std::vector part_plans; - part_plans.reserve(data_parts.size()); - - auto metadata_snapshot = merge_tree_data.getInMemoryMetadataPtr(); - Names columns_to_read = metadata_snapshot->getColumns().getNamesOfPhysical(); - StorageSnapshotPtr storage_snapshot = merge_tree_data.getStorageSnapshot(metadata_snapshot, local_context); - - QueryPlan plan; - - /// using the mutations type for now - MergeTreeSequentialSourceType read_type = MergeTreeSequentialSourceType::Mutation; - - bool apply_deleted_mask = true; - bool read_with_direct_io = false; - bool prefetch = false; - - QueryPipeline export_pipeline; - - std::vector export_list_entries; - - for (const auto & data_part : data_parts) - { - const auto partition_columns = configuration->partition_strategy->getPartitionColumns(); - - auto block_with_partition_values = data_part->partition.getBlockWithPartitionValues(partition_columns); - - const auto column_with_partition_key = configuration->partition_strategy->computePartitionKey(block_with_partition_values); - - const auto file_path = configuration->file_path_generator->getWritingPath(column_with_partition_key->getDataAt(0).toString()); - - export_list_entries.emplace_back(local_context->getGlobalContext()->getExportsList().insert( - merge_tree_data.getStorageID(), - getStorageID(), - data_part->name, - file_path - )); - - MergeTreeData::IMutationsSnapshot::Params params - { - .metadata_version = metadata_snapshot->getMetadataVersion(), - .min_part_metadata_version = data_part->getMetadataVersion(), - }; - - auto mutations_snapshot = merge_tree_data.getMutationsSnapshot(params); - - auto alter_conversions = MergeTreeData::getAlterConversionsForPart( - data_part, - mutations_snapshot, - local_context); - - QueryPlan plan_for_part; - - createReadFromPartStep( - read_type, - plan_for_part, - merge_tree_data, - storage_snapshot, - RangesInDataPart(data_part), - alter_conversions, - nullptr, - columns_to_read, - nullptr, - apply_deleted_mask, - std::nullopt, - read_with_direct_io, - prefetch, - local_context, - getLogger("ExportPartition")); - - QueryPlanOptimizationSettings optimization_settings(local_context); - auto pipeline_settings = BuildQueryPipelineSettings(local_context); - auto builder = plan_for_part.buildQueryPipeline(optimization_settings, pipeline_settings); - auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); - - auto sink = std::make_shared( - data_part, - file_path, - object_storage, - configuration, - format_settings, - metadata_snapshot->getSampleBlock(), - part_log, - local_context - ); - - pipeline.complete(sink); - export_pipeline.addCompletedPipeline(std::move(pipeline)); - } - - if (!export_pipeline.completed()) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Root pipeline is not completed"); - } - - export_pipeline.setNumThreads(local_context->getSettingsRef()[Setting::max_threads]); - - CompletedPipelineExecutor exec(export_pipeline); - exec.execute(); - - // NOTE: Do not write commit file here. The caller manages commit via JSON manifest. -} - 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; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index aaf92a41d399..ac8c7de72721 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -101,12 +101,6 @@ class StorageObjectStorage : public IStorage bool supportsImportMergeTreePartition() const override; - void importMergeTreePartition( - const MergeTreeData & merge_tree_data, - const std::vector & data_parts, - ContextPtr /*context*/, - std::function part_log) override; - void importMergeTreePart( const MergeTreeData & merge_tree_data, const DataPartPtr & data_part, From 46cda681bc0e2433662475646ba1de1616af4ba9 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 4 Sep 2025 09:16:56 -0300 Subject: [PATCH 38/48] fix settingschangehist? --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 89a550f3326a..c75d08b4e046 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -71,6 +71,7 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() addSettingsChanges(settings_changes_history, "25.6.5.2000", { {"output_format_parquet_enum_as_byte_array", true, true, "Enable writing Enum as byte array in Parquet by default"}, + {"allow_experimental_export_merge_tree_partition", true, false, "Enable experimental export merge tree partition"}, }); addSettingsChanges(settings_changes_history, "25.6", { From d4f01f1774e216d611eab4d247769c0239971e38 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 4 Sep 2025 09:27:47 -0300 Subject: [PATCH 39/48] remove exportslist --- src/Interpreters/Context.cpp | 4 -- src/Interpreters/Context.h | 4 -- src/Storages/ExportsList.cpp | 34 ---------- src/Storages/ExportsList.h | 62 ------------------- .../ObjectStorage/StorageObjectStorage.cpp | 1 - src/Storages/System/StorageSystemExports.cpp | 1 - 6 files changed, 106 deletions(-) delete mode 100644 src/Storages/ExportsList.cpp delete mode 100644 src/Storages/ExportsList.h diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index caf1831e114e..db58e2352267 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -34,7 +34,6 @@ #include #include #include -#include #include #include #include @@ -464,7 +463,6 @@ struct ContextSharedPart : boost::noncopyable GlobalOvercommitTracker global_overcommit_tracker; MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) MovesList moves_list; /// The list of executing moves (for (Replicated)?MergeTree) - ExportsList exports_list; /// The list of executing exports MergeTree -> Object storage ReplicatedFetchList replicated_fetch_list; RefreshSet refresh_set; /// The list of active refreshes (for MaterializedView) ConfigurationPtr users_config TSA_GUARDED_BY(mutex); /// Config with the users, profiles and quotas sections. @@ -1150,8 +1148,6 @@ MergeList & Context::getMergeList() { return shared->merge_list; } const MergeList & Context::getMergeList() const { return shared->merge_list; } MovesList & Context::getMovesList() { return shared->moves_list; } const MovesList & Context::getMovesList() const { return shared->moves_list; } -ExportsList & Context::getExportsList() { return shared->exports_list; } -const ExportsList & Context::getExportsList() const { return shared->exports_list; } ReplicatedFetchList & Context::getReplicatedFetchList() { return shared->replicated_fetch_list; } const ReplicatedFetchList & Context::getReplicatedFetchList() const { return shared->replicated_fetch_list; } RefreshSet & Context::getRefreshSet() { return shared->refresh_set; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 344e86ceb14f..f26899ea3c72 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -88,7 +88,6 @@ class AsynchronousMetrics; class BackgroundSchedulePool; class MergeList; class MovesList; -class ExportsList; class ReplicatedFetchList; class RefreshSet; class Cluster; @@ -1142,9 +1141,6 @@ class Context: public ContextData, public std::enable_shared_from_this MovesList & getMovesList(); const MovesList & getMovesList() const; - ExportsList & getExportsList(); - const ExportsList & getExportsList() const; - ReplicatedFetchList & getReplicatedFetchList(); const ReplicatedFetchList & getReplicatedFetchList() const; diff --git a/src/Storages/ExportsList.cpp b/src/Storages/ExportsList.cpp deleted file mode 100644 index b1c5afebbb7b..000000000000 --- a/src/Storages/ExportsList.cpp +++ /dev/null @@ -1,34 +0,0 @@ -#include -#include "base/getThreadId.h" - -namespace DB -{ - -ExportsListElement::ExportsListElement( - const StorageID & source_table_id_, - const StorageID & destination_table_id_, - const std::string & part_name_, - const std::string & destination_path_) -: source_table_id(source_table_id_), - destination_table_id(destination_table_id_), - part_name(part_name_), - destination_path(destination_path_), - thread_id(getThreadId()) -{ -} - -ExportInfo ExportsListElement::getInfo() const -{ - ExportInfo res; - res.source_database = source_table_id.database_name; - res.source_table = source_table_id.table_name; - res.destination_database = destination_table_id.database_name; - res.destination_table = destination_table_id.table_name; - res.part_name = part_name; - res.destination_path = destination_path; - res.elapsed = watch.elapsedSeconds(); - res.thread_id = thread_id; - return res; -} - -} diff --git a/src/Storages/ExportsList.h b/src/Storages/ExportsList.h deleted file mode 100644 index 053b89aeba2c..000000000000 --- a/src/Storages/ExportsList.h +++ /dev/null @@ -1,62 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - -namespace CurrentMetrics -{ -extern const Metric Export; -} - -namespace DB -{ - -struct ExportInfo -{ - std::string source_database; - std::string destination_database; - std::string source_table; - std::string destination_table; - std::string part_name; - std::string destination_path; - - Float64 elapsed; - UInt64 thread_id; -}; - -struct ExportsListElement : private boost::noncopyable -{ - const StorageID source_table_id; - const StorageID destination_table_id; - const std::string part_name; - const std::string destination_path; - - Stopwatch watch; - const UInt64 thread_id; - - ExportsListElement( - const StorageID & source_table_id_, - const StorageID & destination_table_id_, - const std::string & part_name_, - const std::string & destination_path_); - - ExportInfo getInfo() const; -}; - - -/// List of currently processing moves -class ExportsList final : public BackgroundProcessList -{ -private: - using Parent = BackgroundProcessList; - -public: - ExportsList() - : Parent(CurrentMetrics::Export) - {} -}; - -} diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 29b1b9aebefd..1508527c1c5f 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -19,7 +19,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/System/StorageSystemExports.cpp b/src/Storages/System/StorageSystemExports.cpp index a919d7d9d88a..eb6fe90b602e 100644 --- a/src/Storages/System/StorageSystemExports.cpp +++ b/src/Storages/System/StorageSystemExports.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include #include "Columns/ColumnString.h" From 4c39630341c58ca6ebd3f94f9b927d2ba7e5d729 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 4 Sep 2025 10:17:38 -0300 Subject: [PATCH 40/48] small changes --- .../MergeTree/MergeMutateSelectedEntry.h | 4 +-- .../ExportPartPlainMergeTreeTask.cpp | 8 ++--- src/Storages/StorageMergeTree.cpp | 32 ++++++++++--------- 3 files changed, 23 insertions(+), 21 deletions(-) diff --git a/src/Storages/MergeTree/MergeMutateSelectedEntry.h b/src/Storages/MergeTree/MergeMutateSelectedEntry.h index bb44d350d3ba..60c05ab23ee6 100644 --- a/src/Storages/MergeTree/MergeMutateSelectedEntry.h +++ b/src/Storages/MergeTree/MergeMutateSelectedEntry.h @@ -53,9 +53,9 @@ using MergeMutateSelectedEntryPtr = std::shared_ptr; struct CurrentlyExportingPartsTagger { - std::vector parts_to_export; + DataPartPtr part_to_export; StorageMergeTree & storage; - CurrentlyExportingPartsTagger(std::vector && parts_to_export_, StorageMergeTree & storage_); + CurrentlyExportingPartsTagger(DataPartPtr part_to_export_, StorageMergeTree & storage_); ~CurrentlyExportingPartsTagger(); }; diff --git a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp index 93aa1abcff2e..87f4dbb218f0 100644 --- a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp +++ b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp @@ -63,7 +63,7 @@ bool ExportPartPlainMergeTreeTask::executeStep() retry_count++; LOG_INFO(getLogger("ExportPartPlainMergeTreeTask"), "Retrying export attempt {} for part {}", - retry_count, exports_tagger->parts_to_export[0]->name); + retry_count, exports_tagger->part_to_export->name); state = State::NEED_EXECUTE; } else @@ -84,7 +84,7 @@ bool ExportPartPlainMergeTreeTask::executeStep() retry_count++; LOG_INFO(getLogger("ExportPartPlainMergeTreeTask"), "Retrying export attempt {} for part {}", - retry_count, exports_tagger->parts_to_export[0]->name); + retry_count, exports_tagger->part_to_export->name); state = State::NEED_COMMIT; } else @@ -157,7 +157,7 @@ bool ExportPartPlainMergeTreeTask::executeExport() { destination_storage->importMergeTreePart( storage, - exports_tagger->parts_to_export[0], + exports_tagger->part_to_export, context, part_log_wrapper); @@ -165,7 +165,7 @@ bool ExportPartPlainMergeTreeTask::executeExport() } catch (...) { - LOG_ERROR(getLogger("ExportPartPlainMergeTreeTask"), "Failed to export part {}", exports_tagger->parts_to_export[0]->name); + LOG_ERROR(getLogger("ExportPartPlainMergeTreeTask"), "Failed to export part {}", exports_tagger->part_to_export->name); return false; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 6106583fb74e..c95cd4a40b03 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -559,7 +559,14 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, const auto transaction_id = std::to_string(generateSnowflakeID()); - /// TODO missing parts lock here with tagger + std::vector> taggers; + taggers.reserve(all_parts.size()); + + for (const auto & part : all_parts) + { + taggers.push_back(std::make_shared(part, *this)); + } + const auto manifest = MergeTreeExportManifest::create( getStoragePolicy()->getAnyDisk(), relative_data_path, @@ -574,9 +581,8 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, export_partition_transaction_id_to_manifest[transaction_id] = manifest; } - for (const auto & part : all_parts) + for (const auto & tagger : taggers) { - auto tagger = std::make_shared(std::vector{part}, *this); auto task = std::make_shared(*this, tagger, dest_storage, getContext(), manifest, moves_assignee_trigger); background_moves_assignee.scheduleMoveTask(task); } @@ -656,25 +662,21 @@ CurrentlyMergingPartsTagger::~CurrentlyMergingPartsTagger() storage.currently_processing_in_background_condition.notify_all(); } -CurrentlyExportingPartsTagger::CurrentlyExportingPartsTagger(std::vector && parts_to_export_, StorageMergeTree & storage_) - : parts_to_export(std::move(parts_to_export_)), storage(storage_) +CurrentlyExportingPartsTagger::CurrentlyExportingPartsTagger(DataPartPtr part_to_export_, StorageMergeTree & storage_) + : part_to_export(std::move(part_to_export_)), storage(storage_) { /// assume it is already locked - for (const auto & part : parts_to_export) - if (!storage.currently_merging_mutating_parts.emplace(part).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Tagging already tagged part {}. This is a bug.", part->name); + if (!storage.currently_merging_mutating_parts.emplace(part_to_export).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Tagging already tagged part {}. This is a bug.", part_to_export->name); } CurrentlyExportingPartsTagger::~CurrentlyExportingPartsTagger() { std::lock_guard lock(storage.currently_processing_in_background_mutex); - for (const auto & part : parts_to_export) - { - if (!storage.currently_merging_mutating_parts.contains(part)) - std::terminate(); - storage.currently_merging_mutating_parts.erase(part); - } + if (!storage.currently_merging_mutating_parts.contains(part_to_export)) + std::terminate(); + storage.currently_merging_mutating_parts.erase(part_to_export); storage.currently_processing_in_background_condition.notify_all(); } @@ -1230,7 +1232,7 @@ void StorageMergeTree::resumeExportPartitionTasks() for (const auto & part : parts_to_export) { - auto tagger = std::make_shared(std::vector{part}, *this); + auto tagger = std::make_shared(part, *this); auto task = std::make_shared(*this, tagger, destination_storage, getContext(), manifest, moves_assignee_trigger); background_moves_assignee.scheduleMoveTask(task); } From 46b1724ff66e89a8431144918bad800b7aff7ad7 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 4 Sep 2025 15:03:26 -0300 Subject: [PATCH 41/48] try to remove deadlock, fail --- .../ExportPartPlainMergeTreeTask.cpp | 24 +++++----- src/Storages/StorageMergeTree.cpp | 45 ++++++++++++------- 2 files changed, 41 insertions(+), 28 deletions(-) diff --git a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp index 87f4dbb218f0..9784e23d5cdf 100644 --- a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp +++ b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp @@ -126,20 +126,8 @@ bool ExportPartPlainMergeTreeTask::executeExport() return false; std::function part_log_wrapper = [this](MergeTreePartImportStats stats) { - - std::lock_guard lock(storage.export_partition_transaction_id_to_manifest_mutex); auto table_id = storage.getStorageID(); - if (stats.status.code != 0) - { - LOG_INFO(getLogger("ExportMergeTreePartitionToObjectStorageTask"), "Error importing part {}: {}", stats.part->name, stats.status.message); - return; - } - - storage.export_partition_transaction_id_to_manifest[manifest->transaction_id]->updateRemotePathAndWrite( - stats.part->name, - stats.file_path); - UInt64 elapsed_ns = stopwatch_ptr->elapsedNanoseconds(); storage.writePartLog( @@ -151,6 +139,18 @@ bool ExportPartPlainMergeTreeTask::executeExport() {stats.part}, nullptr, nullptr); + + if (stats.status.code != 0) + { + LOG_INFO(getLogger("ExportMergeTreePartitionToObjectStorageTask"), "Error importing part {}: {}", stats.part->name, stats.status.message); + return; + } + + std::lock_guard lock(storage.export_partition_transaction_id_to_manifest_mutex); + + storage.export_partition_transaction_id_to_manifest[manifest->transaction_id]->updateRemotePathAndWrite( + stats.part->name, + stats.file_path); }; try diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c95cd4a40b03..13bce247a3e0 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -548,25 +548,28 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, return; } - /// Do not put this in a scope because `CurrentlyExportingPartsTagger` instantiated above relies on this already being locked - /// shitty design I came up with huh - std::lock_guard lock_background_mutex(currently_processing_in_background_mutex); + std::vector> taggers; - if (!already_exported_partition_ids.emplace(partition_id).second) { - throw Exception(ErrorCodes::PART_IS_LOCKED, "Partition {} has already been exported", partition_id); - } + /// Do not put this in a scope because `CurrentlyExportingPartsTagger` instantiated above relies on this already being locked + /// shitty design I came up with huh + std::lock_guard lock_background_mutex(currently_processing_in_background_mutex); - const auto transaction_id = std::to_string(generateSnowflakeID()); - - std::vector> taggers; - taggers.reserve(all_parts.size()); + if (!already_exported_partition_ids.emplace(partition_id).second) + { + throw Exception(ErrorCodes::PART_IS_LOCKED, "Partition {} has already been exported", partition_id); + } + + taggers.reserve(all_parts.size()); - for (const auto & part : all_parts) - { - taggers.push_back(std::make_shared(part, *this)); + for (const auto & part : all_parts) + { + taggers.push_back(std::make_shared(part, *this)); + } } + const auto transaction_id = std::to_string(generateSnowflakeID()); + const auto manifest = MergeTreeExportManifest::create( getStoragePolicy()->getAnyDisk(), relative_data_path, @@ -584,7 +587,10 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, for (const auto & tagger : taggers) { auto task = std::make_shared(*this, tagger, dest_storage, getContext(), manifest, moves_assignee_trigger); - background_moves_assignee.scheduleMoveTask(task); + if (!background_moves_assignee.scheduleMoveTask(task)) + { + LOG_ERROR(log, "Failed to schedule export task for part {}", tagger->part_to_export->name); + } } } @@ -1212,6 +1218,9 @@ void StorageMergeTree::resumeExportPartitionTasks() auto pending_part_names = manifest->pendingParts(); + /// apparently, it is possible that pending parts are empty + /// if it is empty, I have to somehow commit and mark as completed.. + std::vector parts_to_export; for (const auto & part_name : pending_part_names) @@ -1220,11 +1229,15 @@ void StorageMergeTree::resumeExportPartitionTasks() if (!part) { - throw Exception( - ErrorCodes::LOGICAL_ERROR, "Part {} is present in the manifest file {}, but not found in the storage {}", + LOG_ERROR(log, "Part {} is present in the manifest file {}, but not found in the storage {}", part_name, manifest->transaction_id, getStorageID().getNameForLogs()); + manifest->status = MergeTreeExportManifest::Status::failed; + manifest->write(); + + already_exported_partition_ids.erase(manifest->partition_id); + continue; } parts_to_export.emplace_back(part); From 543ff3619439e0ed1e63a8a5bf46c81db779d98c Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 4 Sep 2025 20:36:24 -0300 Subject: [PATCH 42/48] use the background scheduler instead of scheduling upon every request. very hackish for now, need to improve part selection and blocking --- .../MergeTree/MergeMutateSelectedEntry.h | 2 +- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/MergeTreeExportManifest.h | 10 ++ .../ExportPartPlainMergeTreeTask.cpp | 21 ++- .../MergeTree/ExportPartPlainMergeTreeTask.h | 4 +- src/Storages/StorageMergeTree.cpp | 142 +++++++++++------- src/Storages/StorageMergeTree.h | 2 + 7 files changed, 119 insertions(+), 64 deletions(-) diff --git a/src/Storages/MergeTree/MergeMutateSelectedEntry.h b/src/Storages/MergeTree/MergeMutateSelectedEntry.h index 60c05ab23ee6..fe9546b056a6 100644 --- a/src/Storages/MergeTree/MergeMutateSelectedEntry.h +++ b/src/Storages/MergeTree/MergeMutateSelectedEntry.h @@ -55,8 +55,8 @@ struct CurrentlyExportingPartsTagger { DataPartPtr part_to_export; StorageMergeTree & storage; - CurrentlyExportingPartsTagger(DataPartPtr part_to_export_, StorageMergeTree & storage_); ~CurrentlyExportingPartsTagger(); + CurrentlyExportingPartsTagger(DataPartPtr part_to_export_, StorageMergeTree & storage_); }; using CurrentlyExportingPartsTaggerPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 1f46fd1b50b9..7d1f6b1f9a34 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1153,7 +1153,7 @@ class MergeTreeData : public IStorage, public WithMutableContext /// Schedules background job to like merge/mutate/fetch an executor virtual bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) = 0; /// Schedules job to move parts between disks/volumes and so on. - bool scheduleDataMovingJob(BackgroundJobsAssignee & assignee); + virtual bool scheduleDataMovingJob(BackgroundJobsAssignee & assignee); bool areBackgroundMovesNeeded() const; diff --git a/src/Storages/MergeTree/MergeTreeExportManifest.h b/src/Storages/MergeTree/MergeTreeExportManifest.h index c1aec96cdbcb..25c351c4e6ff 100644 --- a/src/Storages/MergeTree/MergeTreeExportManifest.h +++ b/src/Storages/MergeTree/MergeTreeExportManifest.h @@ -51,6 +51,7 @@ struct MergeTreeExportManifest { String part_name; String remote_path; // empty until uploaded + bool in_progress = false; /// this is just a hackish workaround for now }; @@ -174,6 +175,15 @@ struct MergeTreeExportManifest write(); } + void setInProgress(const String & part_name) + { + for (auto & i : items) + { + if (i.part_name == part_name) + i.in_progress = true; + } + } + std::vector pendingParts() const { std::vector res; diff --git a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp index 9784e23d5cdf..eb76eaeac620 100644 --- a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp +++ b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp @@ -11,14 +11,14 @@ namespace DB ExportPartPlainMergeTreeTask::ExportPartPlainMergeTreeTask( StorageMergeTree & storage_, - const std::shared_ptr & exports_tagger_, + const DataPartPtr & part_to_export_, const StoragePtr & destination_storage_, ContextPtr context_, std::shared_ptr manifest_, IExecutableTask::TaskResultCallback & task_result_callback_, size_t max_retries_) : storage(storage_) - , exports_tagger(exports_tagger_) + , part_to_export(part_to_export_) , destination_storage(destination_storage_) , context(std::move(context_)) , manifest(std::move(manifest_)) @@ -63,7 +63,7 @@ bool ExportPartPlainMergeTreeTask::executeStep() retry_count++; LOG_INFO(getLogger("ExportPartPlainMergeTreeTask"), "Retrying export attempt {} for part {}", - retry_count, exports_tagger->part_to_export->name); + retry_count, part_to_export->name); state = State::NEED_EXECUTE; } else @@ -84,7 +84,7 @@ bool ExportPartPlainMergeTreeTask::executeStep() retry_count++; LOG_INFO(getLogger("ExportPartPlainMergeTreeTask"), "Retrying export attempt {} for part {}", - retry_count, exports_tagger->part_to_export->name); + retry_count, part_to_export->name); state = State::NEED_COMMIT; } else @@ -101,16 +101,23 @@ bool ExportPartPlainMergeTreeTask::executeStep() manifest->status = MergeTreeExportManifest::Status::failed; manifest->write(); - storage.already_exported_partition_ids.erase(manifest->partition_id); + /// this is a mess, what if several fail? I need to re-think the architecture + /// I'll leave this commented out for now + // storage.already_exported_partition_ids.erase(manifest->partition_id); + + storage.currently_merging_mutating_parts.erase(part_to_export); return false; } case State::SUCCESS: { + storage.currently_merging_mutating_parts.erase(part_to_export); + return false; } } + storage.currently_merging_mutating_parts.erase(part_to_export); return false; } @@ -157,7 +164,7 @@ bool ExportPartPlainMergeTreeTask::executeExport() { destination_storage->importMergeTreePart( storage, - exports_tagger->part_to_export, + part_to_export, context, part_log_wrapper); @@ -165,7 +172,7 @@ bool ExportPartPlainMergeTreeTask::executeExport() } catch (...) { - LOG_ERROR(getLogger("ExportPartPlainMergeTreeTask"), "Failed to export part {}", exports_tagger->part_to_export->name); + LOG_ERROR(getLogger("ExportPartPlainMergeTreeTask"), "Failed to export part {}", part_to_export->name); return false; } diff --git a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h index 9c694b8fce6c..a81addfbc5be 100644 --- a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h +++ b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h @@ -18,7 +18,7 @@ class ExportPartPlainMergeTreeTask : public IExecutableTask public: ExportPartPlainMergeTreeTask( StorageMergeTree & storage_, - const std::shared_ptr & exports_tagger_, + const DataPartPtr & part_to_export_, const StoragePtr & destination_storage_, ContextPtr context_, std::shared_ptr manifest_, @@ -50,7 +50,7 @@ class ExportPartPlainMergeTreeTask : public IExecutableTask State state{State::NEED_PREPARE}; StorageMergeTree & storage; - std::shared_ptr exports_tagger; + DataPartPtr part_to_export; StoragePtr destination_storage; ContextPtr context; std::shared_ptr manifest; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 13bce247a3e0..0e39141330fd 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -548,8 +548,6 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, return; } - std::vector> taggers; - { /// Do not put this in a scope because `CurrentlyExportingPartsTagger` instantiated above relies on this already being locked /// shitty design I came up with huh @@ -559,12 +557,11 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, { throw Exception(ErrorCodes::PART_IS_LOCKED, "Partition {} has already been exported", partition_id); } - - taggers.reserve(all_parts.size()); for (const auto & part : all_parts) { - taggers.push_back(std::make_shared(part, *this)); + if (!currently_merging_mutating_parts.emplace(part).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Tagging already tagged part {}. This is a bug.", part->name); } } @@ -584,14 +581,7 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, export_partition_transaction_id_to_manifest[transaction_id] = manifest; } - for (const auto & tagger : taggers) - { - auto task = std::make_shared(*this, tagger, dest_storage, getContext(), manifest, moves_assignee_trigger); - if (!background_moves_assignee.scheduleMoveTask(task)) - { - LOG_ERROR(log, "Failed to schedule export task for part {}", tagger->part_to_export->name); - } - } + background_moves_assignee.trigger(); } /// While exists, marks parts as 'currently_merging_mutating_parts' and reserves free space on filesystem. @@ -1198,58 +1188,53 @@ void StorageMergeTree::readExportPartitionManifests() } } } + + background_moves_assignee.trigger(); } void StorageMergeTree::resumeExportPartitionTasks() { /// Initially I opted for having two separate methods: read and resume because I wanted to schedule the tasks in order /// but it turns out the background executor schedules tasks based on their priority, so it is likely this is not needed anymore. - for (const auto & [transaction_id, manifest] : export_partition_transaction_id_to_manifest) - { - if (manifest->status != MergeTreeExportManifest::Status::pending) - continue; + // for (const auto & [transaction_id, manifest] : export_partition_transaction_id_to_manifest) + // { + // if (manifest->status != MergeTreeExportManifest::Status::pending) + // continue; - auto destination_storage = DatabaseCatalog::instance().tryGetTable(manifest->destination_storage_id, getContext()); - if (!destination_storage) - { - LOG_ERROR(log, "Failed to reconstruct destination storage: {}", manifest->destination_storage_id.getNameForLogs()); - continue; - } + // auto destination_storage = DatabaseCatalog::instance().tryGetTable(manifest->destination_storage_id, getContext()); + // if (!destination_storage) + // { + // LOG_ERROR(log, "Failed to reconstruct destination storage: {}", manifest->destination_storage_id.getNameForLogs()); + // continue; + // } - auto pending_part_names = manifest->pendingParts(); + // auto pending_part_names = manifest->pendingParts(); - /// apparently, it is possible that pending parts are empty - /// if it is empty, I have to somehow commit and mark as completed.. + // /// apparently, it is possible that pending parts are empty + // /// if it is empty, I have to somehow commit and mark as completed.. - std::vector parts_to_export; + // std::vector parts_to_export; - for (const auto & part_name : pending_part_names) - { - auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active}); + // for (const auto & part_name : pending_part_names) + // { + // auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active}); - if (!part) - { - LOG_ERROR(log, "Part {} is present in the manifest file {}, but not found in the storage {}", - part_name, - manifest->transaction_id, - getStorageID().getNameForLogs()); - manifest->status = MergeTreeExportManifest::Status::failed; - manifest->write(); - - already_exported_partition_ids.erase(manifest->partition_id); - continue; - } + // if (!part) + // { + // LOG_ERROR(log, "Part {} is present in the manifest file {}, but not found in the storage {}", + // part_name, + // manifest->transaction_id, + // getStorageID().getNameForLogs()); + // manifest->status = MergeTreeExportManifest::Status::failed; + // manifest->write(); - parts_to_export.emplace_back(part); - } + // already_exported_partition_ids.erase(manifest->partition_id); + // continue; + // } - for (const auto & part : parts_to_export) - { - auto tagger = std::make_shared(part, *this); - auto task = std::make_shared(*this, tagger, destination_storage, getContext(), manifest, moves_assignee_trigger); - background_moves_assignee.scheduleMoveTask(task); - } - } + // parts_to_export.emplace_back(part); + // } + // } } void StorageMergeTree::loadExportPartition() @@ -1662,6 +1647,57 @@ UInt32 StorageMergeTree::getMaxLevelInBetween(const PartProperties & left, const return level; } + +bool StorageMergeTree::scheduleDataMovingJob(BackgroundJobsAssignee & assignee) +{ + if (MergeTreeData::scheduleDataMovingJob(assignee)) + { + return true; + } + + /// Try to schedule one export part task if any pending export exists + { + std::lock_guard lock(export_partition_transaction_id_to_manifest_mutex); + for (const auto & [transaction_id, manifest] : export_partition_transaction_id_to_manifest) + { + if (manifest->status != MergeTreeExportManifest::Status::pending) + continue; + + auto destination_storage = DatabaseCatalog::instance().tryGetTable(manifest->destination_storage_id, getContext()); + if (!destination_storage) + { + LOG_ERROR(log, "Failed to reconstruct destination storage: {}", manifest->destination_storage_id.getNameForLogs()); + continue; + } + + for (auto & item : manifest->items) + { + if (item.in_progress) + continue; + + auto part = getPartIfExists(item.part_name, {MergeTreeDataPartState::Active}); + if (!part) + { + LOG_ERROR(log, "Part {} is present in the manifest file {}, but not found in the storage {}", + item.part_name, + manifest->transaction_id, + getStorageID().getNameForLogs()); + manifest->status = MergeTreeExportManifest::Status::failed; + manifest->write(); + already_exported_partition_ids.erase(manifest->partition_id); + continue; + } + + auto task = std::make_shared(*this, part, destination_storage, getContext(), manifest, moves_assignee_trigger); + item.in_progress = background_moves_assignee.scheduleMoveTask(task); + return true; + + } + } + } + return false; +} + bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) { if (shutdown_called) @@ -2949,8 +2985,8 @@ MutationCounters StorageMergeTree::getMutationCounters() const void StorageMergeTree::startBackgroundMovesIfNeeded() { - if (areBackgroundMovesNeeded()) - background_moves_assignee.start(); + // if (areBackgroundMovesNeeded()) + background_moves_assignee.start(); } std::unique_ptr StorageMergeTree::getDefaultSettings() const diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index a23f43849453..ffc86b5ec851 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -119,6 +119,8 @@ class StorageMergeTree final : public MergeTreeData bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; + bool scheduleDataMovingJob(BackgroundJobsAssignee & assignee) override; + std::map getUnfinishedMutationCommands() const override; MergeTreeDeduplicationLog * getDeduplicationLog() { return deduplication_log.get(); } From 61e43cf644edddcb19ae1e693575230cc635be64 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Sun, 7 Sep 2025 14:44:39 -0300 Subject: [PATCH 43/48] do not lock parts, only hold references so they are not deleted from disk. first attempt --- .../MergeTree/MergeMutateSelectedEntry.h | 10 ---- .../MergeTree/MergeTreeExportManifest.h | 8 ++- .../ExportPartPlainMergeTreeTask.cpp | 11 +--- src/Storages/StorageMergeTree.cpp | 55 +++++++++---------- 4 files changed, 35 insertions(+), 49 deletions(-) diff --git a/src/Storages/MergeTree/MergeMutateSelectedEntry.h b/src/Storages/MergeTree/MergeMutateSelectedEntry.h index fe9546b056a6..c420cbca12b5 100644 --- a/src/Storages/MergeTree/MergeMutateSelectedEntry.h +++ b/src/Storages/MergeTree/MergeMutateSelectedEntry.h @@ -51,14 +51,4 @@ struct MergeMutateSelectedEntry using MergeMutateSelectedEntryPtr = std::shared_ptr; -struct CurrentlyExportingPartsTagger -{ - DataPartPtr part_to_export; - StorageMergeTree & storage; - ~CurrentlyExportingPartsTagger(); - CurrentlyExportingPartsTagger(DataPartPtr part_to_export_, StorageMergeTree & storage_); -}; - -using CurrentlyExportingPartsTaggerPtr = std::shared_ptr; - } diff --git a/src/Storages/MergeTree/MergeTreeExportManifest.h b/src/Storages/MergeTree/MergeTreeExportManifest.h index 25c351c4e6ff..407dc579cd18 100644 --- a/src/Storages/MergeTree/MergeTreeExportManifest.h +++ b/src/Storages/MergeTree/MergeTreeExportManifest.h @@ -52,6 +52,7 @@ struct MergeTreeExportManifest String part_name; String remote_path; // empty until uploaded bool in_progress = false; /// this is just a hackish workaround for now + DataPartPtr part; // hold reference to part so it does not get deleted from disk even if it is outdated. Should be null once we are done with it }; @@ -82,12 +83,15 @@ struct MergeTreeExportManifest manifest->file_path = std::filesystem::path(path_prefix) / ("export_partition_" + partition_id_ + "_transaction_" + transaction_id_ + ".json"); manifest->items.reserve(data_parts.size()); for (const auto & data_part : data_parts) - manifest->items.push_back({data_part->name, ""}); + manifest->items.emplace_back(data_part->name, "", false, data_part); manifest->write(); return manifest; } - static std::shared_ptr read(const DiskPtr & disk_, const String & file_path_) + /// will not fill parts ref, maybe I should. + static std::shared_ptr read( + const DiskPtr & disk_, + const String & file_path_) { auto manifest = std::make_shared(); manifest->disk = disk_; diff --git a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp index eb76eaeac620..7749a1044f83 100644 --- a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp +++ b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp @@ -101,23 +101,18 @@ bool ExportPartPlainMergeTreeTask::executeStep() manifest->status = MergeTreeExportManifest::Status::failed; manifest->write(); - /// this is a mess, what if several fail? I need to re-think the architecture - /// I'll leave this commented out for now - // storage.already_exported_partition_ids.erase(manifest->partition_id); - - storage.currently_merging_mutating_parts.erase(part_to_export); + /// doesn't sound ideal, but it is actually ok to allow this partition to be re-exported as soon as a single part fails + /// this is because the ongoing export will never commit, so it won't cause duplicates + storage.already_exported_partition_ids.erase(manifest->partition_id); return false; } case State::SUCCESS: { - storage.currently_merging_mutating_parts.erase(part_to_export); - return false; } } - storage.currently_merging_mutating_parts.erase(part_to_export); return false; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 0e39141330fd..83a1f46f09d6 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -47,6 +47,7 @@ #include #include #include "Core/BackgroundSchedulePool.h" +#include "Storages/MergeTree/MergeTreeDataPartState.h" #include "Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h" #include #include @@ -549,20 +550,12 @@ void StorageMergeTree::exportPartitionToTable(const PartitionCommand & command, } { - /// Do not put this in a scope because `CurrentlyExportingPartsTagger` instantiated above relies on this already being locked - /// shitty design I came up with huh std::lock_guard lock_background_mutex(currently_processing_in_background_mutex); if (!already_exported_partition_ids.emplace(partition_id).second) { throw Exception(ErrorCodes::PART_IS_LOCKED, "Partition {} has already been exported", partition_id); } - - for (const auto & part : all_parts) - { - if (!currently_merging_mutating_parts.emplace(part).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Tagging already tagged part {}. This is a bug.", part->name); - } } const auto transaction_id = std::to_string(generateSnowflakeID()); @@ -658,26 +651,6 @@ CurrentlyMergingPartsTagger::~CurrentlyMergingPartsTagger() storage.currently_processing_in_background_condition.notify_all(); } -CurrentlyExportingPartsTagger::CurrentlyExportingPartsTagger(DataPartPtr part_to_export_, StorageMergeTree & storage_) - : part_to_export(std::move(part_to_export_)), storage(storage_) -{ - /// assume it is already locked - if (!storage.currently_merging_mutating_parts.emplace(part_to_export).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Tagging already tagged part {}. This is a bug.", part_to_export->name); -} - -CurrentlyExportingPartsTagger::~CurrentlyExportingPartsTagger() -{ - std::lock_guard lock(storage.currently_processing_in_background_mutex); - - if (!storage.currently_merging_mutating_parts.contains(part_to_export)) - std::terminate(); - storage.currently_merging_mutating_parts.erase(part_to_export); - - storage.currently_processing_in_background_condition.notify_all(); -} - - Int64 StorageMergeTree::startMutation(const MutationCommands & commands, ContextPtr query_context) { /// Choose any disk, because when we load mutations we search them at each disk @@ -1150,6 +1123,7 @@ void StorageMergeTree::loadMutations() void StorageMergeTree::readExportPartitionManifests() { + static const auto states = {MergeTreeDataPartState::Active, MergeTreeDataPartState::Deleting, MergeTreeDataPartState::Outdated, MergeTreeDataPartState::DeleteOnDestroy}; for (const auto & disk : getDisks()) { for (auto it = disk->iterateDirectory(relative_data_path); it->isValid(); it->next()) @@ -1177,6 +1151,28 @@ void StorageMergeTree::readExportPartitionManifests() } } + for (auto & item : manifest->items) + { + /// if this part has not been pushed yet + if (item.remote_path.empty()) + { + item.part = getPartIfExists(item.part_name, states); + + if (!item.part) + { + LOG_ERROR(log, "Part {} is present in the manifest file {}, but not found in the storage {}", + item.part_name, + manifest->transaction_id, + getStorageID().getNameForLogs()); + + manifest->status = MergeTreeExportManifest::Status::failed; + manifest->write(); + already_exported_partition_ids.erase(manifest->partition_id); + continue; + } + } + } + export_partition_transaction_id_to_manifest.emplace(manifest->transaction_id, manifest); LOG_DEBUG(log, "Loaded export transaction manifest: {} (transaction_id: {})", name, manifest->transaction_id); @@ -1670,12 +1666,13 @@ bool StorageMergeTree::scheduleDataMovingJob(BackgroundJobsAssignee & assignee) continue; } + static const auto states = {MergeTreeDataPartState::Active, MergeTreeDataPartState::Deleting, MergeTreeDataPartState::Outdated, MergeTreeDataPartState::DeleteOnDestroy}; for (auto & item : manifest->items) { if (item.in_progress) continue; - auto part = getPartIfExists(item.part_name, {MergeTreeDataPartState::Active}); + auto part = getPartIfExists(item.part_name, states); if (!part) { LOG_ERROR(log, "Part {} is present in the manifest file {}, but not found in the storage {}", From 646a69fe8fd6dbe4d7ea3d89ade689bcf0e44297 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Sun, 7 Sep 2025 18:19:36 -0300 Subject: [PATCH 44/48] do not capture exception in importer sink --- ...ObjectStorageMergeTreePartImporterSink.cpp | 33 ++++++------------- ...geObjectStorageMergeTreePartImporterSink.h | 11 +++---- 2 files changed, 14 insertions(+), 30 deletions(-) diff --git a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.cpp b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.cpp index f027e4ac950c..bb2d6514956e 100644 --- a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.cpp +++ b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.cpp @@ -38,42 +38,29 @@ String StorageObjectStorageMergeTreePartImporterSink::getName() const void StorageObjectStorageMergeTreePartImporterSink::consume(Chunk & chunk) { - if (stats.status.code != 0) - return; - - try - { - sink->consume(chunk); - stats.read_bytes += chunk.bytes(); - stats.read_rows += chunk.getNumRows(); - - stats.status = ExecutionStatus(0, "Success"); - } catch (...) { - stats.status = ExecutionStatus(-1, "Error importing part"); - part_log(stats); - } + sink->consume(chunk); + stats.read_bytes += chunk.bytes(); + stats.read_rows += chunk.getNumRows(); } void StorageObjectStorageMergeTreePartImporterSink::onFinish() { - if (stats.status.code != 0) - { - sink->cancel(); - return; - } - sink->onFinish(); + if (const auto object_metadata = object_storage->tryGetObjectMetadata(stats.file_path)) { stats.bytes_on_disk = object_metadata->size_bytes; } + part_log(stats); } -void StorageObjectStorageMergeTreePartImporterSink::onException(std::exception_ptr) +void StorageObjectStorageMergeTreePartImporterSink::onException(std::exception_ptr exception) { - /// we should not reach here - std::terminate(); + sink->onException(exception); + + stats.status = ExecutionStatus(-1, "Error importing part"); + part_log(stats); } } diff --git a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.h b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.h index d5f1677c69ae..21f435a08e4b 100644 --- a/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.h +++ b/src/Storages/ObjectStorage/MergeTree/StorageObjectStorageMergeTreePartImporterSink.h @@ -2,13 +2,10 @@ #include #include -#include "Core/Settings.h" -#include "Disks/ObjectStorages/IObjectStorage.h" -#include "Disks/ObjectStorages/StoredObject.h" -#include "Formats/FormatFactory.h" -#include "IO/CompressionMethod.h" -#include "Processors/Formats/IOutputFormat.h" -#include "Storages/MergeTree/IMergeTreeDataPart.h" +#include +#include +#include +#include namespace DB { From 20244c35b7a9a578c3911a80f5d247f138a3d9d7 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 8 Sep 2025 08:57:26 -0300 Subject: [PATCH 45/48] disable file level parallelism --- src/Formats/FormatFactory.cpp | 8 ++++++-- .../MergeTree/ExportPartPlainMergeTreeTask.cpp | 14 +++++++++++++- 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index db22070bd7f6..8b1cd15c542c 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -568,8 +568,12 @@ OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible( auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); const Settings & settings = context->getSettingsRef(); - if (settings[Setting::output_format_parallel_formatting] && getCreators(name).supports_parallel_formatting - && !settings[Setting::output_format_json_array_of_rows]) + const bool parallel_formatting_enabled = settings[Setting::output_format_parallel_formatting]; + const bool supports_parallel_formatting = getCreators(name).supports_parallel_formatting; + + bool are_we_doing_parallel_formatting = parallel_formatting_enabled && supports_parallel_formatting && !settings[Setting::output_format_json_array_of_rows]; + + if (are_we_doing_parallel_formatting) { auto formatter_creator = [output_getter, sample, format_settings] (WriteBuffer & output) -> OutputFormatPtr { diff --git a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp index 7749a1044f83..d2ad84cb11e3 100644 --- a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp +++ b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp @@ -5,10 +5,16 @@ #include #include #include +#include namespace DB { +namespace Setting +{ + extern const SettingsBool output_format_parallel_formatting; +} + ExportPartPlainMergeTreeTask::ExportPartPlainMergeTreeTask( StorageMergeTree & storage_, const DataPartPtr & part_to_export_, @@ -157,10 +163,16 @@ bool ExportPartPlainMergeTreeTask::executeExport() try { + auto context_copy = Context::createCopy(context); + + /// Manually disable parallelism because the idea is to control parallelism with tasks, not with formatting + context_copy->setSetting("output_format_parallel_formatting", false); + context_copy->setSetting("max_threads", 1); + destination_storage->importMergeTreePart( storage, part_to_export, - context, + context_copy, part_log_wrapper); return true; From 8f2557f020a7bba619716c39cbf91b8a11317602 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 8 Sep 2025 09:07:27 -0300 Subject: [PATCH 46/48] set max_retries --- src/Core/ServerSettings.cpp | 1 + src/Storages/StorageMergeTree.cpp | 24 +++++++++++++++++++----- 2 files changed, 20 insertions(+), 5 deletions(-) diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index fc964e21892f..f904e8b50184 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -1064,6 +1064,7 @@ The policy on how to perform a scheduling of CPU slots specified by `concurrent_ See [Controlling behavior on server CPU overload](/operations/settings/server-overload) for more details. )", 0) \ DECLARE(Float, distributed_cache_keep_up_free_connections_ratio, 0.1f, "Soft limit for number of active connection distributed cache will try to keep free. After the number of free connections goes below distributed_cache_keep_up_free_connections_ratio * max_connections, connections with oldest activity will be closed until the number goes above the limit.", 0) \ + DECLARE(UInt64, export_merge_tree_partition_max_retries, 3, "Max number of retries for exporting merge tree partition. Currently not persisted across re-starts", 0) \ // clang-format on diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 83a1f46f09d6..a1515c0c4ca3 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -46,9 +46,10 @@ #include #include #include -#include "Core/BackgroundSchedulePool.h" -#include "Storages/MergeTree/MergeTreeDataPartState.h" -#include "Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h" +#include +#include +#include +#include #include #include #include @@ -84,6 +85,11 @@ namespace Setting extern const SettingsBool allow_experimental_export_merge_tree_partition; } +namespace ServerSetting +{ + extern const ServerSettingsUInt64 export_merge_tree_partition_max_retries; +} + namespace MergeTreeSetting { extern const MergeTreeSettingsBool allow_experimental_replacing_merge_with_cleanup; @@ -1685,10 +1691,18 @@ bool StorageMergeTree::scheduleDataMovingJob(BackgroundJobsAssignee & assignee) continue; } - auto task = std::make_shared(*this, part, destination_storage, getContext(), manifest, moves_assignee_trigger); + auto task = std::make_shared( + *this, + part, + destination_storage, + getContext(), + manifest, + moves_assignee_trigger, + getContext()->getServerSettings()[ServerSetting::export_merge_tree_partition_max_retries]); item.in_progress = background_moves_assignee.scheduleMoveTask(task); - return true; + /// todo arthur is returning true always correct? + return true; } } } From ff551f6bfec5cc4611ce504e6a681d508851aa87 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 8 Sep 2025 09:51:59 -0300 Subject: [PATCH 47/48] exports throtler --- src/Core/ServerSettings.cpp | 1 + src/Interpreters/Context.cpp | 11 +++++++++ src/Interpreters/Context.h | 1 + .../MergeTree/MergeTreeSequentialSource.cpp | 3 +++ .../MergeTree/MergeTreeSequentialSource.h | 1 + .../ExportPartPlainMergeTreeTask.cpp | 5 ---- .../MergeTree/ExportPartPlainMergeTreeTask.h | 2 +- .../ObjectStorage/StorageObjectStorage.cpp | 23 ++++++++++++++----- 8 files changed, 35 insertions(+), 12 deletions(-) diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index f904e8b50184..c8986e3053c6 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -99,6 +99,7 @@ namespace DB DECLARE(UInt64, max_unexpected_parts_loading_thread_pool_size, 8, R"(The number of threads to load inactive set of data parts (Unexpected ones) at startup.)", 0) \ DECLARE(UInt64, max_parts_cleaning_thread_pool_size, 128, R"(The number of threads for concurrent removal of inactive data parts.)", 0) \ DECLARE(UInt64, max_mutations_bandwidth_for_server, 0, R"(The maximum read speed of all mutations on server in bytes per second. Zero means unlimited.)", 0) \ + DECLARE(UInt64, max_exports_bandwidth_for_server, 0, R"(The maximum read speed of all exports on server in bytes per second. Zero means unlimited.)", 0) \ DECLARE(UInt64, max_merges_bandwidth_for_server, 0, R"(The maximum read speed of all merges on server in bytes per second. Zero means unlimited.)", 0) \ DECLARE(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, R"(The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.)", 0) \ DECLARE(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, R"(The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.)", 0) \ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index db58e2352267..f226ecb98972 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -288,6 +288,7 @@ namespace ServerSetting extern const ServerSettingsUInt64 max_local_write_bandwidth_for_server; extern const ServerSettingsUInt64 max_merges_bandwidth_for_server; extern const ServerSettingsUInt64 max_mutations_bandwidth_for_server; + extern const ServerSettingsUInt64 max_exports_bandwidth_for_server; extern const ServerSettingsUInt64 max_remote_read_network_bandwidth_for_server; extern const ServerSettingsUInt64 max_remote_write_network_bandwidth_for_server; extern const ServerSettingsUInt64 max_replicated_fetches_network_bandwidth_for_server; @@ -504,6 +505,8 @@ struct ContextSharedPart : boost::noncopyable mutable ThrottlerPtr mutations_throttler; /// A server-wide throttler for mutations mutable ThrottlerPtr merges_throttler; /// A server-wide throttler for merges + mutable ThrottlerPtr exports_throttler; /// A server-wide throttler for exports + MultiVersion macros; /// Substitutions extracted from config. std::unique_ptr ddl_worker TSA_GUARDED_BY(mutex); /// Process ddl commands from zk. LoadTaskPtr ddl_worker_startup_task; /// To postpone `ddl_worker->startup()` after all tables startup @@ -992,6 +995,9 @@ struct ContextSharedPart : boost::noncopyable if (auto bandwidth = server_settings[ServerSetting::max_merges_bandwidth_for_server]) merges_throttler = std::make_shared(bandwidth); + + if (auto bandwidth = server_settings[ServerSetting::max_exports_bandwidth_for_server]) + exports_throttler = std::make_shared(bandwidth); } }; @@ -4041,6 +4047,11 @@ ThrottlerPtr Context::getMergesThrottler() const return shared->merges_throttler; } +ThrottlerPtr Context::getExportsThrottler() const +{ + return shared->exports_throttler; +} + void Context::reloadRemoteThrottlerConfig(size_t read_bandwidth, size_t write_bandwidth) const { if (read_bandwidth) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f26899ea3c72..bff74c55bc11 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1635,6 +1635,7 @@ class Context: public ContextData, public std::enable_shared_from_this ThrottlerPtr getMutationsThrottler() const; ThrottlerPtr getMergesThrottler() const; + ThrottlerPtr getExportsThrottler() const; void reloadRemoteThrottlerConfig(size_t read_bandwidth, size_t write_bandwidth) const; void reloadLocalThrottlerConfig(size_t read_bandwidth, size_t write_bandwidth) const; diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 9335e08fa4c2..d74a04744a11 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -140,6 +140,9 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( case Merge: read_settings.local_throttler = context->getMergesThrottler(); break; + case Export: + read_settings.local_throttler = context->getExportsThrottler(); + break; } read_settings.remote_throttler = read_settings.local_throttler; diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.h b/src/Storages/MergeTree/MergeTreeSequentialSource.h index abba230d9e79..a858adf33bb5 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.h +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -15,6 +15,7 @@ enum MergeTreeSequentialSourceType { Mutation, Merge, + Export, }; /// Create stream for reading single part from MergeTree. diff --git a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp index d2ad84cb11e3..e58693291115 100644 --- a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp +++ b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.cpp @@ -10,11 +10,6 @@ namespace DB { -namespace Setting -{ - extern const SettingsBool output_format_parallel_formatting; -} - ExportPartPlainMergeTreeTask::ExportPartPlainMergeTreeTask( StorageMergeTree & storage_, const DataPartPtr & part_to_export_, diff --git a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h index a81addfbc5be..cc1b40e60a94 100644 --- a/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h +++ b/src/Storages/ObjectStorage/MergeTree/ExportPartPlainMergeTreeTask.h @@ -23,7 +23,7 @@ class ExportPartPlainMergeTreeTask : public IExecutableTask ContextPtr context_, std::shared_ptr manifest_, IExecutableTask::TaskResultCallback & task_result_callback_, - size_t max_retries_ = 3); + size_t max_retries_); void onCompleted() override; bool executeStep() override; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 1508527c1c5f..eba74d5758e0 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -565,20 +565,31 @@ void StorageObjectStorage::importMergeTreePart( QueryPlan plan; - /// using the mutations type for now - MergeTreeSequentialSourceType read_type = MergeTreeSequentialSourceType::Mutation; + /// using the mutations type for now. This impacts in the throttling strategy + MergeTreeSequentialSourceType read_type = MergeTreeSequentialSourceType::Export; + /// todo implement these settings bool apply_deleted_mask = true; bool read_with_direct_io = false; bool prefetch = false; - const auto partition_columns = configuration->partition_strategy->getPartitionColumns(); + std::string partition_key; - auto block_with_partition_values = data_part->partition.getBlockWithPartitionValues(partition_columns); + if (configuration->partition_strategy) + { + const auto partition_columns = configuration->partition_strategy->getPartitionColumns(); + + auto block_with_partition_values = data_part->partition.getBlockWithPartitionValues(partition_columns); + + const auto column_with_partition_key = configuration->partition_strategy->computePartitionKey(block_with_partition_values); - const auto column_with_partition_key = configuration->partition_strategy->computePartitionKey(block_with_partition_values); + if (!column_with_partition_key->empty()) + { + partition_key = column_with_partition_key->getDataAt(0).toString(); + } + } - const auto file_path = configuration->file_path_generator->getWritingPath(column_with_partition_key->getDataAt(0).toString()); + const auto file_path = configuration->file_path_generator->getWritingPath(partition_key); MergeTreeData::IMutationsSnapshot::Params params { From bad3bc08098d264fb4e80dd24cecdc462c9962b3 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Mon, 8 Sep 2025 10:45:58 -0300 Subject: [PATCH 48/48] add comment --- src/Storages/StorageMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a1515c0c4ca3..c17140155a30 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2996,7 +2996,7 @@ MutationCounters StorageMergeTree::getMutationCounters() const void StorageMergeTree::startBackgroundMovesIfNeeded() { - // if (areBackgroundMovesNeeded()) + /// always starting it regardless of areBackgroundMovesNeeded() because we need it for exports background_moves_assignee.start(); }