diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 843d8ee7a72f..3ebf43b0e6ae 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -211,6 +211,7 @@ enum class AccessType : uint8_t M(ALTER_SETTINGS, "ALTER SETTING, ALTER MODIFY SETTING, MODIFY SETTING, RESET SETTING", TABLE, ALTER_TABLE) /* allows to execute ALTER MODIFY SETTING */\ M(ALTER_MOVE_PARTITION, "ALTER MOVE PART, MOVE PARTITION, MOVE PART", TABLE, ALTER_TABLE) \ M(ALTER_EXPORT_PART, "ALTER EXPORT PART, EXPORT PART", TABLE, ALTER_TABLE) \ + M(ALTER_EXPORT_PARTITION, "ALTER EXPORT PARTITION, EXPORT PARTITION", TABLE, ALTER_TABLE) \ M(ALTER_FETCH_PARTITION, "ALTER FETCH PART, FETCH PARTITION", TABLE, ALTER_TABLE) \ M(ALTER_FREEZE_PARTITION, "FREEZE PARTITION, UNFREEZE", TABLE, ALTER_TABLE) \ M(ALTER_UNLOCK_SNAPSHOT, "UNLOCK SNAPSHOT", TABLE, ALTER_TABLE) \ diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 8d0a31538dc3..28a660647541 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -1144,6 +1144,7 @@ The policy on how to perform a scheduling of CPU slots specified by `concurrent_ DECLARE(UInt64, object_storage_list_objects_cache_max_entries, 1000, "Maximum size of ObjectStorage list objects cache in entries. Zero means disabled.", 0) \ DECLARE(UInt64, object_storage_list_objects_cache_ttl, 3600, "Time to live of records in ObjectStorage list objects cache in seconds. Zero means unlimited", 0) \ DECLARE(UInt64, input_format_parquet_metadata_cache_max_size, 500000000, "Maximum size of parquet file metadata cache", 0) \ + DECLARE(Bool, enable_experimental_export_merge_tree_partition_feature, false, "Enable export replicated merge tree partition feature. It is experimental and not yet ready for production use.", 0) \ // clang-format on /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in dumpToSystemServerSettingsColumns below diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 4f8c2e9905eb..2c1b5498ba56 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -6873,6 +6873,22 @@ Use roaring bitmap for iceberg positional deletes. )", 0) \ DECLARE(Bool, export_merge_tree_part_overwrite_file_if_exists, false, R"( Overwrite file if it already exists when exporting a merge tree part +)", 0) \ + DECLARE(Bool, export_merge_tree_partition_force_export, false, R"( +Ignore existing partition export and overwrite the zookeeper entry +)", 0) \ + DECLARE(UInt64, export_merge_tree_partition_max_retries, 3, R"( +Maximum number of retries for exporting a merge tree part in an export partition task +)", 0) \ + DECLARE(UInt64, export_merge_tree_partition_manifest_ttl, 180, R"( +Determines how long the manifest will live in ZooKeeper. It prevents the same partition from being exported twice to the same destination. +This setting does not affect / delete in progress tasks. It'll only cleanup the completed ones. +)", 0) \ + DECLARE(MergeTreePartExportFileAlreadyExistsPolicy, export_merge_tree_part_file_already_exists_policy, MergeTreePartExportFileAlreadyExistsPolicy::skip, R"( +Possible values: +- skip - Skip the file if it already exists. +- error - Throw an error if the file already exists. +- overwrite - Overwrite the file. )", 0) \ DECLARE(Timezone, iceberg_timezone_for_timestamptz, "UTC", R"( Timezone for Iceberg timestamptz field. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 546c687e445a..c70767e21aca 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -81,6 +81,7 @@ class WriteBuffer; M(CLASS_NAME, LogsLevel) \ M(CLASS_NAME, Map) \ M(CLASS_NAME, MaxThreads) \ + M(CLASS_NAME, MergeTreePartExportFileAlreadyExistsPolicy) \ M(CLASS_NAME, Milliseconds) \ M(CLASS_NAME, MsgPackUUIDRepresentation) \ M(CLASS_NAME, MySQLDataTypesSupport) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 5f0a92e449b5..0d3105849722 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -47,6 +47,10 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"allow_retries_in_cluster_requests", false, false, "New setting"}, {"object_storage_remote_initiator", false, false, "New setting."}, {"allow_experimental_export_merge_tree_part", false, true, "Turned ON by default for Antalya."}, + {"export_merge_tree_partition_force_export", false, false, "New setting."}, + {"export_merge_tree_partition_max_retries", 3, 3, "New setting."}, + {"export_merge_tree_partition_manifest_ttl", 180, 180, "New setting."}, + {"export_merge_tree_part_file_already_exists_policy", "skip", "skip", "New setting."}, {"iceberg_timezone_for_timestamptz", "UTC", "UTC", "New setting."} }); addSettingsChanges(settings_changes_history, "25.8", diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 8e34fc296b1f..2fb4f1668ed4 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -370,4 +370,6 @@ IMPLEMENT_SETTING_ENUM( {"manifest_list_entry", IcebergMetadataLogLevel::ManifestListEntry}, {"manifest_file_metadata", IcebergMetadataLogLevel::ManifestFileMetadata}, {"manifest_file_entry", IcebergMetadataLogLevel::ManifestFileEntry}}) + +IMPLEMENT_SETTING_AUTO_ENUM(MergeTreePartExportFileAlreadyExistsPolicy, ErrorCodes::BAD_ARGUMENTS); } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index d4472e339edf..bb058974944f 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -480,4 +480,14 @@ enum class IcebergMetadataLogLevel : uint8_t }; DECLARE_SETTING_ENUM(IcebergMetadataLogLevel) + +enum class MergeTreePartExportFileAlreadyExistsPolicy : uint8_t +{ + skip, + error, + overwrite, +}; + +DECLARE_SETTING_ENUM(MergeTreePartExportFileAlreadyExistsPolicy) + } diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index 5055f548a0ee..d3c8137e6611 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -154,6 +154,11 @@ uint64_t generateSnowflakeID() return fromSnowflakeId(snowflake_id); } +std::string generateSnowflakeIDString() +{ + return std::to_string(generateSnowflakeID()); +} + class FunctionGenerateSnowflakeID : public IFunction { public: diff --git a/src/Functions/generateSnowflakeID.h b/src/Functions/generateSnowflakeID.h index 38fa684a9b4b..4fc173dcf1be 100644 --- a/src/Functions/generateSnowflakeID.h +++ b/src/Functions/generateSnowflakeID.h @@ -7,4 +7,6 @@ namespace DB uint64_t generateSnowflakeID(); +std::string generateSnowflakeIDString(); + } diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 988df44e7049..e280cd6318f9 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -545,6 +545,12 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS required_access.emplace_back(AccessType::INSERT, command.to_database, command.to_table); break; } + case ASTAlterCommand::EXPORT_PARTITION: + { + required_access.emplace_back(AccessType::ALTER_EXPORT_PARTITION, 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/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 51ea9051ddfc..4054b1e7211a 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -37,11 +38,17 @@ namespace Setting extern const SettingsUInt64 max_parser_depth; } +namespace ServerSetting +{ + extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; +} + namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int ACCESS_DENIED; extern const int NOT_IMPLEMENTED; + extern const int SUPPORT_IS_DISABLED; } @@ -250,6 +257,82 @@ BlockIO InterpreterKillQueryQuery::execute() break; } + case ASTKillQueryQuery::Type::ExportPartition: + { + if (!getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Exporting merge tree partition is experimental. Set the server setting `enable_experimental_export_merge_tree_partition_feature` to enable it"); + } + + Block exports_block = getSelectResult( + "source_database, source_table, transaction_id, destination_database, destination_table, partition_id", + "system.replicated_partition_exports"); + if (exports_block.empty()) + return res_io; + + const ColumnString & src_db_col = typeid_cast(*exports_block.getByName("source_database").column); + const ColumnString & src_table_col = typeid_cast(*exports_block.getByName("source_table").column); + const ColumnString & dst_db_col = typeid_cast(*exports_block.getByName("destination_database").column); + const ColumnString & dst_table_col = typeid_cast(*exports_block.getByName("destination_table").column); + const ColumnString & tx_col = typeid_cast(*exports_block.getByName("transaction_id").column); + + auto header = exports_block.cloneEmpty(); + header.insert(0, {ColumnString::create(), std::make_shared(), "kill_status"}); + + MutableColumns res_columns = header.cloneEmptyColumns(); + AccessRightsElements required_access_rights; + auto access = getContext()->getAccess(); + bool access_denied = false; + + for (size_t i = 0; i < exports_block.rows(); ++i) + { + const auto src_database = src_db_col.getDataAt(i).toString(); + const auto src_table = src_table_col.getDataAt(i).toString(); + const auto dst_database = dst_db_col.getDataAt(i).toView(); + const auto dst_table = dst_table_col.getDataAt(i).toView(); + + const auto table_id = StorageID{src_database, src_table}; + const auto transaction_id = tx_col.getDataAt(i).toString(); + + CancellationCode code = CancellationCode::Unknown; + if (!query.test) + { + auto storage = DatabaseCatalog::instance().tryGetTable(table_id, getContext()); + if (!storage) + code = CancellationCode::NotFound; + else + { + ASTAlterCommand alter_command{}; + alter_command.type = ASTAlterCommand::EXPORT_PARTITION; + alter_command.move_destination_type = DataDestinationType::TABLE; + alter_command.from_database = src_database; + alter_command.from_table = src_table; + alter_command.to_database = dst_database; + alter_command.to_table = dst_table; + + required_access_rights = InterpreterAlterQuery::getRequiredAccessForCommand( + alter_command, table_id.database_name, table_id.table_name); + if (!access->isGranted(required_access_rights)) + { + access_denied = true; + continue; + } + code = storage->killExportPartition(transaction_id); + } + } + + insertResultRow(i, code, exports_block, header, res_columns); + } + + if (res_columns[0]->empty() && access_denied) + throw Exception(ErrorCodes::ACCESS_DENIED, "Not allowed to kill export partition. " + "To execute this query, it's necessary to have the grant {}", required_access_rights.toString()); + + res_io.pipeline = QueryPipeline(Pipe(std::make_shared(std::make_shared(header.cloneWithColumns(std::move(res_columns)))))); + + break; + } case ASTKillQueryQuery::Type::Mutation: { Block mutations_block = getSelectResult("database, table, mutation_id, command", "system.mutations"); @@ -462,6 +545,9 @@ AccessRightsElements InterpreterKillQueryQuery::getRequiredAccessForDDLOnCluster | AccessType::ALTER_MATERIALIZE_COLUMN | AccessType::ALTER_MATERIALIZE_TTL ); + /// todo arthur think about this + else if (query.type == ASTKillQueryQuery::Type::ExportPartition) + required_access.emplace_back(AccessType::ALTER_EXPORT_PARTITION); return required_access; } diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 30b5f9dca156..57d81c92898f 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -378,6 +378,17 @@ void ASTAlterCommand::formatImpl(WriteBuffer & ostr, const FormatSettings & sett } } + else if (type == ASTAlterCommand::EXPORT_PARTITION) + { + ostr << "EXPORT PARTITION "; + partition->format(ostr, settings, state, frame); + ostr << " TO TABLE "; + if (!to_database.empty()) + { + ostr << backQuoteIfNeed(to_database) << "."; + } + ostr << backQuoteIfNeed(to_table); + } else if (type == ASTAlterCommand::REPLACE_PARTITION) { ostr << (replace ? "REPLACE" : "ATTACH") << " PARTITION " diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index d8d502cb87c6..7683b2e11c3d 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -72,6 +72,7 @@ class ASTAlterCommand : public IAST UNFREEZE_PARTITION, UNFREEZE_ALL, EXPORT_PART, + EXPORT_PARTITION, DELETE, UPDATE, diff --git a/src/Parsers/ASTKillQueryQuery.cpp b/src/Parsers/ASTKillQueryQuery.cpp index 0334b78d559e..9911e60b5ed9 100644 --- a/src/Parsers/ASTKillQueryQuery.cpp +++ b/src/Parsers/ASTKillQueryQuery.cpp @@ -27,6 +27,9 @@ void ASTKillQueryQuery::formatQueryImpl(WriteBuffer & ostr, const FormatSettings case Type::Transaction: ostr << "TRANSACTION"; break; + case Type::ExportPartition: + ostr << "EXPORT PARTITION"; + break; } formatOnCluster(ostr, settings); diff --git a/src/Parsers/ASTKillQueryQuery.h b/src/Parsers/ASTKillQueryQuery.h index 99a14c56d72b..13d2811534f0 100644 --- a/src/Parsers/ASTKillQueryQuery.h +++ b/src/Parsers/ASTKillQueryQuery.h @@ -13,6 +13,7 @@ class ASTKillQueryQuery : public ASTQueryWithOutput, public ASTQueryWithOnCluste { Query, /// KILL QUERY Mutation, /// KILL MUTATION + ExportPartition, /// KILL EXPORT_PARTITION PartMoveToShard, /// KILL PART_MOVE_TO_SHARD Transaction, /// KILL TRANSACTION }; diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 58694bde8984..c846a12eab02 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -333,6 +333,7 @@ namespace DB MR_MACROS(MOVE_PART, "MOVE PART") \ MR_MACROS(MOVE_PARTITION, "MOVE PARTITION") \ MR_MACROS(EXPORT_PART, "EXPORT PART") \ + MR_MACROS(EXPORT_PARTITION, "EXPORT PARTITION") \ MR_MACROS(MOVE, "MOVE") \ MR_MACROS(MS, "MS") \ MR_MACROS(MUTATION, "MUTATION") \ diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 775d495492cf..eff14253b97f 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -83,6 +83,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_move_partition(Keyword::MOVE_PARTITION); ParserKeyword s_move_part(Keyword::MOVE_PART); ParserKeyword s_export_part(Keyword::EXPORT_PART); + ParserKeyword s_export_partition(Keyword::EXPORT_PARTITION); ParserKeyword s_drop_detached_partition(Keyword::DROP_DETACHED_PARTITION); ParserKeyword s_drop_detached_part(Keyword::DROP_DETACHED_PART); ParserKeyword s_fetch_partition(Keyword::FETCH_PARTITION); @@ -553,6 +554,22 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected return false; command->move_destination_type = DataDestinationType::TABLE; } + else if (s_export_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command_partition, expected)) + return false; + + command->type = ASTAlterCommand::EXPORT_PARTITION; + + if (!s_to_table.ignore(pos, expected)) + { + return false; + } + + if (!parseDatabaseAndTableName(pos, expected, command->to_database, command->to_table)) + return false; + command->move_destination_type = DataDestinationType::TABLE; + } else if (s_move_partition.ignore(pos, expected)) { if (!parser_partition.parse(pos, command_partition, expected)) diff --git a/src/Parsers/ParserKillQueryQuery.cpp b/src/Parsers/ParserKillQueryQuery.cpp index 55bd5100009e..7e06ae8d30b7 100644 --- a/src/Parsers/ParserKillQueryQuery.cpp +++ b/src/Parsers/ParserKillQueryQuery.cpp @@ -17,6 +17,7 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect ParserKeyword p_kill{Keyword::KILL}; ParserKeyword p_query{Keyword::QUERY}; ParserKeyword p_mutation{Keyword::MUTATION}; + ParserKeyword p_export_partition{Keyword::EXPORT_PARTITION}; ParserKeyword p_part_move_to_shard{Keyword::PART_MOVE_TO_SHARD}; ParserKeyword p_transaction{Keyword::TRANSACTION}; ParserKeyword p_on{Keyword::ON}; @@ -33,6 +34,8 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect query->type = ASTKillQueryQuery::Type::Query; else if (p_mutation.ignore(pos, expected)) query->type = ASTKillQueryQuery::Type::Mutation; + else if (p_export_partition.ignore(pos, expected)) + query->type = ASTKillQueryQuery::Type::ExportPartition; else if (p_part_move_to_shard.ignore(pos, expected)) query->type = ASTKillQueryQuery::Type::PartMoveToShard; else if (p_transaction.ignore(pos, expected)) diff --git a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h new file mode 100644 index 000000000000..81f61b5b9f12 --- /dev/null +++ b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h @@ -0,0 +1,179 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +struct ExportReplicatedMergeTreePartitionProcessingPartEntry +{ + + enum class Status + { + PENDING, + COMPLETED, + FAILED + }; + + String part_name; + Status status; + size_t retry_count; + String finished_by; + + std::string toJsonString() const + { + Poco::JSON::Object json; + + json.set("part_name", part_name); + json.set("status", String(magic_enum::enum_name(status))); + json.set("retry_count", retry_count); + json.set("finished_by", finished_by); + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + + return oss.str(); + } + + static ExportReplicatedMergeTreePartitionProcessingPartEntry fromJsonString(const std::string & json_string) + { + Poco::JSON::Parser parser; + auto json = parser.parse(json_string).extract(); + chassert(json); + + ExportReplicatedMergeTreePartitionProcessingPartEntry entry; + + entry.part_name = json->getValue("part_name"); + entry.status = magic_enum::enum_cast(json->getValue("status")).value(); + entry.retry_count = json->getValue("retry_count"); + if (json->has("finished_by")) + { + entry.finished_by = json->getValue("finished_by"); + } + return entry; + } +}; + +struct ExportReplicatedMergeTreePartitionProcessedPartEntry +{ + String part_name; + String path_in_destination; + String finished_by; + + std::string toJsonString() const + { + Poco::JSON::Object json; + json.set("part_name", part_name); + json.set("path_in_destination", path_in_destination); + json.set("finished_by", finished_by); + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + return oss.str(); + } + + static ExportReplicatedMergeTreePartitionProcessedPartEntry fromJsonString(const std::string & json_string) + { + Poco::JSON::Parser parser; + auto json = parser.parse(json_string).extract(); + chassert(json); + + ExportReplicatedMergeTreePartitionProcessedPartEntry entry; + + entry.part_name = json->getValue("part_name"); + entry.path_in_destination = json->getValue("path_in_destination"); + entry.finished_by = json->getValue("finished_by"); + + return entry; + } +}; + +struct ExportReplicatedMergeTreePartitionManifest +{ + String transaction_id; + String partition_id; + String destination_database; + String destination_table; + String source_replica; + size_t number_of_parts; + std::vector parts; + time_t create_time; + size_t max_retries; + size_t ttl_seconds; + size_t max_threads; + bool parallel_formatting; + bool parquet_parallel_encoding; + MergeTreePartExportManifest::FileAlreadyExistsPolicy file_already_exists_policy; + + std::string toJsonString() const + { + Poco::JSON::Object json; + json.set("transaction_id", transaction_id); + json.set("partition_id", partition_id); + json.set("destination_database", destination_database); + json.set("destination_table", destination_table); + json.set("source_replica", source_replica); + json.set("number_of_parts", number_of_parts); + + Poco::JSON::Array::Ptr parts_array = new Poco::JSON::Array(); + for (const auto & part : parts) + parts_array->add(part); + json.set("parts", parts_array); + json.set("parallel_formatting", parallel_formatting); + json.set("max_threads", max_threads); + json.set("parquet_parallel_encoding", parquet_parallel_encoding); + json.set("file_already_exists_policy", String(magic_enum::enum_name(file_already_exists_policy))); + json.set("create_time", create_time); + json.set("max_retries", max_retries); + json.set("ttl_seconds", ttl_seconds); + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + return oss.str(); + } + + static ExportReplicatedMergeTreePartitionManifest fromJsonString(const std::string & json_string) + { + Poco::JSON::Parser parser; + auto json = parser.parse(json_string).extract(); + chassert(json); + + ExportReplicatedMergeTreePartitionManifest manifest; + manifest.transaction_id = json->getValue("transaction_id"); + manifest.partition_id = json->getValue("partition_id"); + manifest.destination_database = json->getValue("destination_database"); + manifest.destination_table = json->getValue("destination_table"); + manifest.source_replica = json->getValue("source_replica"); + manifest.number_of_parts = json->getValue("number_of_parts"); + manifest.max_retries = json->getValue("max_retries"); + auto parts_array = json->getArray("parts"); + for (size_t i = 0; i < parts_array->size(); ++i) + manifest.parts.push_back(parts_array->getElement(static_cast(i))); + + manifest.create_time = json->getValue("create_time"); + manifest.ttl_seconds = json->getValue("ttl_seconds"); + manifest.max_threads = json->getValue("max_threads"); + manifest.parallel_formatting = json->getValue("parallel_formatting"); + manifest.parquet_parallel_encoding = json->getValue("parquet_parallel_encoding"); + + if (json->has("file_already_exists_policy")) + { + const auto file_already_exists_policy = magic_enum::enum_cast(json->getValue("file_already_exists_policy")); + if (file_already_exists_policy) + { + manifest.file_already_exists_policy = file_already_exists_policy.value(); + } + + /// what to do if it's not a valid value? + } + + return manifest; + } +}; + +} diff --git a/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h new file mode 100644 index 000000000000..76674bfc4a92 --- /dev/null +++ b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h @@ -0,0 +1,78 @@ +#pragma once + +#include +#include +#include "Core/QualifiedTableName.h" +#include +#include +#include +#include + +namespace DB +{ +struct ExportReplicatedMergeTreePartitionTaskEntry +{ + using DataPartPtr = std::shared_ptr; + ExportReplicatedMergeTreePartitionManifest manifest; + + enum class Status + { + PENDING, + COMPLETED, + FAILED, + KILLED + }; + + /// Allows us to skip completed / failed entries during scheduling + mutable Status status; + + /// References to the parts that should be exported + /// This is used to prevent the parts from being deleted before finishing the export operation + /// It does not mean this replica will export all the parts + /// There is also a chance this replica does not contain a given part and it is totally ok. + std::vector part_references; + + std::string getCompositeKey() const + { + const auto qualified_table_name = QualifiedTableName {manifest.destination_database, manifest.destination_table}; + return manifest.partition_id + "_" + qualified_table_name.getFullName(); + } + + std::string getTransactionId() const + { + return manifest.transaction_id; + } + + /// Get create_time for sorted iteration + time_t getCreateTime() const + { + return manifest.create_time; + } +}; + +struct ExportPartitionTaskEntryTagByCompositeKey {}; +struct ExportPartitionTaskEntryTagByCreateTime {}; +struct ExportPartitionTaskEntryTagByTransactionId {}; + +// Multi-index container for export partition task entries +// - Index 0 (TagByCompositeKey): hashed_unique on composite key for O(1) lookup +// - Index 1 (TagByCreateTime): ordered_non_unique on create_time for sorted iteration +using ExportPartitionTaskEntriesContainer = boost::multi_index_container< + ExportReplicatedMergeTreePartitionTaskEntry, + boost::multi_index::indexed_by< + boost::multi_index::hashed_unique< + boost::multi_index::tag, + boost::multi_index::const_mem_fun + >, + boost::multi_index::ordered_non_unique< + boost::multi_index::tag, + boost::multi_index::const_mem_fun + >, + boost::multi_index::hashed_unique< + boost::multi_index::tag, + boost::multi_index::const_mem_fun + > + > +>; + +} diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 215c832624c1..3766afafe6b9 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -321,6 +321,11 @@ CancellationCode IStorage::killPartMoveToShard(const UUID & /*task_uuid*/) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Part moves between shards are not supported by storage {}", getName()); } +CancellationCode IStorage::killExportPartition(const String & /*transaction_id*/) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Export partition is not supported by storage {}", getName()); +} + StorageID IStorage::getStorageID() const { std::lock_guard lock(id_mutex); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 334dc7bdfc8f..15938d9f3c22 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -473,6 +473,15 @@ It is currently only implemented in StorageObjectStorage. { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Import is not implemented for storage {}", getName()); } + + virtual void commitExportPartitionTransaction( + const String & /* transaction_id */, + const String & /* partition_id */, + const Strings & /* exported_paths */, + ContextPtr /* local_context */) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "commitExportPartitionTransaction is not implemented for storage type {}", getName()); + } /** Writes the data to a table in distributed manner. @@ -583,6 +592,9 @@ It is currently only implemented in StorageObjectStorage. virtual void setMutationCSN(const String & /*mutation_id*/, UInt64 /*csn*/); + /// Cancel a replicated partition export by transaction id. + virtual CancellationCode killExportPartition(const String & /*transaction_id*/); + /// Cancel a part move to shard. virtual CancellationCode killPartMoveToShard(const UUID & /*task_uuid*/); diff --git a/src/Storages/MergeTree/ExportList.h b/src/Storages/MergeTree/ExportList.h index ade18b69480c..3c4daa07737b 100644 --- a/src/Storages/MergeTree/ExportList.h +++ b/src/Storages/MergeTree/ExportList.h @@ -41,7 +41,7 @@ struct ExportsListElement : private boost::noncopyable const StorageID destination_table_id; const UInt64 part_size; const String part_name; - const String destination_file_path; + String destination_file_path; UInt64 rows_read {0}; UInt64 total_rows_to_read {0}; UInt64 total_size_bytes_compressed {0}; diff --git a/src/Storages/MergeTree/ExportPartTask.cpp b/src/Storages/MergeTree/ExportPartTask.cpp new file mode 100644 index 000000000000..654e803f035b --- /dev/null +++ b/src/Storages/MergeTree/ExportPartTask.cpp @@ -0,0 +1,291 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace ProfileEvents +{ + extern const Event PartsExportDuplicated; + extern const Event PartsExportFailures; + extern const Event PartsExports; + extern const Event PartsExportTotalMilliseconds; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_TABLE; + extern const int FILE_ALREADY_EXISTS; + extern const int LOGICAL_ERROR; + extern const int QUERY_WAS_CANCELLED; +} + +namespace Setting +{ + extern const SettingsUInt64 min_bytes_to_use_direct_io; +} + +ExportPartTask::ExportPartTask(MergeTreeData & storage_, const MergeTreePartExportManifest & manifest_, ContextPtr context_) + : storage(storage_), + manifest(manifest_), + local_context(context_) +{ +} + +bool ExportPartTask::executeStep() +{ + auto metadata_snapshot = storage.getInMemoryMetadataPtr(); + Names columns_to_read = metadata_snapshot->getColumns().getNamesOfPhysical(); + StorageSnapshotPtr storage_snapshot = storage.getStorageSnapshot(metadata_snapshot, local_context); + + MergeTreeSequentialSourceType read_type = MergeTreeSequentialSourceType::Export; + + Block block_with_partition_values; + if (metadata_snapshot->hasPartitionKey()) + { + /// todo arthur do I need to init minmax_idx? + block_with_partition_values = manifest.data_part->minmax_idx->getBlock(storage); + } + + auto destination_storage = DatabaseCatalog::instance().tryGetTable(manifest.destination_storage_id, local_context); + if (!destination_storage) + { + std::lock_guard inner_lock(storage.export_manifests_mutex); + + const auto destination_storage_id_name = manifest.destination_storage_id.getNameForLogs(); + storage.export_manifests.erase(manifest); + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Failed to reconstruct destination storage: {}", destination_storage_id_name); + } + + auto exports_list_entry = storage.getContext()->getExportsList().insert( + getStorageID(), + manifest.destination_storage_id, + manifest.data_part->getBytesOnDisk(), + manifest.data_part->name, + "not_computed_yet", + manifest.data_part->rows_count, + manifest.data_part->getBytesOnDisk(), + manifest.data_part->getBytesUncompressedOnDisk(), + manifest.create_time, + local_context); + + SinkToStoragePtr sink; + + try + { + sink = destination_storage->import( + manifest.data_part->name + "_" + manifest.data_part->checksums.getTotalChecksumHex(), + block_with_partition_values, + (*exports_list_entry)->destination_file_path, + manifest.file_already_exists_policy == MergeTreePartExportManifest::FileAlreadyExistsPolicy::overwrite, + manifest.format_settings, + local_context); + } + catch (const Exception & e) + { + if (e.code() == ErrorCodes::FILE_ALREADY_EXISTS) + { + ProfileEvents::increment(ProfileEvents::PartsExportDuplicated); + + /// File already exists and the policy is NO_OP, treat it as success. + if (manifest.file_already_exists_policy == MergeTreePartExportManifest::FileAlreadyExistsPolicy::skip) + { + storage.writePartLog( + PartLogElement::Type::EXPORT_PART, + {}, + static_cast((*exports_list_entry)->elapsed * 1000000000), + manifest.data_part->name, + manifest.data_part, + {manifest.data_part}, + nullptr, + nullptr, + exports_list_entry.get()); + + std::lock_guard inner_lock(storage.export_manifests_mutex); + storage.export_manifests.erase(manifest); + + ProfileEvents::increment(ProfileEvents::PartsExports); + ProfileEvents::increment(ProfileEvents::PartsExportTotalMilliseconds, static_cast((*exports_list_entry)->elapsed * 1000)); + + if (manifest.completion_callback) + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createSuccess((*exports_list_entry)->destination_file_path)); + return false; + } + } + + tryLogCurrentException(__PRETTY_FUNCTION__); + + ProfileEvents::increment(ProfileEvents::PartsExportFailures); + + std::lock_guard inner_lock(storage.export_manifests_mutex); + storage.export_manifests.erase(manifest); + + if (manifest.completion_callback) + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createFailure(e)); + return false; + } + + bool apply_deleted_mask = true; + bool read_with_direct_io = local_context->getSettingsRef()[Setting::min_bytes_to_use_direct_io] > manifest.data_part->getBytesOnDisk(); + bool prefetch = false; + + MergeTreeData::IMutationsSnapshot::Params params + { + .metadata_version = metadata_snapshot->getMetadataVersion(), + .min_part_metadata_version = manifest.data_part->getMetadataVersion(), + }; + + auto mutations_snapshot = storage.getMutationsSnapshot(params); + + auto alter_conversions = MergeTreeData::getAlterConversionsForPart( + manifest.data_part, + mutations_snapshot, + local_context); + + QueryPlan plan_for_part; + + createReadFromPartStep( + read_type, + plan_for_part, + storage, + storage_snapshot, + RangesInDataPart(manifest.data_part), + alter_conversions, + nullptr, + columns_to_read, + nullptr, + apply_deleted_mask, + std::nullopt, + read_with_direct_io, + prefetch, + local_context, + getLogger("ExportPartition")); + + + ThreadGroupSwitcher switcher((*exports_list_entry)->thread_group, ""); + + QueryPlanOptimizationSettings optimization_settings(local_context); + auto pipeline_settings = BuildQueryPipelineSettings(local_context); + auto builder = plan_for_part.buildQueryPipeline(optimization_settings, pipeline_settings); + + builder->setProgressCallback([&exports_list_entry](const Progress & progress) + { + (*exports_list_entry)->bytes_read_uncompressed += progress.read_bytes; + (*exports_list_entry)->rows_read += progress.read_rows; + (*exports_list_entry)->elapsed = (*exports_list_entry)->watch.elapsedSeconds(); + }); + + pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + + pipeline.complete(sink); + + try + { + CompletedPipelineExecutor exec(pipeline); + + auto is_cancelled_callback = [this]() + { + return isCancelled(); + }; + + exec.setCancelCallback(is_cancelled_callback, 100); + + exec.execute(); + + if (isCancelled()) + { + throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Export part was cancelled"); + } + + std::lock_guard inner_lock(storage.export_manifests_mutex); + storage.writePartLog( + PartLogElement::Type::EXPORT_PART, + {}, + static_cast((*exports_list_entry)->elapsed * 1000000000), + manifest.data_part->name, + manifest.data_part, + {manifest.data_part}, + nullptr, + nullptr, + exports_list_entry.get()); + + storage.export_manifests.erase(manifest); + + ProfileEvents::increment(ProfileEvents::PartsExports); + ProfileEvents::increment(ProfileEvents::PartsExportTotalMilliseconds, static_cast((*exports_list_entry)->elapsed * 1000)); + + if (manifest.completion_callback) + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createSuccess((*exports_list_entry)->destination_file_path)); + } + catch (const Exception & e) + { + tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("while exporting the part {}. User should retry.", manifest.data_part->name)); + + ProfileEvents::increment(ProfileEvents::PartsExportFailures); + + std::lock_guard inner_lock(storage.export_manifests_mutex); + storage.writePartLog( + PartLogElement::Type::EXPORT_PART, + ExecutionStatus::fromCurrentException("", true), + static_cast((*exports_list_entry)->elapsed * 1000000000), + manifest.data_part->name, + manifest.data_part, + {manifest.data_part}, + nullptr, + nullptr, + exports_list_entry.get()); + + storage.export_manifests.erase(manifest); + + if (manifest.completion_callback) + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createFailure(e)); + + throw; + } + return false; +} + +void ExportPartTask::cancel() noexcept +{ + cancel_requested.store(true); + pipeline.cancel(); +} + +bool ExportPartTask::isCancelled() const +{ + return cancel_requested.load() || storage.parts_mover.moves_blocker.isCancelled(); +} + +void ExportPartTask::onCompleted() +{ +} + +StorageID ExportPartTask::getStorageID() const +{ + return storage.getStorageID(); +} + +Priority ExportPartTask::getPriority() const +{ + return Priority{}; +} + +String ExportPartTask::getQueryId() const +{ + return manifest.transaction_id; +} + +} diff --git a/src/Storages/MergeTree/ExportPartTask.h b/src/Storages/MergeTree/ExportPartTask.h new file mode 100644 index 000000000000..bcec68b2b737 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartTask.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class ExportPartTask : public IExecutableTask +{ +public: + explicit ExportPartTask( + MergeTreeData & storage_, + const MergeTreePartExportManifest & manifest_, + ContextPtr context_); + bool executeStep() override; + void onCompleted() override; + StorageID getStorageID() const override; + Priority getPriority() const override; + String getQueryId() const override; + + void cancel() noexcept override; + +private: + MergeTreeData & storage; + MergeTreePartExportManifest manifest; + ContextPtr local_context; + QueryPipeline pipeline; + std::atomic cancel_requested = false; + + bool isCancelled() const; +}; + +} diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp new file mode 100644 index 000000000000..79b92663b7bf --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp @@ -0,0 +1,310 @@ +#include +#include +#include +#include "Storages/MergeTree/ExportPartitionUtils.h" +#include "Common/logger_useful.h" +#include +#include +#include + +namespace DB +{ +namespace +{ + /* + Remove expired entries and fix non-committed exports that have already exported all parts. + + Return values: + - true: the cleanup was successful, the entry is removed from the entries_by_key container and the function returns true. Proceed to the next entry. + - false: the cleanup was not successful, the entry is not removed from the entries_by_key container and the function returns false. + */ + bool tryCleanup( + const zkutil::ZooKeeperPtr & zk, + const std::string & entry_path, + const LoggerPtr & log, + const ContextPtr & context, + const std::string & key, + const ExportReplicatedMergeTreePartitionManifest & metadata, + const time_t now, + const bool is_pending, + auto & entries_by_key + ) + { + bool has_expired = metadata.create_time < now - static_cast(metadata.ttl_seconds); + + if (has_expired && !is_pending) + { + zk->tryRemoveRecursive(fs::path(entry_path)); + auto it = entries_by_key.find(key); + if (it != entries_by_key.end()) + entries_by_key.erase(it); + LOG_INFO(log, "ExportPartition Manifest Updating Task: Removed {}: expired", key); + + return true; + } + else if (is_pending) + { + std::vector parts_in_processing_or_pending; + if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(entry_path) / "processing", parts_in_processing_or_pending)) + { + LOG_INFO(log, "ExportPartition Manifest Updating Task: Failed to get parts in processing or pending, skipping"); + return false; + } + + if (parts_in_processing_or_pending.empty()) + { + LOG_INFO(log, "ExportPartition Manifest Updating Task: Cleanup found PENDING for {} with all parts exported, try to fix it by committing the export", entry_path); + + const auto destination_storage_id = StorageID(QualifiedTableName {metadata.destination_database, metadata.destination_table}); + const auto destination_storage = DatabaseCatalog::instance().tryGetTable(destination_storage_id, context); + if (!destination_storage) + { + LOG_INFO(log, "ExportPartition Manifest Updating Task: Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); + return false; + } + + /// it sounds like a replica exported the last part, but was not able to commit the export. Try to fix it + ExportPartitionUtils::commit(metadata, destination_storage, zk, log, entry_path, context); + + return true; + } + } + + return false; + } +} + +ExportPartitionManifestUpdatingTask::ExportPartitionManifestUpdatingTask(StorageReplicatedMergeTree & storage_) + : storage(storage_) +{ +} + +void ExportPartitionManifestUpdatingTask::poll() +{ + std::lock_guard lock(storage.export_merge_tree_partition_mutex); + + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Polling for new entries for table {}. Current number of entries: {}", storage.getStorageID().getNameForLogs(), storage.export_merge_tree_partition_task_entries_by_key.size()); + + auto zk = storage.getZooKeeper(); + + const std::string exports_path = fs::path(storage.zookeeper_path) / "exports"; + const std::string cleanup_lock_path = fs::path(storage.zookeeper_path) / "exports_cleanup_lock"; + + auto cleanup_lock = zkutil::EphemeralNodeHolder::tryCreate(cleanup_lock_path, *zk, storage.replica_name); + if (cleanup_lock) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Cleanup lock acquired, will remove stale entries"); + } + + Coordination::Stat stat; + const auto children = zk->getChildrenWatch(exports_path, &stat, storage.export_merge_tree_partition_watch_callback); + const std::unordered_set zk_children(children.begin(), children.end()); + + const auto now = time(nullptr); + + auto & entries_by_key = storage.export_merge_tree_partition_task_entries_by_key; + + /// Load new entries + /// If we have the cleanup lock, also remove stale entries from zk and local + /// Upload dangling commit files if any + for (const auto & key : zk_children) + { + const std::string entry_path = fs::path(exports_path) / key; + + std::string metadata_json; + if (!zk->tryGet(fs::path(entry_path) / "metadata.json", metadata_json)) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: missing metadata.json", key); + continue; + } + + const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + + const auto local_entry = entries_by_key.find(key); + + /// If the zk entry has been replaced with export_merge_tree_partition_force_export, checking only for the export key is not enough + /// we need to make sure it is the same transaction id. If it is not, it needs to be replaced. + bool has_local_entry_and_is_up_to_date = local_entry != entries_by_key.end() + && local_entry->manifest.transaction_id == metadata.transaction_id; + + /// If the entry is up to date and we don't have the cleanup lock, early exit, nothing to be done. + if (!cleanup_lock && has_local_entry_and_is_up_to_date) + continue; + + std::weak_ptr weak_manifest_updater = storage.export_merge_tree_partition_manifest_updater; + + auto status_watch_callback = std::make_shared([weak_manifest_updater, key](const Coordination::WatchResponse &) + { + /// If the table is dropped but the watch is not removed, we need to prevent use after free + /// below code assumes that if manifest updater is still alive, the status handling task is also alive + if (auto manifest_updater = weak_manifest_updater.lock()) + { + manifest_updater->addStatusChange(key); + manifest_updater->storage.export_merge_tree_partition_status_handling_task->schedule(); + } + }); + + std::string status; + if (!zk->tryGetWatch(fs::path(entry_path) / "status", status, nullptr, status_watch_callback)) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: missing status", key); + continue; + } + + bool is_pending = status == "PENDING"; + + /// if we have the cleanup lock, try to cleanup + /// if we successfully cleaned it up, early exit + if (cleanup_lock) + { + bool cleanup_successful = tryCleanup( + zk, + entry_path, + storage.log.load(), + storage.getContext(), + key, + metadata, + now, + is_pending, entries_by_key); + + if (cleanup_successful) + continue; + } + + if (!is_pending) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: status is not PENDING", key); + continue; + } + + if (has_local_entry_and_is_up_to_date) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: already exists", key); + continue; + } + + addTask(metadata, key, entries_by_key); + } + + /// Remove entries that were deleted by someone else + removeStaleEntries(zk_children, entries_by_key); + + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: finished polling for new entries. Number of entries: {}", entries_by_key.size()); + + storage.export_merge_tree_partition_select_task->schedule(); +} + +void ExportPartitionManifestUpdatingTask::addTask( + const ExportReplicatedMergeTreePartitionManifest & metadata, + const std::string & key, + auto & entries_by_key +) +{ + std::vector part_references; + + for (const auto & part_name : metadata.parts) + { + if (const auto part = storage.getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated})) + { + part_references.push_back(part); + } + } + + /// Insert or update entry. The multi_index container automatically maintains both indexes. + auto entry = ExportReplicatedMergeTreePartitionTaskEntry {metadata, ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING, std::move(part_references)}; + auto it = entries_by_key.find(key); + if (it != entries_by_key.end()) + entries_by_key.replace(it, entry); + else + entries_by_key.insert(entry); +} + +void ExportPartitionManifestUpdatingTask::removeStaleEntries( + const std::unordered_set & zk_children, + auto & entries_by_key +) +{ + for (auto it = entries_by_key.begin(); it != entries_by_key.end();) + { + const auto & key = it->getCompositeKey(); + if (zk_children.contains(key)) + { + ++it; + continue; + } + + const auto & transaction_id = it->manifest.transaction_id; + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Export task {} was deleted, calling killExportPartition for transaction {}", key, transaction_id); + + try + { + storage.killExportPart(transaction_id); + } + catch (...) + { + tryLogCurrentException(storage.log, __PRETTY_FUNCTION__); + } + + it = entries_by_key.erase(it); + } +} + +void ExportPartitionManifestUpdatingTask::addStatusChange(const std::string & key) +{ + std::lock_guard lock(status_changes_mutex); + status_changes.emplace(key); +} + +void ExportPartitionManifestUpdatingTask::handleStatusChanges() +{ + std::lock_guard lock(status_changes_mutex); + std::lock_guard task_entries_lock(storage.export_merge_tree_partition_mutex); + auto zk = storage.getZooKeeper(); + + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: handling status changes. Number of status changes: {}", status_changes.size()); + + while (!status_changes.empty()) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: handling status change for task {}", status_changes.front()); + const auto key = status_changes.front(); + status_changes.pop(); + + auto it = storage.export_merge_tree_partition_task_entries_by_key.find(key); + if (it == storage.export_merge_tree_partition_task_entries_by_key.end()) + continue; + + /// get new status from zk + std::string new_status_string; + if (!zk->tryGet(fs::path(storage.zookeeper_path) / "exports" / key / "status", new_status_string)) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Failed to get new status for task {}, skipping", key); + continue; + } + + const auto new_status = magic_enum::enum_cast(new_status_string); + if (!new_status) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Invalid status {} for task {}, skipping", new_status_string, key); + continue; + } + + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: status changed for task {}. New status: {}", key, magic_enum::enum_name(*new_status).data()); + + /// If status changed to KILLED, cancel local export operations + if (*new_status == ExportReplicatedMergeTreePartitionTaskEntry::Status::KILLED) + { + try + { + storage.killExportPart(it->manifest.transaction_id); + } + catch (...) + { + tryLogCurrentException(storage.log, __PRETTY_FUNCTION__); + } + } + + it->status = *new_status; + } +} + +} diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h new file mode 100644 index 000000000000..ea52f679d654 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h @@ -0,0 +1,42 @@ +#pragma once + +#include +#include +#include +#include +namespace DB +{ + +class StorageReplicatedMergeTree; +struct ExportReplicatedMergeTreePartitionManifest; + +class ExportPartitionManifestUpdatingTask +{ +public: + ExportPartitionManifestUpdatingTask(StorageReplicatedMergeTree & storage); + + void poll(); + + void handleStatusChanges(); + + void addStatusChange(const std::string & key); + +private: + StorageReplicatedMergeTree & storage; + + void addTask( + const ExportReplicatedMergeTreePartitionManifest & metadata, + const std::string & key, + auto & entries_by_key + ); + + void removeStaleEntries( + const std::unordered_set & zk_children, + auto & entries_by_key + ); + + std::mutex status_changes_mutex; + std::queue status_changes; +}; + +} diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp new file mode 100644 index 000000000000..528841e21188 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -0,0 +1,387 @@ +#include +#include +#include +#include +#include +#include +#include "Storages/MergeTree/ExportPartitionUtils.h" +#include "Storages/MergeTree/MergeTreePartExportManifest.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int QUERY_WAS_CANCELLED; + extern const int LOGICAL_ERROR; +} + +namespace +{ + ContextPtr getContextCopyWithTaskSettings(const ContextPtr & context, const ExportReplicatedMergeTreePartitionManifest & manifest) + { + auto context_copy = Context::createCopy(context); + context_copy->setSetting("output_format_parallel_formatting", manifest.parallel_formatting); + context_copy->setSetting("output_format_parquet_parallel_encoding", manifest.parquet_parallel_encoding); + context_copy->setSetting("max_threads", manifest.max_threads); + context_copy->setSetting("export_merge_tree_part_file_already_exists_policy", String(magic_enum::enum_name(manifest.file_already_exists_policy))); + return context_copy; + } +} + +ExportPartitionTaskScheduler::ExportPartitionTaskScheduler(StorageReplicatedMergeTree & storage_) + : storage(storage_) +{ +} + +void ExportPartitionTaskScheduler::run() +{ + std::lock_guard lock(storage.export_merge_tree_partition_mutex); + + auto zk = storage.getZooKeeper(); + + // Iterate sorted by create_time + for (auto & entry : storage.export_merge_tree_partition_task_entries_by_create_time) + { + const auto & manifest = entry.manifest; + const auto key = entry.getCompositeKey(); + const auto database = storage.getContext()->resolveDatabase(manifest.destination_database); + const auto & table = manifest.destination_table; + + /// No need to query zk for status if the local one is not PENDING + if (entry.status != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Skipping... Local status is {}", magic_enum::enum_name(entry.status).data()); + continue; + } + + const auto destination_storage_id = StorageID(QualifiedTableName {database, table}); + + const auto destination_storage = DatabaseCatalog::instance().tryGetTable(destination_storage_id, storage.getContext()); + + if (!destination_storage) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); + continue; + } + + std::string status_in_zk_string; + if (!zk->tryGet(fs::path(storage.zookeeper_path) / "exports" / key / "status", status_in_zk_string)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get status, skipping"); + continue; + } + + const auto status_in_zk = magic_enum::enum_cast(status_in_zk_string); + + if (!status_in_zk) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get status from zk, skipping"); + continue; + } + + if (status_in_zk.value() != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + entry.status = status_in_zk.value(); + LOG_INFO(storage.log, "ExportPartition scheduler task: Skipping... Status from zk is {}", entry.status); + continue; + } + + std::vector parts_in_processing_or_pending; + + if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(storage.zookeeper_path) / "exports" / key / "processing", parts_in_processing_or_pending)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get parts in processing or pending, skipping"); + continue; + } + + if (parts_in_processing_or_pending.empty()) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: No parts in processing or pending, skipping"); + continue; + } + + std::vector locked_parts; + + if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(storage.zookeeper_path) / "exports" / key / "locks", locked_parts)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get locked parts, skipping"); + continue; + } + + std::unordered_set locked_parts_set(locked_parts.begin(), locked_parts.end()); + + for (const auto & zk_part_name : parts_in_processing_or_pending) + { + if (locked_parts_set.contains(zk_part_name)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is locked, skipping", zk_part_name); + continue; + } + + const auto part = storage.getPartIfExists(zk_part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + if (!part) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} not found locally, skipping", zk_part_name); + continue; + } + + if (Coordination::Error::ZOK != zk->tryCreate(fs::path(storage.zookeeper_path) / "exports" / key / "locks" / zk_part_name, storage.replica_name, zkutil::CreateMode::Ephemeral)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to lock part {}, skipping", zk_part_name); + continue; + } + + try + { + storage.exportPartToTable( + part->name, + destination_storage_id, + manifest.transaction_id, + getContextCopyWithTaskSettings(storage.getContext(), manifest), + [this, key, zk_part_name, manifest, destination_storage] + (MergeTreePartExportManifest::CompletionCallbackResult result) + { + handlePartExportCompletion(key, zk_part_name, manifest, destination_storage, result); + }); + } + catch (const Exception &) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + zk->tryRemove(fs::path(storage.zookeeper_path) / "exports" / key / "locks" / zk_part_name); + /// we should not increment retry_count because the node might just be full + } + } + } + + /// maybe we failed to schedule or failed to export, need to retry eventually + storage.export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); +} + +void ExportPartitionTaskScheduler::handlePartExportCompletion( + const std::string & export_key, + const std::string & part_name, + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const MergeTreePartExportManifest::CompletionCallbackResult & result) +{ + const auto export_path = fs::path(storage.zookeeper_path) / "exports" / export_key; + const auto processing_parts_path = export_path / "processing"; + const auto processed_part_path = export_path / "processed" / part_name; + const auto zk = storage.getZooKeeper(); + + if (result.success) + { + handlePartExportSuccess(manifest, destination_storage, processing_parts_path, processed_part_path, part_name, export_path, zk, result.relative_path_in_destination_storage); + } + else + { + handlePartExportFailure(processing_parts_path, part_name, export_path, zk, result.exception, manifest.max_retries); + } +} + +void ExportPartitionTaskScheduler::handlePartExportSuccess( + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const std::filesystem::path & processing_parts_path, + const std::filesystem::path & processed_part_path, + const std::string & part_name, + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk, + const String & relative_path_in_destination_storage +) +{ + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} exported successfully", relative_path_in_destination_storage); + + if (!tryToMovePartToProcessed(export_path, processing_parts_path, processed_part_path, part_name, relative_path_in_destination_storage, zk)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to move part to processed, will not commit export partition"); + return; + } + + LOG_INFO(storage.log, "ExportPartition scheduler task: Marked part export {} as completed", part_name); + + if (!areAllPartsProcessed(export_path, zk)) + { + return; + } + + LOG_INFO(storage.log, "ExportPartition scheduler task: All parts are processed, will try to commit export partition"); + + ExportPartitionUtils::commit(manifest, destination_storage, zk, storage.log.load(), export_path, storage.getContext()); +} + +void ExportPartitionTaskScheduler::handlePartExportFailure( + const std::filesystem::path & processing_parts_path, + const std::string & part_name, + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk, + const std::optional & exception, + size_t max_retries +) +{ + if (!exception) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "ExportPartition scheduler task: No exception provided for error handling. Sounds like a bug"); + } + + /// Early exit if the query was cancelled - no need to increment error counts + if (exception->code() == ErrorCodes::QUERY_WAS_CANCELLED) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} export was cancelled, skipping error handling", part_name); + return; + } + + Coordination::Stat locked_by_stat; + std::string locked_by; + + if (!zk->tryGet(export_path / "locks" / part_name, locked_by, &locked_by_stat)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is not locked by any replica, will not increment error counts", part_name); + return; + } + + if (locked_by != storage.replica_name) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is locked by another replica, will not increment error counts", part_name); + return; + } + + Coordination::Requests ops; + + const auto processing_part_path = processing_parts_path / part_name; + + std::string processing_part_string; + + if (!zk->tryGet(processing_part_path, processing_part_string)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get processing part, will not increment error counts"); + return; + } + + /// todo arthur could this have been cached? + auto processing_part_entry = ExportReplicatedMergeTreePartitionProcessingPartEntry::fromJsonString(processing_part_string); + + processing_part_entry.retry_count++; + + if (processing_part_entry.retry_count) + { + ops.emplace_back(zkutil::makeRemoveRequest(export_path / "locks" / part_name, locked_by_stat.version)); + ops.emplace_back(zkutil::makeSetRequest(processing_part_path, processing_part_entry.toJsonString(), -1)); + + if (processing_part_entry.retry_count >= max_retries) + { + /// just set status in processing_part_path and finished_by + processing_part_entry.status = ExportReplicatedMergeTreePartitionProcessingPartEntry::Status::FAILED; + processing_part_entry.finished_by = storage.replica_name; + + ops.emplace_back(zkutil::makeSetRequest(export_path / "status", String(magic_enum::enum_name(ExportReplicatedMergeTreePartitionTaskEntry::Status::FAILED)).data(), -1)); + LOG_INFO(storage.log, "ExportPartition scheduler task: Retry count limit exceeded for part {}, will try to fail the entire task", part_name); + } + + std::size_t num_exceptions = 0; + + const auto exceptions_per_replica_path = export_path / "exceptions_per_replica" / storage.replica_name; + const auto count_path = exceptions_per_replica_path / "count"; + const auto last_exception_path = exceptions_per_replica_path / "last_exception"; + + if (zk->exists(exceptions_per_replica_path)) + { + std::string num_exceptions_string; + zk->tryGet(count_path, num_exceptions_string); + num_exceptions = std::stoull(num_exceptions_string.c_str()); + + ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "part", part_name, -1)); + ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "exception", exception->message(), -1)); + } + else + { + ops.emplace_back(zkutil::makeCreateRequest(exceptions_per_replica_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(count_path, "0", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "part", part_name, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "exception", exception->message(), zkutil::CreateMode::Persistent)); + } + + num_exceptions++; + ops.emplace_back(zkutil::makeSetRequest(count_path, std::to_string(num_exceptions), -1)); + + Coordination::Responses responses; + if (Coordination::Error::ZOK != zk->tryMulti(ops, responses)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: All failure mechanism failed, will not try to update it"); + return; + } + } +} + +bool ExportPartitionTaskScheduler::tryToMovePartToProcessed( + const std::filesystem::path & export_path, + const std::filesystem::path & processing_parts_path, + const std::filesystem::path & processed_part_path, + const std::string & part_name, + const String & relative_path_in_destination_storage, + const zkutil::ZooKeeperPtr & zk +) +{ + Coordination::Stat locked_by_stat; + std::string locked_by; + + if (!zk->tryGet(export_path / "locks" / part_name, locked_by, &locked_by_stat)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is not locked by any replica, will not commit or set it as completed", part_name); + return false; + } + + /// Is this a good idea? what if the file we just pushed to s3 ends up triggering an exception in the replica that actually locks the part and it does not commit? + /// I guess we should not throw if file already exists for export partition, hard coded. + if (locked_by != storage.replica_name) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is locked by another replica, will not commit or set it as completed", part_name); + return false; + } + + Coordination::Requests requests; + + ExportReplicatedMergeTreePartitionProcessedPartEntry processed_part_entry; + processed_part_entry.part_name = part_name; + processed_part_entry.path_in_destination = relative_path_in_destination_storage; + processed_part_entry.finished_by = storage.replica_name; + + requests.emplace_back(zkutil::makeRemoveRequest(processing_parts_path / part_name, -1)); + requests.emplace_back(zkutil::makeCreateRequest(processed_part_path, processed_part_entry.toJsonString(), zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeRemoveRequest(export_path / "locks" / part_name, locked_by_stat.version)); + + Coordination::Responses responses; + if (Coordination::Error::ZOK != zk->tryMulti(requests, responses)) + { + /// todo arthur remember what to do here + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to update export path, skipping"); + return false; + } + + return true; +} + +bool ExportPartitionTaskScheduler::areAllPartsProcessed( + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk) +{ + Strings parts_in_processing_or_pending; + if (Coordination::Error::ZOK != zk->tryGetChildren(export_path / "processing", parts_in_processing_or_pending)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get parts in processing or pending, will not try to commit export partition"); + return false; + } + + if (!parts_in_processing_or_pending.empty()) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: There are still parts in processing or pending, will not try to commit export partition"); + return false; + } + + return true; +} + +} diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.h b/src/Storages/MergeTree/ExportPartitionTaskScheduler.h new file mode 100644 index 000000000000..0045019a4ec7 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.h @@ -0,0 +1,66 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class Exception; +class StorageReplicatedMergeTree; + +struct ExportReplicatedMergeTreePartitionManifest; + +/// todo arthur remember to add check(lock, version) when updating stuff because maybe if we believe we have the lock, we might not actually have it +class ExportPartitionTaskScheduler +{ +public: + ExportPartitionTaskScheduler(StorageReplicatedMergeTree & storage); + + void run(); +private: + StorageReplicatedMergeTree & storage; + + /// todo arthur maybe it is invalid to grab the manifst here + void handlePartExportCompletion( + const std::string & export_key, + const std::string & part_name, + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const MergeTreePartExportManifest::CompletionCallbackResult & result); + + void handlePartExportSuccess( + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const std::filesystem::path & processing_parts_path, + const std::filesystem::path & processed_part_path, + const std::string & part_name, + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk, + const String & relative_path_in_destination_storage + ); + + void handlePartExportFailure( + const std::filesystem::path & processing_parts_path, + const std::string & part_name, + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk, + const std::optional & exception, + size_t max_retries); + + bool tryToMovePartToProcessed( + const std::filesystem::path & export_path, + const std::filesystem::path & processing_parts_path, + const std::filesystem::path & processed_part_path, + const std::string & part_name, + const String & relative_path_in_destination_storage, + const zkutil::ZooKeeperPtr & zk + ); + + bool areAllPartsProcessed( + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk + ); +}; + +} diff --git a/src/Storages/MergeTree/ExportPartitionUtils.cpp b/src/Storages/MergeTree/ExportPartitionUtils.cpp new file mode 100644 index 000000000000..466eb79e8367 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionUtils.cpp @@ -0,0 +1,95 @@ +#include +#include +#include +#include "Storages/ExportReplicatedMergeTreePartitionManifest.h" +#include "Storages/ExportReplicatedMergeTreePartitionTaskEntry.h" +#include + +namespace DB +{ + +namespace fs = std::filesystem; + +namespace ExportPartitionUtils +{ + /// Collect all the exported paths from the processed parts + /// If multiRead is supported by the keeper implementation, it is done in a single request + /// Otherwise, multiple async requests are sent + std::vector getExportedPaths(const LoggerPtr & log, const zkutil::ZooKeeperPtr & zk, const std::string & export_path) + { + std::vector exported_paths; + + LOG_INFO(log, "ExportPartition: Getting exported paths for {}", export_path); + + const auto processed_parts_path = fs::path(export_path) / "processed"; + + std::vector processed_parts; + if (Coordination::Error::ZOK != zk->tryGetChildren(processed_parts_path, processed_parts)) + { + /// todo arthur do something here + LOG_INFO(log, "ExportPartition: Failed to get parts children, exiting"); + return {}; + } + + std::vector get_paths; + + for (const auto & processed_part : processed_parts) + { + get_paths.emplace_back(processed_parts_path / processed_part); + } + + auto responses = zk->tryGet(get_paths); + + responses.waitForResponses(); + + for (size_t i = 0; i < responses.size(); ++i) + { + if (responses[i].error != Coordination::Error::ZOK) + { + /// todo arthur what to do in this case? + /// It could be that zk is corrupt, in that case we should fail the task + /// but it can also be some temporary network issue? not sure + LOG_INFO(log, "ExportPartition: Failed to get exported path, exiting"); + return {}; + } + + const auto processed_part_entry = ExportReplicatedMergeTreePartitionProcessedPartEntry::fromJsonString(responses[i].data); + + exported_paths.emplace_back(processed_part_entry.path_in_destination); + } + + return exported_paths; + } + + void commit( + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const zkutil::ZooKeeperPtr & zk, + const LoggerPtr & log, + const std::string & entry_path, + const ContextPtr & context) + { + const auto exported_paths = ExportPartitionUtils::getExportedPaths(log, zk, entry_path); + + if (exported_paths.size() != manifest.parts.size()) + { + LOG_INFO(log, "ExportPartition: Skipping {}: exported paths size does not match parts size, this is a BUG", entry_path); + return; + } + + LOG_INFO(log, "ExportPartition: Exported paths size matches parts size, commit the export"); + destination_storage->commitExportPartitionTransaction(manifest.transaction_id, manifest.partition_id, exported_paths, context); + + LOG_INFO(log, "ExportPartition: Committed export, mark as completed"); + if (Coordination::Error::ZOK == zk->trySet(fs::path(entry_path) / "status", String(magic_enum::enum_name(ExportReplicatedMergeTreePartitionTaskEntry::Status::COMPLETED)).data(), -1)) + { + LOG_INFO(log, "ExportPartition: Marked export as completed"); + } + else + { + LOG_INFO(log, "ExportPartition: Failed to mark export as completed, will not try to fix it"); + } + } +} + +} diff --git a/src/Storages/MergeTree/ExportPartitionUtils.h b/src/Storages/MergeTree/ExportPartitionUtils.h new file mode 100644 index 000000000000..40fe04a5bfd3 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionUtils.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include +#include +#include +#include "Storages/IStorage.h" + +namespace DB +{ + +struct ExportReplicatedMergeTreePartitionManifest; + +namespace ExportPartitionUtils +{ + std::vector getExportedPaths(const LoggerPtr & log, const zkutil::ZooKeeperPtr & zk, const std::string & export_path); + + void commit( + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const zkutil::ZooKeeperPtr & zk, + const LoggerPtr & log, + const std::string & entry_path, + const ContextPtr & context + ); +} + +} diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9427ff9334d8..f889a4cc50d1 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -24,6 +24,7 @@ #include #include #include +#include "Storages/MergeTree/ExportPartTask.h" #include #include #include @@ -102,6 +103,7 @@ #include #include #include +#include #include @@ -212,7 +214,7 @@ namespace Setting extern const SettingsBool apply_patch_parts; extern const SettingsBool allow_experimental_export_merge_tree_part; extern const SettingsUInt64 min_bytes_to_use_direct_io; - extern const SettingsBool export_merge_tree_part_overwrite_file_if_exists; + extern const SettingsMergeTreePartExportFileAlreadyExistsPolicy export_merge_tree_part_file_already_exists_policy; extern const SettingsBool output_format_parallel_formatting; extern const SettingsBool output_format_parquet_parallel_encoding; } @@ -6208,10 +6210,23 @@ void MergeTreeData::exportPartToTable(const PartitionCommand & command, ContextP "Exporting merge tree part is experimental. Set `allow_experimental_export_merge_tree_part` to enable it"); } - String dest_database = query_context->resolveDatabase(command.to_database); - auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, command.to_table}, query_context); + const auto part_name = command.partition->as().value.safeGet(); - if (dest_storage->getStorageID() == this->getStorageID()) + const auto database_name = query_context->resolveDatabase(command.to_database); + + exportPartToTable(part_name, StorageID{database_name, command.to_table}, generateSnowflakeIDString(), query_context); +} + +void MergeTreeData::exportPartToTable( + const std::string & part_name, + const StorageID & destination_storage_id, + const String & transaction_id, + ContextPtr query_context, + std::function completion_callback) +{ + auto dest_storage = DatabaseCatalog::instance().getTable(destination_storage_id, query_context); + + if (destination_storage_id == this->getStorageID()) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Exporting to the same table is not allowed"); } @@ -6233,8 +6248,6 @@ void MergeTreeData::exportPartToTable(const PartitionCommand & command, ContextP if (query_to_string(src_snapshot->getPartitionKeyAST()) != query_to_string(destination_snapshot->getPartitionKeyAST())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key"); - auto part_name = command.partition->as().value.safeGet(); - auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); if (!part) @@ -6243,11 +6256,13 @@ void MergeTreeData::exportPartToTable(const PartitionCommand & command, ContextP { const auto format_settings = getFormatSettings(query_context); - MergeTreeExportManifest manifest( + MergeTreePartExportManifest manifest( dest_storage->getStorageID(), part, - query_context->getSettingsRef()[Setting::export_merge_tree_part_overwrite_file_if_exists], - format_settings); + transaction_id, + query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value, + format_settings, + completion_callback); std::lock_guard lock(export_manifests_mutex); @@ -6261,169 +6276,21 @@ void MergeTreeData::exportPartToTable(const PartitionCommand & command, ContextP background_moves_assignee.trigger(); } -void MergeTreeData::exportPartToTableImpl( - const MergeTreeExportManifest & manifest, - ContextPtr local_context) +void MergeTreeData::killExportPart(const String & transaction_id) { - auto metadata_snapshot = getInMemoryMetadataPtr(); - Names columns_to_read = metadata_snapshot->getColumns().getNamesOfPhysical(); - StorageSnapshotPtr storage_snapshot = getStorageSnapshot(metadata_snapshot, local_context); - - MergeTreeSequentialSourceType read_type = MergeTreeSequentialSourceType::Export; - - Block block_with_partition_values; - if (metadata_snapshot->hasPartitionKey()) - { - /// todo arthur do I need to init minmax_idx? - block_with_partition_values = manifest.data_part->minmax_idx->getBlock(*this); - } - - auto destination_storage = DatabaseCatalog::instance().tryGetTable(manifest.destination_storage_id, getContext()); - if (!destination_storage) - { - std::lock_guard inner_lock(export_manifests_mutex); - - const auto destination_storage_id_name = manifest.destination_storage_id.getNameForLogs(); - export_manifests.erase(manifest); - throw Exception(ErrorCodes::UNKNOWN_TABLE, "Failed to reconstruct destination storage: {}", destination_storage_id_name); - } - - SinkToStoragePtr sink; - std::string destination_file_path; + std::lock_guard lock(export_manifests_mutex); - try - { - sink = destination_storage->import( - manifest.data_part->name + "_" + manifest.data_part->checksums.getTotalChecksumHex(), - block_with_partition_values, - destination_file_path, - manifest.overwrite_file_if_exists, - manifest.format_settings, - local_context); - } - catch (const Exception & e) + std::erase_if(export_manifests, [&](const auto & manifest) { - if (e.code() == ErrorCodes::FILE_ALREADY_EXISTS) + if (manifest.transaction_id == transaction_id) { - ProfileEvents::increment(ProfileEvents::PartsExportDuplicated); - } - - ProfileEvents::increment(ProfileEvents::PartsExportFailures); - - std::lock_guard inner_lock(export_manifests_mutex); - export_manifests.erase(manifest); - return; - } - - bool apply_deleted_mask = true; - bool read_with_direct_io = local_context->getSettingsRef()[Setting::min_bytes_to_use_direct_io] > manifest.data_part->getBytesOnDisk(); - bool prefetch = false; - - MergeTreeData::IMutationsSnapshot::Params params - { - .metadata_version = metadata_snapshot->getMetadataVersion(), - .min_part_metadata_version = manifest.data_part->getMetadataVersion(), - }; - - auto mutations_snapshot = getMutationsSnapshot(params); - - auto alter_conversions = MergeTreeData::getAlterConversionsForPart( - manifest.data_part, - mutations_snapshot, - local_context); + if (manifest.task) + manifest.task->cancel(); - QueryPlan plan_for_part; - - createReadFromPartStep( - read_type, - plan_for_part, - *this, - storage_snapshot, - RangesInDataPart(manifest.data_part), - alter_conversions, - nullptr, - columns_to_read, - nullptr, - apply_deleted_mask, - std::nullopt, - read_with_direct_io, - prefetch, - local_context, - getLogger("ExportPartition")); - - auto exports_list_entry = getContext()->getExportsList().insert( - getStorageID(), - manifest.destination_storage_id, - manifest.data_part->getBytesOnDisk(), - manifest.data_part->name, - destination_file_path, - manifest.data_part->rows_count, - manifest.data_part->getBytesOnDisk(), - manifest.data_part->getBytesUncompressedOnDisk(), - manifest.create_time, - local_context); - - ThreadGroupSwitcher switcher((*exports_list_entry)->thread_group, ""); - - QueryPlanOptimizationSettings optimization_settings(local_context); - auto pipeline_settings = BuildQueryPipelineSettings(local_context); - auto builder = plan_for_part.buildQueryPipeline(optimization_settings, pipeline_settings); - - builder->setProgressCallback([&exports_list_entry](const Progress & progress) - { - (*exports_list_entry)->bytes_read_uncompressed += progress.read_bytes; - (*exports_list_entry)->rows_read += progress.read_rows; - (*exports_list_entry)->elapsed = (*exports_list_entry)->watch.elapsedSeconds(); + return true; + } + return false; }); - - auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); - - pipeline.complete(sink); - - try - { - CompletedPipelineExecutor exec(pipeline); - exec.execute(); - - std::lock_guard inner_lock(export_manifests_mutex); - writePartLog( - PartLogElement::Type::EXPORT_PART, - {}, - static_cast((*exports_list_entry)->elapsed * 1000000000), - manifest.data_part->name, - manifest.data_part, - {manifest.data_part}, - nullptr, - nullptr, - exports_list_entry.get()); - - export_manifests.erase(manifest); - - ProfileEvents::increment(ProfileEvents::PartsExports); - ProfileEvents::increment(ProfileEvents::PartsExportTotalMilliseconds, static_cast((*exports_list_entry)->elapsed * 1000)); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("while exporting the part {}. User should retry.", manifest.data_part->name)); - - ProfileEvents::increment(ProfileEvents::PartsExportFailures); - - std::lock_guard inner_lock(export_manifests_mutex); - writePartLog( - PartLogElement::Type::EXPORT_PART, - ExecutionStatus::fromCurrentException("", true), - static_cast((*exports_list_entry)->elapsed * 1000000000), - manifest.data_part->name, - manifest.data_part, - {manifest.data_part}, - nullptr, - nullptr, - exports_list_entry.get()); - - export_manifests.erase(manifest); - - throw; - } } void MergeTreeData::movePartitionToShard(const ASTPtr & /*partition*/, bool /*move_part*/, const String & /*to*/, ContextPtr /*query_context*/) @@ -6483,6 +6350,12 @@ Pipe MergeTreeData::alterPartition( break; } + case PartitionCommand::EXPORT_PARTITION: + { + exportPartitionToTable(command, query_context); + break; + } + case PartitionCommand::DROP_DETACHED_PARTITION: dropDetached(command.partition, command.part, query_context); break; @@ -9258,15 +9131,18 @@ bool MergeTreeData::scheduleDataMovingJob(BackgroundJobsAssignee & assignee) continue; } - manifest.in_progress = assignee.scheduleMoveTask(std::make_shared( - [this, manifest] () mutable { - exportPartToTableImpl(manifest, getContext()); - return true; - }, - moves_assignee_trigger, - getStorageID())); + auto task = std::make_shared(*this, manifest, getContext()); + + manifest.in_progress = assignee.scheduleMoveTask(task); - return manifest.in_progress; + if (!manifest.in_progress) + { + continue; + } + + manifest.task = task; + + return true; } return false; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 7072b8d52e82..521bc7e50279 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -38,8 +38,8 @@ #include #include #include -#include -#include +#include +#include #include #include @@ -984,9 +984,19 @@ class MergeTreeData : public IStorage, public WithMutableContext void exportPartToTable(const PartitionCommand & command, ContextPtr query_context); - void exportPartToTableImpl( - const MergeTreeExportManifest & manifest, - ContextPtr local_context); + void exportPartToTable( + const std::string & part_name, + const StorageID & destination_storage_id, + const String & transaction_id, + ContextPtr query_context, + std::function completion_callback = {}); + + void killExportPart(const String & transaction_id); + + virtual void exportPartitionToTable(const PartitionCommand &, ContextPtr) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "EXPORT PARTITION is not implemented for engine {}", getName()); + } /// Checks that Partition could be dropped right now /// Otherwise - throws an exception with detailed information. @@ -1253,7 +1263,7 @@ class MergeTreeData : public IStorage, public WithMutableContext mutable std::mutex export_manifests_mutex; - std::set export_manifests; + std::set export_manifests; PinnedPartUUIDsPtr getPinnedPartUUIDs() const; @@ -1348,6 +1358,7 @@ class MergeTreeData : public IStorage, public WithMutableContext friend class MergeTask; friend class IPartMetadataManager; friend class IMergedBlockOutputStream; // for access to log + friend class ExportPartTask; bool require_part_metadata; diff --git a/src/Storages/MergeTree/MergeTreeExportManifest.h b/src/Storages/MergeTree/MergeTreeExportManifest.h deleted file mode 100644 index 05506ecb004a..000000000000 --- a/src/Storages/MergeTree/MergeTreeExportManifest.h +++ /dev/null @@ -1,50 +0,0 @@ -#include -#include - -namespace DB -{ - -struct MergeTreeExportManifest -{ - using DataPartPtr = std::shared_ptr; - - - MergeTreeExportManifest( - const StorageID & destination_storage_id_, - const DataPartPtr & data_part_, - bool overwrite_file_if_exists_, - const FormatSettings & format_settings_) - : destination_storage_id(destination_storage_id_), - data_part(data_part_), - overwrite_file_if_exists(overwrite_file_if_exists_), - format_settings(format_settings_), - create_time(time(nullptr)) {} - - StorageID destination_storage_id; - DataPartPtr data_part; - bool overwrite_file_if_exists; - FormatSettings format_settings; - - time_t create_time; - mutable bool in_progress = false; - - bool operator<(const MergeTreeExportManifest & rhs) const - { - // Lexicographic comparison: first compare destination storage, then part name - auto lhs_storage = destination_storage_id.getQualifiedName(); - auto rhs_storage = rhs.destination_storage_id.getQualifiedName(); - - if (lhs_storage != rhs_storage) - return lhs_storage < rhs_storage; - - return data_part->name < rhs.data_part->name; - } - - bool operator==(const MergeTreeExportManifest & rhs) const - { - return destination_storage_id.getQualifiedName() == rhs.destination_storage_id.getQualifiedName() - && data_part->name == rhs.data_part->name; - } -}; - -} diff --git a/src/Storages/MergeTree/MergeTreePartExportManifest.h b/src/Storages/MergeTree/MergeTreePartExportManifest.h new file mode 100644 index 000000000000..c89ddc0daf68 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreePartExportManifest.h @@ -0,0 +1,90 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class Exception; + +class ExportPartTask; + +struct MergeTreePartExportManifest +{ + using FileAlreadyExistsPolicy = MergeTreePartExportFileAlreadyExistsPolicy; + + using DataPartPtr = std::shared_ptr; + + struct CompletionCallbackResult + { + private: + CompletionCallbackResult(bool success_, const String & relative_path_in_destination_storage_, std::optional exception_) + : success(success_), relative_path_in_destination_storage(relative_path_in_destination_storage_), exception(std::move(exception_)) {} + public: + + static CompletionCallbackResult createSuccess(const String & relative_path_in_destination_storage_) + { + return CompletionCallbackResult(true, relative_path_in_destination_storage_, std::nullopt); + } + + static CompletionCallbackResult createFailure(Exception exception_) + { + return CompletionCallbackResult(false, "", std::move(exception_)); + } + + bool success = false; + String relative_path_in_destination_storage; + std::optional exception; + }; + + MergeTreePartExportManifest( + const StorageID & destination_storage_id_, + const DataPartPtr & data_part_, + const String & transaction_id_, + FileAlreadyExistsPolicy file_already_exists_policy_, + const FormatSettings & format_settings_, + std::function completion_callback_ = {}) + : destination_storage_id(destination_storage_id_), + data_part(data_part_), + transaction_id(transaction_id_), + file_already_exists_policy(file_already_exists_policy_), + format_settings(format_settings_), + completion_callback(completion_callback_), + create_time(time(nullptr)) {} + + StorageID destination_storage_id; + DataPartPtr data_part; + /// Used for killing the export. + String transaction_id; + FileAlreadyExistsPolicy file_already_exists_policy; + FormatSettings format_settings; + + std::function completion_callback; + + time_t create_time; + mutable bool in_progress = false; + mutable std::shared_ptr task = nullptr; + + bool operator<(const MergeTreePartExportManifest & rhs) const + { + // Lexicographic comparison: first compare destination storage, then part name + auto lhs_storage = destination_storage_id.getQualifiedName(); + auto rhs_storage = rhs.destination_storage_id.getQualifiedName(); + + if (lhs_storage != rhs_storage) + return lhs_storage < rhs_storage; + + return data_part->name < rhs.data_part->name; + } + + bool operator==(const MergeTreePartExportManifest & rhs) const + { + return destination_storage_id.getQualifiedName() == rhs.destination_storage_id.getQualifiedName() + && data_part->name == rhs.data_part->name; + } +}; + +} diff --git a/src/Storages/MergeTree/MergeTreeExportStatus.h b/src/Storages/MergeTree/MergeTreePartExportStatus.h similarity index 100% rename from src/Storages/MergeTree/MergeTreeExportStatus.h rename to src/Storages/MergeTree/MergeTreePartExportStatus.h diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 8420cd5738c2..aca997b1e443 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -11,6 +11,7 @@ #include #include #include +#include namespace CurrentMetrics @@ -27,6 +28,11 @@ namespace MergeTreeSetting extern const MergeTreeSettingsSeconds zookeeper_session_expiration_check_period; } +namespace ServerSetting +{ + extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; +} + namespace ErrorCodes { extern const int REPLICA_IS_ALREADY_ACTIVE; @@ -171,10 +177,19 @@ bool ReplicatedMergeTreeRestartingThread::runImpl() storage.mutations_updating_task->activateAndSchedule(); storage.mutations_finalizing_task->activateAndSchedule(); storage.merge_selecting_task->activateAndSchedule(); + + if (storage.getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + storage.export_merge_tree_partition_updating_task->activateAndSchedule(); + storage.export_merge_tree_partition_select_task->activateAndSchedule(); + storage.export_merge_tree_partition_status_handling_task->activateAndSchedule(); + } + storage.cleanup_thread.start(); storage.async_block_ids_cache.start(); storage.part_check_thread.start(); + LOG_DEBUG(log, "Table started successfully"); return true; } diff --git a/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp b/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp new file mode 100644 index 000000000000..a3b74a2f5e6e --- /dev/null +++ b/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp @@ -0,0 +1,72 @@ +#include +#include + +namespace DB +{ + +class ExportPartitionOrderingTest : public ::testing::Test +{ +protected: + ExportPartitionTaskEntriesContainer container; + ExportPartitionTaskEntriesContainer::index::type & by_key; + ExportPartitionTaskEntriesContainer::index::type & by_create_time; + + ExportPartitionOrderingTest() + : by_key(container.get()) + , by_create_time(container.get()) + { + } +}; + +TEST_F(ExportPartitionOrderingTest, IterationOrderMatchesCreateTime) +{ + time_t base_time = 1000; + + ExportReplicatedMergeTreePartitionManifest manifest1; + manifest1.partition_id = "2020"; + manifest1.destination_database = "db1"; + manifest1.destination_table = "table1"; + manifest1.create_time = base_time + 300; // Latest + + ExportReplicatedMergeTreePartitionManifest manifest2; + manifest2.partition_id = "2021"; + manifest2.destination_database = "db1"; + manifest2.destination_table = "table1"; + manifest2.create_time = base_time + 100; // Middle + + ExportReplicatedMergeTreePartitionManifest manifest3; + manifest3.partition_id = "2022"; + manifest3.destination_database = "db1"; + manifest3.destination_table = "table1"; + manifest3.create_time = base_time; // Oldest + + ExportReplicatedMergeTreePartitionTaskEntry entry1{manifest1, ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING, {}}; + ExportReplicatedMergeTreePartitionTaskEntry entry2{manifest2, ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING, {}}; + ExportReplicatedMergeTreePartitionTaskEntry entry3{manifest3, ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING, {}}; + + // Insert in reverse order + by_key.insert(entry1); + by_key.insert(entry2); + by_key.insert(entry3); + + // Verify iteration order matches create_time (ascending) + auto it = by_create_time.begin(); + ASSERT_NE(it, by_create_time.end()); + EXPECT_EQ(it->manifest.partition_id, "2022"); // Oldest first + EXPECT_EQ(it->manifest.create_time, base_time); + + ++it; + ASSERT_NE(it, by_create_time.end()); + EXPECT_EQ(it->manifest.partition_id, "2021"); + EXPECT_EQ(it->manifest.create_time, base_time + 100); + + ++it; + ASSERT_NE(it, by_create_time.end()); + EXPECT_EQ(it->manifest.partition_id, "2020"); + EXPECT_EQ(it->manifest.create_time, base_time + 300); + + ++it; + EXPECT_EQ(it, by_create_time.end()); +} + +} diff --git a/src/Storages/ObjectStorage/ObjectStorageFilePathGenerator.h b/src/Storages/ObjectStorage/ObjectStorageFilePathGenerator.h index a66965984bb0..a1f21dc502d5 100644 --- a/src/Storages/ObjectStorage/ObjectStorageFilePathGenerator.h +++ b/src/Storages/ObjectStorage/ObjectStorageFilePathGenerator.h @@ -55,7 +55,7 @@ namespace DB result += raw_path; - if (!raw_path.empty() && raw_path.back() != '/') + if (!result.empty() && result.back() != '/') { result += "/"; } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 1bda30398e17..c2b6054f6636 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -478,6 +478,7 @@ bool StorageObjectStorage::supportsImport() const return configuration->partition_strategy_type == PartitionStrategyFactory::StrategyType::HIVE; } + SinkToStoragePtr StorageObjectStorage::import( const std::string & file_name, Block & block_with_partition_values, @@ -514,6 +515,26 @@ SinkToStoragePtr StorageObjectStorage::import( local_context); } +void StorageObjectStorage::commitExportPartitionTransaction(const String & transaction_id, const String & partition_id, const Strings & exported_paths, ContextPtr local_context) +{ + const String commit_object = configuration->getRawPath().path + "/commit_" + partition_id + "_" + transaction_id; + + /// if file already exists, nothing to be done + if (object_storage->exists(StoredObject(commit_object))) + { + LOG_DEBUG(getLogger("StorageObjectStorage"), "Commit file already exists, nothing to be done: {}", commit_object); + return; + } + + auto out = object_storage->writeObject(StoredObject(commit_object), WriteMode::Rewrite, /* attributes= */ {}, DBMS_DEFAULT_BUFFER_SIZE, local_context->getWriteSettings()); + for (const auto & p : exported_paths) + { + out->write(p.data(), p.size()); + out->write("\n", 1); + } + out->finalize(); +} + void StorageObjectStorage::truncate( const ASTPtr & /* query */, const StorageMetadataPtr & /* metadata_snapshot */, diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index ebdf87b2c280..21d2224d6b0e 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -89,6 +89,12 @@ class StorageObjectStorage : public IStorage const std::optional & /* format_settings_ */, ContextPtr /* context */) override; + void commitExportPartitionTransaction( + const String & transaction_id, + const String & partition_id, + const Strings & exported_paths, + ContextPtr local_context) override; + void truncate( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index c190c1d3dcb5..0394170dbc4d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -926,6 +926,17 @@ bool StorageObjectStorageCluster::prefersLargeBlocks() const return IStorageCluster::prefersLargeBlocks(); } +void StorageObjectStorageCluster::commitExportPartitionTransaction( + const String & transaction_id, + const String & partition_id, + const Strings & exported_paths, + ContextPtr local_context) +{ + if (pure_storage) + return pure_storage->commitExportPartitionTransaction(transaction_id, partition_id, exported_paths, local_context); + return IStorageCluster::commitExportPartitionTransaction(transaction_id, partition_id, exported_paths, local_context); +} + bool StorageObjectStorageCluster::supportsPartitionBy() const { if (pure_storage) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index 583d549a89a0..72f5bf5dc009 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -134,6 +134,12 @@ class StorageObjectStorageCluster : public IStorageCluster ContextPtr /* context */) override; bool prefersLargeBlocks() const override; + void commitExportPartitionTransaction( + const String & transaction_id, + const String & partition_id, + const Strings & exported_paths, + ContextPtr local_context) override; + bool supportsPartitionBy() const override; bool supportsSubcolumns() const override; diff --git a/src/Storages/PartitionCommands.cpp b/src/Storages/PartitionCommands.cpp index 96f49a60e511..b8ef557604bc 100644 --- a/src/Storages/PartitionCommands.cpp +++ b/src/Storages/PartitionCommands.cpp @@ -140,6 +140,15 @@ std::optional PartitionCommand::parse(const ASTAlterCommand * res.to_table = command_ast->to_table; return res; } + if (command_ast->type == ASTAlterCommand::EXPORT_PARTITION) + { + PartitionCommand res; + res.type = EXPORT_PARTITION; + res.partition = command_ast->partition->clone(); + res.to_database = command_ast->to_database; + res.to_table = command_ast->to_table; + return res; + } return {}; } @@ -183,6 +192,8 @@ std::string PartitionCommand::typeToString() const return "REPLACE PARTITION"; case PartitionCommand::Type::EXPORT_PART: return "EXPORT PART"; + case PartitionCommand::Type::EXPORT_PARTITION: + return "EXPORT PARTITION"; default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Uninitialized partition command"); } diff --git a/src/Storages/PartitionCommands.h b/src/Storages/PartitionCommands.h index 15d2a7fb869f..e3f36d0e7c1f 100644 --- a/src/Storages/PartitionCommands.h +++ b/src/Storages/PartitionCommands.h @@ -34,6 +34,7 @@ struct PartitionCommand UNFREEZE_PARTITION, REPLACE_PARTITION, EXPORT_PART, + EXPORT_PARTITION, }; Type type = UNKNOWN; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index de377150f21e..0b7c3c600e5f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7,6 +7,7 @@ #include #include +#include "Common/ZooKeeper/IKeeper.h" #include #include #include @@ -67,6 +68,7 @@ #include #include #include +#include #include #include #include @@ -116,6 +118,13 @@ #include #include +#include "Functions/generateSnowflakeID.h" +#include "Interpreters/StorageID.h" +#include "QueryPipeline/QueryPlanResourceHolder.h" +#include "Storages/ExportReplicatedMergeTreePartitionManifest.h" +#include "Storages/ExportReplicatedMergeTreePartitionTaskEntry.h" +#include +#include #include #include @@ -183,6 +192,14 @@ namespace Setting extern const SettingsInt64 replication_wait_for_inactive_replica_timeout; extern const SettingsUInt64 select_sequential_consistency; extern const SettingsBool update_sequential_consistency; + extern const SettingsBool allow_experimental_export_merge_tree_part; + extern const SettingsBool export_merge_tree_partition_force_export; + extern const SettingsUInt64 export_merge_tree_partition_max_retries; + extern const SettingsUInt64 export_merge_tree_partition_manifest_ttl; + extern const SettingsBool output_format_parallel_formatting; + extern const SettingsBool output_format_parquet_parallel_encoding; + extern const SettingsMaxThreads max_threads; + extern const SettingsMergeTreePartExportFileAlreadyExistsPolicy export_merge_tree_part_file_already_exists_policy; } namespace MergeTreeSetting @@ -285,6 +302,12 @@ namespace ErrorCodes extern const int FAULT_INJECTED; extern const int CANNOT_FORGET_PARTITION; extern const int TIMEOUT_EXCEEDED; + extern const int INVALID_SETTING_VALUE; +} + +namespace ServerSetting +{ + extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; } namespace ActionLocks @@ -414,6 +437,9 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , merge_strategy_picker(*this) , queue(*this, merge_strategy_picker) , fetcher(*this) + , export_merge_tree_partition_task_entries_by_key(export_merge_tree_partition_task_entries.get()) + , export_merge_tree_partition_task_entries_by_transaction_id(export_merge_tree_partition_task_entries.get()) + , export_merge_tree_partition_task_entries_by_create_time(export_merge_tree_partition_task_entries.get()) , cleanup_thread(*this) , async_block_ids_cache(*this) , part_check_thread(*this) @@ -460,6 +486,31 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( /// Will be activated by restarting thread. mutations_finalizing_task->deactivate(); + if (getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + export_merge_tree_partition_manifest_updater = std::make_shared(*this); + + export_merge_tree_partition_task_scheduler = std::make_shared(*this); + + export_merge_tree_partition_updating_task = getContext()->getSchedulePool().createTask( + getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_updating_task)", [this] { exportMergeTreePartitionUpdatingTask(); }); + + export_merge_tree_partition_updating_task->deactivate(); + + export_merge_tree_partition_status_handling_task = getContext()->getSchedulePool().createTask( + getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_status_handling_task)", [this] { exportMergeTreePartitionStatusHandlingTask(); }); + + export_merge_tree_partition_status_handling_task->deactivate(); + + export_merge_tree_partition_watch_callback = std::make_shared(export_merge_tree_partition_updating_task->getWatchCallback()); + + export_merge_tree_partition_select_task = getContext()->getSchedulePool().createTask( + getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_select_task)", [this] { selectPartsToExport(); }); + + export_merge_tree_partition_select_task->deactivate(); + } + + bool has_zookeeper = getContext()->hasZooKeeper() || getContext()->hasAuxiliaryZooKeeper(zookeeper_info.zookeeper_name); if (has_zookeeper) { @@ -884,6 +935,7 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodesAttempt() const futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/quorum/last_part", String(), zkutil::CreateMode::Persistent)); futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/quorum/failed_parts", String(), zkutil::CreateMode::Persistent)); futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/mutations", String(), zkutil::CreateMode::Persistent)); + futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/exports", String(), zkutil::CreateMode::Persistent)); futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/quorum/parallel", String(), zkutil::CreateMode::Persistent)); @@ -1046,6 +1098,8 @@ bool StorageReplicatedMergeTree::createTableIfNotExistsAttempt(const StorageMeta zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/mutations", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/exports", "", + zkutil::CreateMode::Persistent)); /// And create first replica atomically. See also "createReplica" method that is used to create not the first replicas. @@ -4354,6 +4408,159 @@ void StorageReplicatedMergeTree::mutationsFinalizingTask() } } +void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() +{ + try + { + export_merge_tree_partition_manifest_updater->poll(); + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } + + + export_merge_tree_partition_updating_task->scheduleAfter(30 * 1000); +} + +void StorageReplicatedMergeTree::selectPartsToExport() +{ + try + { + export_merge_tree_partition_task_scheduler->run(); + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } + + export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); +} + +void StorageReplicatedMergeTree::exportMergeTreePartitionStatusHandlingTask() +{ + try + { + export_merge_tree_partition_manifest_updater->handleStatusChanges(); + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } +} + +std::vector StorageReplicatedMergeTree::getPartitionExportsInfo() const +{ + std::vector infos; + + const auto zk = getZooKeeper(); + const auto exports_path = fs::path(zookeeper_path) / "exports"; + std::vector children; + if (Coordination::Error::ZOK != zk->tryGetChildren(exports_path, children)) + { + LOG_INFO(log, "Failed to get children from exports path, returning empty export info list"); + return infos; + } + + for (const auto & child : children) + { + ReplicatedPartitionExportInfo info; + + const auto export_partition_path = fs::path(exports_path) / child; + std::string metadata_json; + if (!zk->tryGet(export_partition_path / "metadata.json", metadata_json)) + { + LOG_INFO(log, "Skipping {}: missing metadata.json", child); + continue; + } + + std::string status; + if (!zk->tryGet(export_partition_path / "status", status)) + { + LOG_INFO(log, "Skipping {}: missing status", child); + continue; + } + + std::vector processing_parts; + if (Coordination::Error::ZOK != zk->tryGetChildren(export_partition_path / "processing", processing_parts)) + { + LOG_INFO(log, "Skipping {}: missing processing parts", child); + continue; + } + + const auto parts_to_do = processing_parts.size(); + + std::string exception_replica; + std::string last_exception; + std::string exception_part; + std::size_t exception_count = 0; + + const auto exceptions_per_replica_path = export_partition_path / "exceptions_per_replica"; + + Strings exception_replicas; + if (Coordination::Error::ZOK != zk->tryGetChildren(exceptions_per_replica_path, exception_replicas)) + { + LOG_INFO(log, "Skipping {}: missing exceptions_per_replica", export_partition_path); + continue; + } + + for (const auto & replica : exception_replicas) + { + std::string exception_count_string; + if (!zk->tryGet(exceptions_per_replica_path / replica / "count", exception_count_string)) + { + LOG_INFO(log, "Skipping {}: missing count", replica); + continue; + } + + exception_count += std::stoull(exception_count_string.c_str()); + + if (last_exception.empty()) + { + const auto last_exception_path = exceptions_per_replica_path / replica / "last_exception"; + std::string last_exception_string; + if (!zk->tryGet(last_exception_path / "exception", last_exception_string)) + { + LOG_INFO(log, "Skipping {}: missing last_exception/exception", last_exception_path); + continue; + } + + std::string exception_part_zk; + if (!zk->tryGet(last_exception_path / "part", exception_part_zk)) + { + LOG_INFO(log, "Skipping {}: missing exception part", last_exception_path); + continue; + } + + exception_replica = replica; + last_exception = last_exception_string; + exception_part = exception_part_zk; + } + } + + const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + + info.destination_database = metadata.destination_database; + info.destination_table = metadata.destination_table; + info.partition_id = metadata.partition_id; + info.transaction_id = metadata.transaction_id; + info.create_time = metadata.create_time; + info.source_replica = metadata.source_replica; + info.parts_count = metadata.number_of_parts; + info.parts_to_do = parts_to_do; + info.parts = metadata.parts; + info.status = status; + info.exception_replica = exception_replica; + info.last_exception = last_exception; + info.exception_part = exception_part; + info.exception_count = exception_count; + + infos.emplace_back(std::move(info)); + } + + return infos; +} + StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::createLogEntryToMergeParts( zkutil::ZooKeeperPtr & zookeeper, @@ -5733,6 +5940,13 @@ void StorageReplicatedMergeTree::partialShutdown() mutations_updating_task->deactivate(); mutations_finalizing_task->deactivate(); + if (getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + export_merge_tree_partition_updating_task->deactivate(); + export_merge_tree_partition_select_task->deactivate(); + export_merge_tree_partition_status_handling_task->deactivate(); + } + cleanup_thread.stop(); async_block_ids_cache.stop(); part_check_thread.stop(); @@ -7877,6 +8091,180 @@ void StorageReplicatedMergeTree::fetchPartition( LOG_TRACE(log, "Fetch took {} sec. ({} tries)", watch.elapsedSeconds(), try_no); } +void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & command, ContextPtr query_context) +{ + if (!query_context->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Exporting merge tree partition is experimental. Set the server setting `enable_experimental_export_merge_tree_partition_feature` to enable it"); + } + + const auto dest_database = query_context->resolveDatabase(command.to_database); + const auto dest_table = command.to_table; + const auto dest_storage_id = StorageID(dest_database, dest_table); + auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, dest_table}, query_context); + + if (dest_storage->getStorageID() == this->getStorageID()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Exporting to the same table is not allowed"); + } + + if (!dest_storage->supportsImport()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Destination storage {} does not support MergeTree parts or uses unsupported partitioning", dest_storage->getName()); + + auto query_to_string = [] (const ASTPtr & ast) + { + return ast ? ast->formatWithSecretsOneLine() : ""; + }; + + auto src_snapshot = getInMemoryMetadataPtr(); + auto destination_snapshot = dest_storage->getInMemoryMetadataPtr(); + + if (destination_snapshot->getColumns().getAllPhysical().sizeOfDifference(src_snapshot->getColumns().getAllPhysical())) + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Tables have different structure"); + + if (query_to_string(src_snapshot->getPartitionKeyAST()) != query_to_string(destination_snapshot->getPartitionKeyAST())) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key"); + + zkutil::ZooKeeperPtr zookeeper = getZooKeeperAndAssertNotReadonly(); + + const String partition_id = getPartitionIDFromQuery(command.partition, query_context); + + const auto exports_path = fs::path(zookeeper_path) / "exports"; + + const auto export_key = partition_id + "_" + dest_storage_id.getQualifiedName().getFullName(); + + const auto partition_exports_path = fs::path(exports_path) / export_key; + + /// check if entry already exists + if (zookeeper->exists(partition_exports_path)) + { + LOG_INFO(log, "Export with key {} is already exported or it is being exported. Checking if it has expired so that we can overwrite it", export_key); + + bool has_expired = false; + + if (zookeeper->exists(fs::path(partition_exports_path) / "metadata.json")) + { + std::string metadata_json; + if (zookeeper->tryGet(fs::path(partition_exports_path) / "metadata.json", metadata_json)) + { + const auto manifest = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + + const auto now = time(nullptr); + const auto expiration_time = manifest.create_time + manifest.ttl_seconds; + + LOG_INFO(log, "Export with key {} has expiration time {}, now is {}", export_key, expiration_time, now); + + if (static_cast(expiration_time) < now) + { + has_expired = true; + } + } + } + + if (!has_expired && !query_context->getSettingsRef()[Setting::export_merge_tree_partition_force_export]) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Export with key {} already exported or it is being exported, and it has not expired. Set `export_merge_tree_partition_force_export` to overwrite it.", export_key); + } + + LOG_INFO(log, "Overwriting export with key {}", export_key); + + /// Not putting in ops (same transaction) because we can't construct a "tryRemoveRecursive" request. + /// It is possible that the zk being used does not support RemoveRecursive requests. + /// It is ok for this to be non transactional. Worst case scenario an on-going export is going to be killed and a new task won't be scheduled. + zookeeper->tryRemoveRecursive(partition_exports_path); + } + + Coordination::Requests ops; + + ops.emplace_back(zkutil::makeCreateRequest(partition_exports_path, "", zkutil::CreateMode::Persistent)); + + auto data_parts_lock = lockParts(); + + const auto parts = getDataPartsVectorInPartitionForInternalUsage(MergeTreeDataPartState::Active, partition_id, &data_parts_lock); + + if (parts.empty()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition {} doesn't exist", partition_id); + } + + std::vector part_names; + for (const auto & part : parts) + { + part_names.push_back(part->name); + } + + /// TODO arthur somehow check if the list of parts is updated "enough" + + ExportReplicatedMergeTreePartitionManifest manifest; + + manifest.transaction_id = generateSnowflakeIDString(); + manifest.partition_id = partition_id; + manifest.destination_database = dest_database; + manifest.destination_table = dest_table; + manifest.source_replica = replica_name; + manifest.number_of_parts = part_names.size(); + manifest.parts = part_names; + manifest.create_time = time(nullptr); + manifest.max_retries = query_context->getSettingsRef()[Setting::export_merge_tree_partition_max_retries]; + manifest.ttl_seconds = query_context->getSettingsRef()[Setting::export_merge_tree_partition_manifest_ttl]; + manifest.max_threads = query_context->getSettingsRef()[Setting::max_threads]; + manifest.parallel_formatting = query_context->getSettingsRef()[Setting::output_format_parallel_formatting]; + manifest.parquet_parallel_encoding = query_context->getSettingsRef()[Setting::output_format_parquet_parallel_encoding]; + + manifest.file_already_exists_policy = query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value; + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "metadata.json", + manifest.toJsonString(), + zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "exceptions_per_replica", + "", + zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "processing", + "", + zkutil::CreateMode::Persistent)); + + for (const auto & part : part_names) + { + ExportReplicatedMergeTreePartitionProcessingPartEntry entry; + entry.status = ExportReplicatedMergeTreePartitionProcessingPartEntry::Status::PENDING; + entry.part_name = part; + entry.retry_count = 0; + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "processing" / part, + entry.toJsonString(), + zkutil::CreateMode::Persistent)); + } + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "processed", + "", + zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "locks", + "", + zkutil::CreateMode::Persistent)); + + /// status: IN_PROGRESS, COMPLETED, FAILED + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "status", + "PENDING", + zkutil::CreateMode::Persistent)); + + Coordination::Responses responses; + Coordination::Error code = zookeeper->tryMulti(ops, responses); + + if (code != Coordination::Error::ZOK) + throw zkutil::KeeperException::fromPath(code, partition_exports_path); +} + void StorageReplicatedMergeTree::forgetPartition(const ASTPtr & partition, ContextPtr query_context) { @@ -9284,6 +9672,89 @@ CancellationCode StorageReplicatedMergeTree::killPartMoveToShard(const UUID & ta return part_moves_between_shards_orchestrator.killPartMoveToShard(task_uuid); } +CancellationCode StorageReplicatedMergeTree::killExportPartition(const String & transaction_id) +{ + auto try_set_status_to_killed = [this](const zkutil::ZooKeeperPtr & zk, const std::string & status_path) + { + Coordination::Stat stat; + std::string status_from_zk_string; + + if (!zk->tryGet(status_path, status_from_zk_string, &stat)) + { + /// found entry locally, but not in zk. It might have been deleted by another replica and we did not have time to update the local entry. + LOG_INFO(log, "Export partition task not found in zk, can not cancel it"); + return CancellationCode::CancelCannotBeSent; + } + + const auto status_from_zk = magic_enum::enum_cast(status_from_zk_string); + + if (!status_from_zk) + { + LOG_INFO(log, "Export partition task status is invalid, can not cancel it"); + return CancellationCode::CancelCannotBeSent; + } + + if (status_from_zk.value() != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + LOG_INFO(log, "Export partition task is {}, can not cancel it", String(magic_enum::enum_name(status_from_zk.value()))); + return CancellationCode::CancelCannotBeSent; + } + + if (zk->trySet(status_path, String(magic_enum::enum_name(ExportReplicatedMergeTreePartitionTaskEntry::Status::KILLED)), stat.version) != Coordination::Error::ZOK) + { + LOG_INFO(log, "Status has been updated while trying to kill the export partition task, can not cancel it"); + return CancellationCode::CancelCannotBeSent; + } + + return CancellationCode::CancelSent; + }; + + std::lock_guard lock(export_merge_tree_partition_mutex); + + const auto zk = getZooKeeper(); + + /// if we have the entry locally, no need to list from zk. we can save some requests. + const auto & entry = export_merge_tree_partition_task_entries_by_transaction_id.find(transaction_id); + if (entry != export_merge_tree_partition_task_entries_by_transaction_id.end()) + { + LOG_INFO(log, "Export partition task found locally, trying to cancel it"); + /// found locally, no need to get children on zk + if (entry->status != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + LOG_INFO(log, "Export partition task is not pending, can not cancel it"); + return CancellationCode::CancelCannotBeSent; + } + + return try_set_status_to_killed(zk, fs::path(zookeeper_path) / "exports" / entry->getCompositeKey() / "status"); + } + else + { + LOG_INFO(log, "Export partition task not found locally, trying to find it on zk"); + /// for some reason, we don't have the entry locally. ls on zk to find the entry + const auto exports_path = fs::path(zookeeper_path) / "exports"; + + const auto export_keys = zk->getChildren(exports_path); + String export_key_to_be_cancelled; + + for (const auto & export_key : export_keys) + { + std::string metadata_json; + if (!zk->tryGet(fs::path(exports_path) / export_key / "metadata.json", metadata_json)) + continue; + const auto manifest = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + if (manifest.transaction_id == transaction_id) + { + LOG_INFO(log, "Export partition task found on zk, trying to cancel it"); + return try_set_status_to_killed(zk, fs::path(exports_path) / export_key / "status"); + } + } + } + + LOG_INFO(log, "Export partition task not found, can not cancel it"); + + return CancellationCode::NotFound; +} + void StorageReplicatedMergeTree::getCommitPartOps( Coordination::Requests & ops, const DataPartPtr & part, diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 5abf14c1400d..712ba0ba4183 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -13,6 +13,10 @@ #include #include #include +#include "Interpreters/CancellationCode.h" +#include "Storages/MergeTree/ExportPartitionManifestUpdatingTask.h" +#include "Storages/MergeTree/ExportPartitionTaskScheduler.h" +#include #include #include #include @@ -97,6 +101,8 @@ namespace DB class ZooKeeperWithFaultInjection; using ZooKeeperWithFaultInjectionPtr = std::shared_ptr; +struct ReplicatedPartitionExportInfo; + class StorageReplicatedMergeTree final : public MergeTreeData { public: @@ -371,6 +377,8 @@ class StorageReplicatedMergeTree final : public MergeTreeData using ShutdownDeadline = std::chrono::time_point; void waitForUniquePartsToBeFetchedByOtherReplicas(ShutdownDeadline shutdown_deadline); + std::vector getPartitionExportsInfo() const; + private: std::atomic_bool are_restoring_replica {false}; @@ -396,6 +404,8 @@ class StorageReplicatedMergeTree final : public MergeTreeData friend class MergeFromLogEntryTask; friend class MutateFromLogEntryTask; friend class ReplicatedMergeMutateTaskBase; + friend class ExportPartitionManifestUpdatingTask; + friend class ExportPartitionTaskScheduler; using MergeStrategyPicker = ReplicatedMergeTreeMergeStrategyPicker; using LogEntry = ReplicatedMergeTreeLogEntry; @@ -510,6 +520,26 @@ class StorageReplicatedMergeTree final : public MergeTreeData /// A task that marks finished mutations as done. BackgroundSchedulePoolTaskHolder mutations_finalizing_task; + BackgroundSchedulePoolTaskHolder export_merge_tree_partition_updating_task; + + /// mostly handle kill operations + BackgroundSchedulePoolTaskHolder export_merge_tree_partition_status_handling_task; + std::shared_ptr export_merge_tree_partition_manifest_updater; + + std::shared_ptr export_merge_tree_partition_task_scheduler; + + Coordination::WatchCallbackPtr export_merge_tree_partition_watch_callback; + + std::mutex export_merge_tree_partition_mutex; + + BackgroundSchedulePoolTaskHolder export_merge_tree_partition_select_task; + + ExportPartitionTaskEntriesContainer export_merge_tree_partition_task_entries; + + // Convenience references to indexes + ExportPartitionTaskEntriesContainer::index::type & export_merge_tree_partition_task_entries_by_key; + ExportPartitionTaskEntriesContainer::index::type & export_merge_tree_partition_task_entries_by_transaction_id; + ExportPartitionTaskEntriesContainer::index::type & export_merge_tree_partition_task_entries_by_create_time; /// A thread that removes old parts, log entries, and blocks. ReplicatedMergeTreeCleanupThread cleanup_thread; @@ -737,6 +767,14 @@ class StorageReplicatedMergeTree final : public MergeTreeData /// Checks if some mutations are done and marks them as done. void mutationsFinalizingTask(); + void selectPartsToExport(); + + /// update in-memory list of partition exports + void exportMergeTreePartitionUpdatingTask(); + + /// handle status changes for export partition tasks + void exportMergeTreePartitionStatusHandlingTask(); + /** Write the selected parts to merge into the log, * Call when merge_selecting_mutex is locked. * Returns false if any part is not in ZK. @@ -916,6 +954,7 @@ class StorageReplicatedMergeTree final : public MergeTreeData void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr query_context) override; void movePartitionToShard(const ASTPtr & partition, bool move_part, const String & to, ContextPtr query_context) override; CancellationCode killPartMoveToShard(const UUID & task_uuid) override; + CancellationCode killExportPartition(const String & transaction_id) override; void fetchPartition( const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, @@ -923,7 +962,8 @@ class StorageReplicatedMergeTree final : public MergeTreeData bool fetch_part, ContextPtr query_context) override; void forgetPartition(const ASTPtr & partition, ContextPtr query_context) override; - + + void exportPartitionToTable(const PartitionCommand &, ContextPtr) override; /// NOTE: there are no guarantees for concurrent merges. Dropping part can /// be concurrently merged into some covering part and dropPart will do diff --git a/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp b/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp new file mode 100644 index 000000000000..018f0c8ffac7 --- /dev/null +++ b/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp @@ -0,0 +1,143 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "Columns/ColumnString.h" +#include "Storages/VirtualColumnUtils.h" + + +namespace DB +{ + +ColumnsDescription StorageSystemReplicatedPartitionExports::getColumnsDescription() +{ + return ColumnsDescription + { + {"source_database", std::make_shared(), "Name of the source database."}, + {"source_table", std::make_shared(), "Name of the source table."}, + {"destination_database", std::make_shared(), "Name of the destination database."}, + {"destination_table", std::make_shared(), "Name of the destination table."}, + {"create_time", std::make_shared(), "Date and time when the export command was submitted"}, + {"partition_id", std::make_shared(), "ID of the partition"}, + {"transaction_id", std::make_shared(), "ID of the transaction."}, + {"source_replica", std::make_shared(), "Name of the source replica."}, + {"parts", std::make_shared(std::make_shared()), "List of part names to be exported."}, + {"parts_count", std::make_shared(), "Number of parts in the export."}, + {"parts_to_do", std::make_shared(), "Number of parts pending to be exported."}, + {"status", std::make_shared(), "Status of the export."}, + {"exception_replica", std::make_shared(), "Replica that caused the last exception"}, + {"last_exception", std::make_shared(), "Last exception message of any part (not necessarily the last global exception)"}, + {"exception_part", std::make_shared(), "Part that caused the last exception"}, + {"exception_count", std::make_shared(), "Number of global exceptions"}, + }; +} + +void StorageSystemReplicatedPartitionExports::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const +{ + const auto access = context->getAccess(); + const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); + + std::map> replicated_merge_tree_tables; + for (const auto & db : DatabaseCatalog::instance().getDatabases()) + { + /// Check if database can contain MergeTree tables + if (!db.second->canContainMergeTreeTables()) + continue; + + const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first); + + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) + { + const auto & table = iterator->table(); + if (!table) + continue; + + StorageReplicatedMergeTree * table_replicated = dynamic_cast(table.get()); + if (!table_replicated) + continue; + + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name())) + continue; + + replicated_merge_tree_tables[db.first][iterator->name()] = table; + } + } + + MutableColumnPtr col_database_mut = ColumnString::create(); + MutableColumnPtr col_table_mut = ColumnString::create(); + + for (auto & db : replicated_merge_tree_tables) + { + for (auto & table : db.second) + { + col_database_mut->insert(db.first); + col_table_mut->insert(table.first); + } + } + + ColumnPtr col_database = std::move(col_database_mut); + ColumnPtr col_table = std::move(col_table_mut); + + /// Determine what tables are needed by the conditions in the query. + { + Block filtered_block + { + { col_database, std::make_shared(), "database" }, + { col_table, std::make_shared(), "table" }, + }; + + VirtualColumnUtils::filterBlockWithPredicate(predicate, filtered_block, context); + + if (!filtered_block.rows()) + return; + + col_database = filtered_block.getByName("database").column; + col_table = filtered_block.getByName("table").column; + } + + for (size_t i_storage = 0; i_storage < col_database->size(); ++i_storage) + { + const auto database = (*col_database)[i_storage].safeGet(); + const auto table = (*col_table)[i_storage].safeGet(); + + std::vector partition_exports_info; + { + const IStorage * storage = replicated_merge_tree_tables[database][table].get(); + if (const auto * replicated_merge_tree = dynamic_cast(storage)) + partition_exports_info = replicated_merge_tree->getPartitionExportsInfo(); + } + + for (const ReplicatedPartitionExportInfo & info : partition_exports_info) + { + std::size_t i = 0; + res_columns[i++]->insert(database); + res_columns[i++]->insert(table); + res_columns[i++]->insert(info.destination_database); + res_columns[i++]->insert(info.destination_table); + res_columns[i++]->insert(info.create_time); + res_columns[i++]->insert(info.partition_id); + res_columns[i++]->insert(info.transaction_id); + res_columns[i++]->insert(info.source_replica); + Array parts_array; + parts_array.reserve(info.parts.size()); + for (const auto & part : info.parts) + parts_array.push_back(part); + res_columns[i++]->insert(parts_array); + res_columns[i++]->insert(info.parts_count); + res_columns[i++]->insert(info.parts_to_do); + res_columns[i++]->insert(info.status); + res_columns[i++]->insert(info.exception_replica); + res_columns[i++]->insert(info.last_exception); + res_columns[i++]->insert(info.exception_part); + res_columns[i++]->insert(info.exception_count); + } + } +} + +} diff --git a/src/Storages/System/StorageSystemReplicatedPartitionExports.h b/src/Storages/System/StorageSystemReplicatedPartitionExports.h new file mode 100644 index 000000000000..de2547437c21 --- /dev/null +++ b/src/Storages/System/StorageSystemReplicatedPartitionExports.h @@ -0,0 +1,42 @@ +#pragma once + +#include + +namespace DB +{ + +class Context; + +struct ReplicatedPartitionExportInfo +{ + String destination_database; + String destination_table; + String partition_id; + String transaction_id; + time_t create_time; + String source_replica; + size_t parts_count; + size_t parts_to_do; + std::vector parts; + String status; + std::string exception_replica; + std::string last_exception; + std::string exception_part; + size_t exception_count; +}; + +class StorageSystemReplicatedPartitionExports final : public IStorageSystemOneBlock +{ +public: + + std::string getName() const override { return "SystemReplicatedPartitionExports"; } + + static ColumnsDescription getColumnsDescription(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 5a3a4d30599d..d6ce54003aea 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -1,10 +1,11 @@ #include +#include #include "config.h" #include #include #include - +#include #include #include #include @@ -129,6 +130,11 @@ namespace DB { +namespace ServerSetting +{ + extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; +} + void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, bool has_zookeeper) { attachNoDescription(context, system_database, "one", "This table contains a single row with a single dummy UInt8 column containing the value 0. Used when the table is not specified explicitly, for example in queries like `SELECT 1`."); @@ -211,6 +217,10 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "merges", "Contains a list of merges currently executing merges of MergeTree tables and their progress. Each merge operation is represented by a single row."); attach(context, system_database, "moves", "Contains information about in-progress data part moves of MergeTree tables. Each data part movement is represented by a single row."); attach(context, system_database, "exports", "Contains a list of exports currently executing exports of MergeTree tables and their progress. Each export operation is represented by a single row."); + if (context->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + attach(context, system_database, "replicated_partition_exports", "Contains a list of partition exports of ReplicatedMergeTree tables and their progress. Each export operation is represented by a single row."); + } attach(context, system_database, "mutations", "Contains a list of mutations and their progress. Each mutation command is represented by a single row."); attachNoDescription(context, system_database, "replicas", "Contains information and status of all table replicas on current server. Each replica is represented by a single row."); attach(context, system_database, "replication_queue", "Contains information about tasks from replication queues stored in ClickHouse Keeper, or ZooKeeper, for each table replica."); diff --git a/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml b/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml new file mode 100644 index 000000000000..72014c9de4db --- /dev/null +++ b/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml @@ -0,0 +1,3 @@ + + 1 + diff --git a/tests/config/install.sh b/tests/config/install.sh index b8874097c940..90b70f3ce5cc 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -85,6 +85,7 @@ ln -sf $SRC_PATH/config.d/blob_storage_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/custom_settings_prefixes.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/database_catalog_drop_table_concurrency.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_access_control_improvements.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/enable_experimental_export_merge_tree_partition.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/macros.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/secure_ports.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/clusters.xml $DEST_SERVER_PATH/config.d/ diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/__init__.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/allow_experimental_export_partition.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/allow_experimental_export_partition.xml new file mode 100644 index 000000000000..f8c5fab1a3be --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/allow_experimental_export_partition.xml @@ -0,0 +1,3 @@ + + 1 + \ No newline at end of file diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/disable_experimental_export_partition.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/disable_experimental_export_partition.xml new file mode 100644 index 000000000000..ba6508ebd660 --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/disable_experimental_export_partition.xml @@ -0,0 +1,3 @@ + + 0 + \ No newline at end of file diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/named_collections.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/named_collections.xml new file mode 100644 index 000000000000..d46920b7ba88 --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/named_collections.xml @@ -0,0 +1,9 @@ + + + + http://minio1:9001/root/data + minio + ClickHouse_Minio_P@ssw0rd + + + \ No newline at end of file diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/users.d/profile.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/users.d/profile.xml new file mode 100644 index 000000000000..518f29708929 --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/users.d/profile.xml @@ -0,0 +1,8 @@ + + + + 3 + + + + diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py new file mode 100644 index 000000000000..3bf14409e82c --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -0,0 +1,687 @@ +import logging +import pytest +import random +import string +import time +from typing import Optional +import uuid + +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "replica1", + main_configs=["configs/named_collections.xml", "configs/allow_experimental_export_partition.xml"], + user_configs=["configs/users.d/profile.xml"], + with_minio=True, + stay_alive=True, + with_zookeeper=True, + ) + cluster.add_instance( + "replica2", + main_configs=["configs/named_collections.xml", "configs/allow_experimental_export_partition.xml"], + user_configs=["configs/users.d/profile.xml"], + with_minio=True, + stay_alive=True, + with_zookeeper=True, + ) + # node that does not participate in the export, but will have visibility over the s3 table + cluster.add_instance( + "watcher_node", + main_configs=["configs/named_collections.xml"], + user_configs=[], + with_minio=True, + ) + cluster.add_instance( + "replica_with_export_disabled", + main_configs=["configs/named_collections.xml", "configs/disable_experimental_export_partition.xml"], + user_configs=["configs/users.d/profile.xml"], + with_minio=True, + stay_alive=True, + with_zookeeper=True, + ) + logging.info("Starting cluster...") + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def create_s3_table(node, s3_table): + node.query(f"CREATE TABLE {s3_table} (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='{s3_table}', format=Parquet, partition_strategy='hive') PARTITION BY year") + + +def create_tables_and_insert_data(node, mt_table, s3_table, replica_name): + node.query(f"CREATE TABLE {mt_table} (id UInt64, year UInt16) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{mt_table}', '{replica_name}') PARTITION BY year ORDER BY tuple()") + node.query(f"INSERT INTO {mt_table} VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)") + + create_s3_table(node, s3_table) + + +def test_restart_nodes_during_export(cluster): + node = cluster.instances["replica1"] + node2 = cluster.instances["replica2"] + watcher_node = cluster.instances["watcher_node"] + + mt_table = "disaster_mt_table" + s3_table = "disaster_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") + create_s3_table(watcher_node, s3_table) + + # Block S3/MinIO requests to keep exports alive via retry mechanism + # This allows ZooKeeper operations to proceed quickly + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses_node1 = { + "destination": node.ip_address, + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_responses_node1) + + pm_rule_reject_responses_node2 = { + "destination": node2.ip_address, + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_responses_node2) + + # Block requests to MinIO (destination: MinIO, destination_port: minio_port) + pm_rule_reject_requests = { + "destination": minio_ip, + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_requests) + + export_queries = f""" + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2020' TO TABLE {s3_table} + SETTINGS export_merge_tree_partition_max_retries = 50; + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2021' TO TABLE {s3_table} + SETTINGS export_merge_tree_partition_max_retries = 50; + """ + + node.query(export_queries) + + # wait for the exports to start + time.sleep(3) + + node.stop_clickhouse(kill=True) + node2.stop_clickhouse(kill=True) + + assert watcher_node.query(f"SELECT count() FROM {s3_table} where year = 2020") == '0\n', "Partition 2020 was written to S3 during network delay crash" + + assert watcher_node.query(f"SELECT count() FROM {s3_table} where year = 2021") == '0\n', "Partition 2021 was written to S3 during network delay crash" + + # start the nodes, they should finish the export + node.start_clickhouse() + node2.start_clickhouse() + + time.sleep(5) + + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") != f'0\n', "Export of partition 2020 did not resume after crash" + + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2021") != f'0\n', "Export of partition 2021 did not resume after crash" + + +def test_kill_export(cluster): + node = cluster.instances["replica1"] + node2 = cluster.instances["replica2"] + watcher_node = cluster.instances["watcher_node"] + + mt_table = "kill_export_mt_table" + s3_table = "kill_export_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") + + # Block S3/MinIO requests to keep exports alive via retry mechanism + # This allows ZooKeeper operations (KILL) to proceed quickly + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "destination": node.ip_address, + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_responses) + + # Block requests to MinIO (destination: MinIO, destination_port: minio_port) + pm_rule_reject_requests = { + "destination": minio_ip, + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_requests) + + export_queries = f""" + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2020' TO TABLE {s3_table} + SETTINGS export_merge_tree_partition_max_retries = 50; + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2021' TO TABLE {s3_table} + SETTINGS export_merge_tree_partition_max_retries = 50; + """ + + node.query(export_queries) + + # Kill only 2020 while S3 is blocked - retry mechanism keeps exports alive + # ZooKeeper operations (KILL) proceed quickly since only S3 is blocked + node.query(f"KILL EXPORT PARTITION WHERE partition_id = '2020' and source_table = '{mt_table}' and destination_table = '{s3_table}'") + + # wait for 2021 to finish + time.sleep(5) + + # checking for the commit file because maybe the data file was too fast? + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '0\n', "Partition 2020 was written to S3, it was not killed as expected" + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2021_*', format=LineAsString)") != f'0\n', "Partition 2021 was not written to S3, but it should have been" + + # check system.replicated_partition_exports for the export, status should be KILLED + assert node.query(f"SELECT status FROM system.replicated_partition_exports WHERE partition_id = '2020' and source_table = '{mt_table}' and destination_table = '{s3_table}'") == 'KILLED\n', "Partition 2020 was not killed as expected" + assert node.query(f"SELECT status FROM system.replicated_partition_exports WHERE partition_id = '2021' and source_table = '{mt_table}' and destination_table = '{s3_table}'") == 'COMPLETED\n', "Partition 2021 was not completed, this is unexpected" + + +def test_drop_source_table_during_export(cluster): + node = cluster.instances["replica1"] + # node2 = cluster.instances["replica2"] + watcher_node = cluster.instances["watcher_node"] + + mt_table = "drop_source_table_during_export_mt_table" + s3_table = "drop_source_table_during_export_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + # create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") + create_s3_table(watcher_node, s3_table) + + # Block S3/MinIO requests to keep exports alive via retry mechanism + # This allows ZooKeeper operations (KILL) to proceed quickly + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "destination": node.ip_address, + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_responses) + + # Block requests to MinIO (destination: MinIO, destination_port: minio_port) + pm_rule_reject_requests = { + "destination": minio_ip, + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_requests) + + export_queries = f""" + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2020' TO TABLE {s3_table}; + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2021' TO TABLE {s3_table}; + """ + + node.query(export_queries) + + # This should kill the background operations and drop the table + node.query(f"DROP TABLE {mt_table}") + + # Sleep some time to let the export finish (assuming it was not properly cancelled) + time.sleep(10) + + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_*', format=LineAsString)") == '0\n', "Background operations completed even with the table dropped" + + +def test_concurrent_exports_to_different_targets(cluster): + node = cluster.instances["replica1"] + + mt_table = "concurrent_diff_targets_mt_table" + s3_table_a = "concurrent_diff_targets_s3_a" + s3_table_b = "concurrent_diff_targets_s3_b" + + create_tables_and_insert_data(node, mt_table, s3_table_a, "replica1") + create_s3_table(node, s3_table_b) + + # Launch two exports of the same partition to two different S3 tables concurrently + with PartitionManager() as pm: + pm.add_network_delay(node, delay_ms=1000) + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table_a}" + ) + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table_b}" + ) + + time.sleep(5) + + # Both targets should receive the same data independently + assert node.query(f"SELECT count() FROM {s3_table_a} WHERE year = 2020") == '3\n', "First target did not receive expected rows" + assert node.query(f"SELECT count() FROM {s3_table_b} WHERE year = 2020") == '3\n', "Second target did not receive expected rows" + + # And both should have a commit marker + assert node.query( + f"SELECT count() FROM s3(s3_conn, filename='{s3_table_a}/commit_2020_*', format=LineAsString)" + ) != '0\n', "Commit file missing for first target" + assert node.query( + f"SELECT count() FROM s3(s3_conn, filename='{s3_table_b}/commit_2020_*', format=LineAsString)" + ) != '0\n', "Commit file missing for second target" + + +def test_failure_is_logged_in_system_table(cluster): + node = cluster.instances["replica1"] + + mt_table = "failure_is_logged_in_system_table_mt_table" + s3_table = "failure_is_logged_in_system_table_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # Block traffic to/from MinIO to force upload errors and retries, following existing S3 tests style + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "destination": node.ip_address, + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_responses) + + # Also block requests to MinIO (destination: MinIO, destination_port: 9001) with REJECT to fail fast + pm_rule_reject_requests = { + "destination": minio_ip, + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_requests) + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_max_retries=1;" + ) + + # Wait so that the export fails + time.sleep(5) + + # Network restored; verify the export is marked as FAILED in the system table + # Also verify we captured at least one exception and no commit file exists + status = node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) + + assert status.strip() == "FAILED", f"Expected FAILED status, got: {status!r}" + + exception_count = node.query( + f""" + SELECT any(exception_count) FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) + assert int(exception_count.strip()) > 0, "Expected non-zero exception_count in system.replicated_partition_exports" + + # No commit should have been produced for this partition + assert node.query( + f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)" + ) == '0\n', "Commit file exists despite forced S3 failures" + + +def test_inject_short_living_failures(cluster): + node = cluster.instances["replica1"] + + mt_table = "inject_short_living_failures_mt_table" + s3_table = "inject_short_living_failures_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # Block traffic to/from MinIO to force upload errors and retries, following existing S3 tests style + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "destination": node.ip_address, + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_responses) + + # Also block requests to MinIO (destination: MinIO, destination_port: 9001) with REJECT to fail fast + pm_rule_reject_requests = { + "destination": minio_ip, + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm._add_rule(pm_rule_reject_requests) + + # set big max_retries so that the export does not fail completely + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_max_retries=100;" + ) + + # wait only for a second to get at least one failure, but not enough to finish the export + time.sleep(5) + + # wait for the export to finish + time.sleep(5) + + # Assert the export succeeded + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") == '3\n', "Export did not succeed" + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '1\n', "Export did not succeed" + + # check system.replicated_partition_exports for the export + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) == "COMPLETED\n", "Export should be marked as COMPLETED" + + exception_count = node.query( + f""" + SELECT exception_count FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) + assert int(exception_count.strip()) >= 1, "Expected at least one exception" + + +def test_export_ttl(cluster): + node = cluster.instances["replica1"] + + mt_table = "export_ttl_mt_table" + s3_table = "export_ttl_s3_table" + + expiration_time = 5 + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # start export + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_manifest_ttl={expiration_time};") + + # assert that I get an error when trying to export the same partition again, query_and_get_error + error = node.query_and_get_error(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table};") + assert "Export with key" in error, "Expected error about expired export" + + # wait for the export to finish and for the manifest to expire + time.sleep(expiration_time) + + # assert that the export succeeded, check the commit file + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '1\n', "Export did not succeed" + + # start export again + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}") + + # wait for the export to finish + time.sleep(expiration_time) + + # assert that the export succeeded, check the commit file + # there should be two commit files now, one for the first export and one for the second export + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '2\n', "Export did not succeed" + + +def test_export_partition_file_already_exists_policy(cluster): + node = cluster.instances["replica1"] + + mt_table = "export_partition_file_already_exists_policy_mt_table" + s3_table = "export_partition_file_already_exists_policy_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # stop merges so part names remain stable. it is important for the test. + node.query(f"SYSTEM STOP MERGES {mt_table}") + + # Export all parts + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}", + ) + + # check system.replicated_partition_exports for the export + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) == "COMPLETED\n", "Export should be marked as COMPLETED" + + # wait for the exports to finish + time.sleep(3) + + # try to export the partition + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_force_export=1" + ) + + time.sleep(3) + + assert node.query( + f""" + SELECT count() FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + AND status = 'COMPLETED' + """ + ) == '1\n', "Expected the export to be marked as COMPLETED" + + # overwrite policy + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_force_export=1, export_merge_tree_part_file_already_exists_policy='overwrite'" + ) + + # wait for the export to finish + time.sleep(3) + + # check system.replicated_partition_exports for the export + # ideally we would make sure the transaction id is different, but I do not have the time to do that now + assert node.query( + f""" + SELECT count() FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + AND status = 'COMPLETED' + """ + ) == '1\n', "Expected the export to be marked as COMPLETED" + + # last but not least, let's try with the error policy + # max retries = 1 so it fails fast + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_force_export=1, export_merge_tree_part_file_already_exists_policy='error', export_merge_tree_partition_max_retries=1", + ) + + # wait for the export to finish + time.sleep(3) + + # check system.replicated_partition_exports for the export + assert node.query( + f""" + SELECT count() FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + AND status = 'FAILED' + """ + ) == '1\n', "Expected the export to be marked as FAILED" + + +def test_export_partition_feature_is_disabled(cluster): + replica_with_export_disabled = cluster.instances["replica_with_export_disabled"] + + mt_table = "export_partition_feature_is_disabled_mt_table" + s3_table = "export_partition_feature_is_disabled_s3_table" + + create_tables_and_insert_data(replica_with_export_disabled, mt_table, s3_table, "replica1") + + error = replica_with_export_disabled.query_and_get_error(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table};") + assert "experimental" in error, "Expected error about disabled feature" + + # make sure kill operation also throws + error = replica_with_export_disabled.query_and_get_error(f"KILL EXPORT PARTITION WHERE partition_id = '2020' and source_table = '{mt_table}' and destination_table = '{s3_table}'") + assert "experimental" in error, "Expected error about disabled feature" + + +def test_export_partition_permissions(cluster): + """Test that export partition validates permissions correctly: + - User needs ALTER permission on source table + - User needs INSERT permission on destination table + """ + node = cluster.instances["replica1"] + + mt_table = "permissions_mt_table" + s3_table = "permissions_s3_table" + + # Create tables as default user + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # Create test users with specific permissions + node.query("CREATE USER IF NOT EXISTS user_no_alter IDENTIFIED WITH no_password") + node.query("CREATE USER IF NOT EXISTS user_no_insert IDENTIFIED WITH no_password") + node.query("CREATE USER IF NOT EXISTS user_with_permissions IDENTIFIED WITH no_password") + + # Grant basic access to all users + node.query(f"GRANT SELECT ON {mt_table} TO user_no_alter") + node.query(f"GRANT SELECT ON {s3_table} TO user_no_alter") + + # user_no_insert has ALTER on source but no INSERT on destination + node.query(f"GRANT ALTER ON {mt_table} TO user_no_insert") + node.query(f"GRANT SELECT ON {s3_table} TO user_no_insert") + + # user_with_permissions has both ALTER and INSERT + node.query(f"GRANT ALTER ON {mt_table} TO user_with_permissions") + node.query(f"GRANT INSERT ON {s3_table} TO user_with_permissions") + + # Test 1: User without ALTER permission should fail + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}", + user="user_no_alter" + ) + assert "ACCESS_DENIED" in error or "Not enough privileges" in error, \ + f"Expected ACCESS_DENIED error for user without ALTER, got: {error}" + + # Test 2: User with ALTER but without INSERT permission should fail + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}", + user="user_no_insert" + ) + assert "ACCESS_DENIED" in error or "Not enough privileges" in error, \ + f"Expected ACCESS_DENIED error for user without INSERT, got: {error}" + + # Test 3: User with both ALTER and INSERT should succeed + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}", + user="user_with_permissions" + ) + + # Wait for export to complete + time.sleep(5) + + # Verify the export succeeded + result = node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") + assert result.strip() == "3", f"Expected 3 rows exported, got: {result}" + + # Verify system table shows COMPLETED status + status = node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) + assert status.strip() == "COMPLETED", f"Expected COMPLETED status, got: {status}" + + +# assert multiple exports within a single query are executed. They all share the same query id +# and previously the transaction id was the query id, which would cause problems +def test_multiple_exports_within_a_single_query(cluster): + node = cluster.instances["replica1"] + + mt_table = "multiple_exports_within_a_single_query_mt_table" + s3_table = "multiple_exports_within_a_single_query_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}, EXPORT PARTITION ID '2021' TO TABLE {s3_table};") + + time.sleep(5) + + # assert the exports have been executed + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") == '3\n', "Export did not succeed" + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2021") == '1\n', "Export did not succeed" + + # check system.replicated_partition_exports for the exports + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) == "COMPLETED\n", "Export should be marked as COMPLETED" + + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2021' + """ + ) == "COMPLETED\n", "Export should be marked as COMPLETED" + +# def test_source_mutations_during_export_snapshot(cluster): +# node = cluster.instances["replica1"] + +# mt_table = "mutations_snapshot_mt_table" +# s3_table = "mutations_snapshot_s3_table" + +# create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + +# # Ensure export sees a consistent snapshot at start time even if we mutate the source later +# with PartitionManager() as pm: +# pm.add_network_delay(node, delay_ms=5000) + +# # Start export of 2020 +# node.query( +# f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table};" +# ) + +# # Mutate the source after export started (delete the same partition) +# node.query(f"ALTER TABLE {mt_table} DROP COLUMN id") + +# # assert the mutation has been applied AND the data has not been exported yet +# assert node.query(f"SELECT count() FROM {mt_table} WHERE year = 2020") == '0\n', "Mutation has not been applied" +# assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") == '0\n', "Data has been exported" + +# # Wait for export to finish and then verify destination still reflects the original snapshot (3 rows) +# time.sleep(5) +# assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") == '3\n', "Export did not preserve snapshot at start time after source mutation" diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 1bdf1cd250fd..a4b9d2dafd9d 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -44,6 +44,7 @@ ALTER MATERIALIZE TTL ['MATERIALIZE TTL'] TABLE ALTER TABLE ALTER SETTINGS ['ALTER SETTING','ALTER MODIFY SETTING','MODIFY SETTING','RESET SETTING'] TABLE ALTER TABLE ALTER MOVE PARTITION ['ALTER MOVE PART','MOVE PARTITION','MOVE PART'] TABLE ALTER TABLE ALTER EXPORT PART ['ALTER EXPORT PART','EXPORT PART'] TABLE ALTER TABLE +ALTER EXPORT PARTITION ['ALTER EXPORT PARTITION','EXPORT PARTITION'] TABLE ALTER TABLE ALTER FETCH PARTITION ['ALTER FETCH PART','FETCH PARTITION'] TABLE ALTER TABLE ALTER FREEZE PARTITION ['FREEZE PARTITION','UNFREEZE'] TABLE ALTER TABLE ALTER UNLOCK SNAPSHOT ['UNLOCK SNAPSHOT'] TABLE ALTER TABLE diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference index eea09bd06035..1b4fcf850453 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference @@ -18,6 +18,8 @@ columns columns creator_info creator_info +exports +exports failed_parts failed_parts flags diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference index 0d6c21be132f..9a250fa65580 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference @@ -8,6 +8,7 @@ blocks columns columns creator_info +exports failed_parts flags host @@ -49,6 +50,7 @@ blocks columns columns creator_info +exports failed_parts flags host diff --git a/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage.sh b/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage.sh index 515feb9676d4..ae7f05dac90a 100755 --- a/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage.sh +++ b/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage.sh @@ -49,6 +49,8 @@ echo "---- Export 2020_1_1_0 and 2021_2_2_0 to wildcard table" query "ALTER TABLE $mt_table EXPORT PART '2020_1_1_0' TO TABLE $s3_table_wildcard SETTINGS allow_experimental_export_merge_tree_part = 1" query "ALTER TABLE $mt_table EXPORT PART '2021_2_2_0' TO TABLE $s3_table_wildcard SETTINGS allow_experimental_export_merge_tree_part = 1" +sleep 3 + echo "---- Both data parts should appear" query "SELECT * FROM s3(s3_conn, filename='$s3_table_wildcard/**.parquet') ORDER BY id" @@ -60,13 +62,14 @@ query "SELECT * FROM s3(s3_conn, filename='$s3_table_wildcard/**.parquet') ORDER query "CREATE TABLE $mt_table_partition_expression_with_function (id UInt64, year UInt16) ENGINE = MergeTree() PARTITION BY toString(year) ORDER BY tuple()" query "CREATE TABLE $s3_table_wildcard_partition_expression_with_function (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$s3_table_wildcard_partition_expression_with_function/{_partition_id}/{_file}.parquet', format=Parquet, partition_strategy='wildcard') PARTITION BY toString(year)" -# insert query "INSERT INTO $mt_table_partition_expression_with_function VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)" echo "---- Export 2020_1_1_0 and 2021_2_2_0 to wildcard table with partition expression with function" query "ALTER TABLE $mt_table_partition_expression_with_function EXPORT PART 'cb217c742dc7d143b61583011996a160_1_1_0' TO TABLE $s3_table_wildcard_partition_expression_with_function SETTINGS allow_experimental_export_merge_tree_part = 1" query "ALTER TABLE $mt_table_partition_expression_with_function EXPORT PART '3be6d49ecf9749a383964bc6fab22d10_2_2_0' TO TABLE $s3_table_wildcard_partition_expression_with_function SETTINGS allow_experimental_export_merge_tree_part = 1" +sleep 1 + echo "---- Both data parts should appear" query "SELECT * FROM s3(s3_conn, filename='$s3_table_wildcard_partition_expression_with_function/**.parquet') ORDER BY id" diff --git a/tests/queries/0_stateless/03604_export_merge_tree_partition.reference b/tests/queries/0_stateless/03604_export_merge_tree_partition.reference new file mode 100644 index 000000000000..d48023362b99 --- /dev/null +++ b/tests/queries/0_stateless/03604_export_merge_tree_partition.reference @@ -0,0 +1,31 @@ +Select from source table +1 2020 +2 2020 +3 2020 +4 2021 +5 2021 +6 2022 +7 2022 +Select from destination table +1 2020 +2 2020 +3 2020 +4 2021 +5 2021 +Export partition 2022 +Select from destination table again +1 2020 +2 2020 +3 2020 +4 2021 +5 2021 +6 2022 +7 2022 +---- Data in roundtrip ReplicatedMergeTree table (should match s3_table) +1 2020 +2 2020 +3 2020 +4 2021 +5 2021 +6 2022 +7 2022 diff --git a/tests/queries/0_stateless/03604_export_merge_tree_partition.sh b/tests/queries/0_stateless/03604_export_merge_tree_partition.sh new file mode 100755 index 000000000000..87503112aadb --- /dev/null +++ b/tests/queries/0_stateless/03604_export_merge_tree_partition.sh @@ -0,0 +1,57 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, replica, no-parallel, no-replicated-database + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +rmt_table="rmt_table_${RANDOM}" +s3_table="s3_table_${RANDOM}" +rmt_table_roundtrip="rmt_table_roundtrip_${RANDOM}" + +query() { + $CLICKHOUSE_CLIENT --query "$1" +} + +query "DROP TABLE IF EXISTS $rmt_table, $s3_table, $rmt_table_roundtrip" + +query "CREATE TABLE $rmt_table (id UInt64, year UInt16) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/$rmt_table', 'replica1') PARTITION BY year ORDER BY tuple()" +query "CREATE TABLE $s3_table (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$s3_table', format=Parquet, partition_strategy='hive') PARTITION BY year" + +query "INSERT INTO $rmt_table VALUES (1, 2020), (2, 2020), (4, 2021)" + +query "INSERT INTO $rmt_table VALUES (3, 2020), (5, 2021)" + +query "INSERT INTO $rmt_table VALUES (6, 2022), (7, 2022)" + +# sync replicas +query "SYSTEM SYNC REPLICA $rmt_table" + +query "ALTER TABLE $rmt_table EXPORT PARTITION ID '2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +query "ALTER TABLE $rmt_table EXPORT PARTITION ID '2021' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +# todo poll some kind of status +sleep 15 + +echo "Select from source table" +query "SELECT * FROM $rmt_table ORDER BY id" + +echo "Select from destination table" +query "SELECT * FROM $s3_table ORDER BY id" + +echo "Export partition 2022" +query "ALTER TABLE $rmt_table EXPORT PARTITION ID '2022' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +# todo poll some kind of status +sleep 5 + +echo "Select from destination table again" +query "SELECT * FROM $s3_table ORDER BY id" + +query "CREATE TABLE $rmt_table_roundtrip ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/$rmt_table_roundtrip', 'replica1') PARTITION BY year ORDER BY tuple() AS SELECT * FROM $s3_table" + +echo "---- Data in roundtrip ReplicatedMergeTree table (should match s3_table)" +query "SELECT * FROM $rmt_table_roundtrip ORDER BY id" + +query "DROP TABLE IF EXISTS $rmt_table, $s3_table, $rmt_table_roundtrip" \ No newline at end of file