diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 907026c73a3..530cc86f1dc 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -479,6 +479,11 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS } break; } + case ASTAlterCommand::EXPORT_PART: + { + required_access.emplace_back(AccessType::ALTER_MOVE_PARTITION, database, table); + break; + } case ASTAlterCommand::REPLACE_PARTITION: { required_access.emplace_back(AccessType::SELECT, command.from_database, command.from_table); diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index fe258b6df15..137e6a87871 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -343,6 +343,12 @@ void ASTAlterCommand::formatImpl(WriteBuffer & ostr, const FormatSettings & sett ostr << quoteString(move_destination_name); } } + else if (type == ASTAlterCommand::EXPORT_PART) + { + ostr << (settings.hilite ? hilite_keyword : "") << "EXPORT " << (part ? "PART " : "PARTITION ") + << (settings.hilite ? hilite_none : ""); + partition->formatImpl(ostr, settings, state, frame); + } else if (type == ASTAlterCommand::REPLACE_PARTITION) { ostr << (settings.hilite ? hilite_keyword : "") << (replace ? "REPLACE" : "ATTACH") << " PARTITION " diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index ab19ed9a55a..9e4280bae58 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -71,6 +71,7 @@ class ASTAlterCommand : public IAST FREEZE_ALL, UNFREEZE_PARTITION, UNFREEZE_ALL, + EXPORT_PART, DELETE, UPDATE, diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index ef52eab6cf7..f0705508f44 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -325,6 +325,7 @@ namespace DB MR_MACROS(MONTHS, "MONTHS") \ MR_MACROS(MOVE_PART, "MOVE PART") \ MR_MACROS(MOVE_PARTITION, "MOVE PARTITION") \ + MR_MACROS(EXPORT_PART, "EXPORT PART") \ MR_MACROS(MOVE, "MOVE") \ MR_MACROS(MS, "MS") \ MR_MACROS(MUTATION, "MUTATION") \ diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 0c8f05c90d2..a3988350bb3 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -82,6 +82,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_forget_partition(Keyword::FORGET_PARTITION); ParserKeyword s_move_partition(Keyword::MOVE_PARTITION); ParserKeyword s_move_part(Keyword::MOVE_PART); + ParserKeyword s_export_part(Keyword::EXPORT_PART); 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); @@ -554,6 +555,31 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->move_destination_name = ast_space_name->as().value.safeGet(); } } + else if (s_export_part.ignore(pos, expected)) + { + if (!parser_string_and_substituion.parse(pos, command_partition, expected)) + return false; + + command->type = ASTAlterCommand::EXPORT_PART; + command->part = true; + +// if (s_to_disk.ignore(pos, expected)) +// command->move_destination_type = DataDestinationType::DISK; +// else if (s_to_volume.ignore(pos, expected)) +// command->move_destination_type = DataDestinationType::VOLUME; +// else if (s_to_shard.ignore(pos, expected)) +// { +// command->move_destination_type = DataDestinationType::SHARD; +// } +// else +// return false; +// +// ASTPtr ast_space_name; +// if (!parser_string_literal.parse(pos, ast_space_name, expected)) +// return false; +// +// command->move_destination_name = ast_space_name->as().value.safeGet(); + } else if (s_add_constraint.ignore(pos, expected)) { if (s_if_not_exists.ignore(pos, expected)) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a8aaa8448c7..3da04e8d4cb 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -85,6 +85,7 @@ #include #include #include +#include #include #include @@ -5594,6 +5595,21 @@ Pipe MergeTreeData::alterPartition( } break; + case PartitionCommand::EXPORT_PART: + { + if (command.part) + { + auto part_name = command.partition->as().value.safeGet(); + auto data_part = getPartIfExists(part_name, {DataPartStates::value_type::Active}); + + if (!data_part) + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in committed state", part_name); + + exportMTPartToParquet(*this, data_part, query_context); + } + break; + } + case PartitionCommand::DROP_DETACHED_PARTITION: dropDetached(command.partition, command.part, query_context); break; diff --git a/src/Storages/MergeTree/exportMTPartToParquet.cpp b/src/Storages/MergeTree/exportMTPartToParquet.cpp new file mode 100644 index 00000000000..ac99f5a87f6 --- /dev/null +++ b/src/Storages/MergeTree/exportMTPartToParquet.cpp @@ -0,0 +1,84 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +void exportMTPartToParquet(const MergeTreeData & data, const MergeTreeData::DataPartPtr & data_part, ContextPtr context) +{ + auto metadata_snapshot = data.getInMemoryMetadataPtr(); + Names columns_to_read = metadata_snapshot->getColumns().getNamesOfPhysical(); + StorageSnapshotPtr storage_snapshot = data.getStorageSnapshot(metadata_snapshot, context); + + MergeTreeData::IMutationsSnapshot::Params params + { + .metadata_version = metadata_snapshot->getMetadataVersion(), + .min_part_metadata_version = data_part->getMetadataVersion(), + }; + + auto mutations_snapshot = data.getMutationsSnapshot(params); + + auto alter_conversions = MergeTreeData::getAlterConversionsForPart( + data_part, + mutations_snapshot, + metadata_snapshot, + context); + + QueryPlan plan; + + // todoa arthur + MergeTreeSequentialSourceType read_type = MergeTreeSequentialSourceType::Merge; + + bool apply_deleted_mask = true; + bool read_with_direct_io = false; + bool prefetch = false; + + createReadFromPartStep( + read_type, + plan, + data, + storage_snapshot, + data_part, + alter_conversions, + columns_to_read, + nullptr, + apply_deleted_mask, + std::nullopt, + read_with_direct_io, + prefetch, + context, + getLogger("abcde")); + + auto pipeline_settings = BuildQueryPipelineSettings::fromContext(context); + auto optimization_settings = QueryPlanOptimizationSettings::fromContext(context); + auto builder = plan.buildQueryPipeline(optimization_settings, pipeline_settings); + + QueryPipeline pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + auto header_block = pipeline.getHeader(); + + auto out_file_name = data_part->name + ".parquet"; + + auto out_file = std::make_shared(out_file_name); + auto parquet_output = FormatFactory::instance().getOutputFormat("Parquet", *out_file, header_block, context); + PullingPipelineExecutor executor(pipeline); + + Block block; + while (executor.pull(block)) + { + parquet_output->write(block); + } + + parquet_output->finalize(); + + out_file->finalize(); +} + +} diff --git a/src/Storages/MergeTree/exportMTPartToParquet.h b/src/Storages/MergeTree/exportMTPartToParquet.h new file mode 100644 index 00000000000..46beef55159 --- /dev/null +++ b/src/Storages/MergeTree/exportMTPartToParquet.h @@ -0,0 +1,10 @@ +#pragma once + +#include + +namespace DB +{ + +void exportMTPartToParquet(const MergeTreeData & data, const MergeTreeData::DataPartPtr & data_part, ContextPtr context); + +} diff --git a/src/Storages/PartitionCommands.cpp b/src/Storages/PartitionCommands.cpp index 0f7dadd75b6..a75211f4512 100644 --- a/src/Storages/PartitionCommands.cpp +++ b/src/Storages/PartitionCommands.cpp @@ -51,6 +51,14 @@ std::optional PartitionCommand::parse(const ASTAlterCommand * res.part = command_ast->part; return res; } + if (command_ast->type == ASTAlterCommand::EXPORT_PART) + { + PartitionCommand res; + res.type = EXPORT_PART; + res.partition = command_ast->partition->clone(); + res.part = command_ast->part; + return res; + } if (command_ast->type == ASTAlterCommand::MOVE_PARTITION) { PartitionCommand res; diff --git a/src/Storages/PartitionCommands.h b/src/Storages/PartitionCommands.h index 917e510f24b..15d2a7fb869 100644 --- a/src/Storages/PartitionCommands.h +++ b/src/Storages/PartitionCommands.h @@ -33,6 +33,7 @@ struct PartitionCommand UNFREEZE_ALL_PARTITIONS, UNFREEZE_PARTITION, REPLACE_PARTITION, + EXPORT_PART, }; Type type = UNKNOWN;