diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index c2ab8776c25..c3ee740a486 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -244,12 +244,12 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS } case ASTAlterCommand::MOVE_PARTITION: { - if ((command.move_destination_type == PartDestinationType::DISK) - || (command.move_destination_type == PartDestinationType::VOLUME)) + if ((command.move_destination_type == DataDestinationType::DISK) + || (command.move_destination_type == DataDestinationType::VOLUME)) { required_access.emplace_back(AccessType::ALTER_MOVE_PARTITION, database, table); } - else if (command.move_destination_type == PartDestinationType::TABLE) + else if (command.move_destination_type == DataDestinationType::TABLE) { required_access.emplace_back(AccessType::SELECT | AccessType::ALTER_DELETE, database, table); required_access.emplace_back(AccessType::INSERT, command.to_database, command.to_table); diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 9ec2fad5768..f323f66be17 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -181,13 +181,13 @@ void ASTAlterCommand::formatImpl( settings.ostr << " TO "; switch (move_destination_type) { - case PartDestinationType::DISK: + case DataDestinationType::DISK: settings.ostr << "DISK "; break; - case PartDestinationType::VOLUME: + case DataDestinationType::VOLUME: settings.ostr << "VOLUME "; break; - case PartDestinationType::TABLE: + case DataDestinationType::TABLE: settings.ostr << "TABLE "; if (!to_database.empty()) { @@ -201,7 +201,7 @@ void ASTAlterCommand::formatImpl( default: break; } - if (move_destination_type != PartDestinationType::TABLE) + if (move_destination_type != DataDestinationType::TABLE) { settings.ostr << quoteString(move_destination_name); } diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 85e9a4d7552..a9ae06863a9 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -136,7 +136,7 @@ public: bool if_exists = false; /// option for DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN - PartDestinationType move_destination_type; /// option for MOVE PART/PARTITION + DataDestinationType move_destination_type; /// option for MOVE PART/PARTITION String move_destination_name; /// option for MOVE PART/PARTITION diff --git a/src/Parsers/ASTTTLElement.cpp b/src/Parsers/ASTTTLElement.cpp index 7e03a73e36d..acc3546ddfb 100644 --- a/src/Parsers/ASTTTLElement.cpp +++ b/src/Parsers/ASTTTLElement.cpp @@ -10,15 +10,15 @@ namespace DB void ASTTTLElement::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { children.front()->formatImpl(settings, state, frame); - if (destination_type == PartDestinationType::DISK) + if (destination_type == DataDestinationType::DISK) { settings.ostr << " TO DISK " << quoteString(destination_name); } - else if (destination_type == PartDestinationType::VOLUME) + else if (destination_type == DataDestinationType::VOLUME) { settings.ostr << " TO VOLUME " << quoteString(destination_name); } - else if (destination_type == PartDestinationType::DELETE) + else if (destination_type == DataDestinationType::DELETE) { /// It would be better to output "DELETE" here but that will break compatibility with earlier versions. } diff --git a/src/Parsers/ASTTTLElement.h b/src/Parsers/ASTTTLElement.h index 02f70094e04..61f4a6e958f 100644 --- a/src/Parsers/ASTTTLElement.h +++ b/src/Parsers/ASTTTLElement.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -11,10 +11,10 @@ namespace DB class ASTTTLElement : public IAST { public: - PartDestinationType destination_type; + DataDestinationType destination_type; String destination_name; - ASTTTLElement(PartDestinationType destination_type_, const String & destination_name_) + ASTTTLElement(DataDestinationType destination_type_, const String & destination_name_) : destination_type(destination_type_) , destination_name(destination_name_) { diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 70a8b282a72..c206db24626 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1462,16 +1462,16 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!parser_exp.parse(pos, expr_elem, expected)) return false; - PartDestinationType destination_type = PartDestinationType::DELETE; + DataDestinationType destination_type = DataDestinationType::DELETE; String destination_name; if (s_to_disk.ignore(pos)) - destination_type = PartDestinationType::DISK; + destination_type = DataDestinationType::DISK; else if (s_to_volume.ignore(pos)) - destination_type = PartDestinationType::VOLUME; + destination_type = DataDestinationType::VOLUME; else s_delete.ignore(pos); - if (destination_type == PartDestinationType::DISK || destination_type == PartDestinationType::VOLUME) + if (destination_type == DataDestinationType::DISK || destination_type == DataDestinationType::VOLUME) { ASTPtr ast_space_name; if (!parser_string_literal.parse(pos, ast_space_name, expected)) diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 623bca440bb..f90d010e9de 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -260,19 +260,19 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->part = true; if (s_to_disk.ignore(pos)) - command->move_destination_type = PartDestinationType::DISK; + command->move_destination_type = DataDestinationType::DISK; else if (s_to_volume.ignore(pos)) - command->move_destination_type = PartDestinationType::VOLUME; + command->move_destination_type = DataDestinationType::VOLUME; else if (s_to_table.ignore(pos)) { if (!parseDatabaseAndTableName(pos, expected, command->to_database, command->to_table)) return false; - command->move_destination_type = PartDestinationType::TABLE; + command->move_destination_type = DataDestinationType::TABLE; } else return false; - if (command->move_destination_type != PartDestinationType::TABLE) + if (command->move_destination_type != DataDestinationType::TABLE) { ASTPtr ast_space_name; if (!parser_string_literal.parse(pos, ast_space_name, expected)) @@ -289,19 +289,19 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->type = ASTAlterCommand::MOVE_PARTITION; if (s_to_disk.ignore(pos)) - command->move_destination_type = PartDestinationType::DISK; + command->move_destination_type = DataDestinationType::DISK; else if (s_to_volume.ignore(pos)) - command->move_destination_type = PartDestinationType::VOLUME; + command->move_destination_type = DataDestinationType::VOLUME; else if (s_to_table.ignore(pos)) { if (!parseDatabaseAndTableName(pos, expected, command->to_database, command->to_table)) return false; - command->move_destination_type = PartDestinationType::TABLE; + command->move_destination_type = DataDestinationType::TABLE; } else return false; - if (command->move_destination_type != PartDestinationType::TABLE) + if (command->move_destination_type != DataDestinationType::TABLE) { ASTPtr ast_space_name; if (!parser_string_literal.parse(pos, ast_space_name, expected)) diff --git a/src/Storages/MergeTree/PartDestinationType.h b/src/Storages/DataDestinationType.h similarity index 73% rename from src/Storages/MergeTree/PartDestinationType.h rename to src/Storages/DataDestinationType.h index 8f3e44537a6..05d1d89c2b5 100644 --- a/src/Storages/MergeTree/PartDestinationType.h +++ b/src/Storages/DataDestinationType.h @@ -4,7 +4,7 @@ namespace DB { -enum class PartDestinationType +enum class DataDestinationType { DISK, VOLUME, diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 728bfa18bd8..ef8e3789cb1 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -621,7 +621,7 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, auto syntax_result = SyntaxAnalyzer(global_context).analyze(ttl_ast, new_columns.getAllPhysical()); result.expression = ExpressionAnalyzer(ttl_ast, syntax_result, global_context).getActions(false); - result.destination_type = PartDestinationType::DELETE; + result.destination_type = DataDestinationType::DELETE; result.result_column = ttl_ast->getColumnName(); checkTTLExpression(result.expression, result.result_column); @@ -665,7 +665,7 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, if (!ttl_element) throw Exception("Unexpected AST element in TTL expression", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - if (ttl_element->destination_type == PartDestinationType::DELETE) + if (ttl_element->destination_type == DataDestinationType::DELETE) { if (seen_delete_ttl) { @@ -688,7 +688,7 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, if (!new_ttl_entry.getDestination(getStoragePolicy())) { String message; - if (new_ttl_entry.destination_type == PartDestinationType::DISK) + if (new_ttl_entry.destination_type == DataDestinationType::DISK) message = "No such disk " + backQuote(new_ttl_entry.destination_name) + " for given storage policy."; else message = "No such volume " + backQuote(new_ttl_entry.destination_name) + " for given storage policy."; @@ -2921,11 +2921,11 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_ SpacePtr destination_ptr = ttl_entry->getDestination(getStoragePolicy()); if (!destination_ptr) { - if (ttl_entry->destination_type == PartDestinationType::VOLUME) + if (ttl_entry->destination_type == DataDestinationType::VOLUME) LOG_WARNING(log, "Would like to reserve space on volume '" << ttl_entry->destination_name << "' by TTL rule of table '" << log_name << "' but volume was not found"); - else if (ttl_entry->destination_type == PartDestinationType::DISK) + else if (ttl_entry->destination_type == DataDestinationType::DISK) LOG_WARNING(log, "Would like to reserve space on disk '" << ttl_entry->destination_name << "' by TTL rule of table '" << log_name << "' but disk was not found"); @@ -2936,11 +2936,11 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_ if (reservation) return reservation; else - if (ttl_entry->destination_type == PartDestinationType::VOLUME) + if (ttl_entry->destination_type == DataDestinationType::VOLUME) LOG_WARNING(log, "Would like to reserve space on volume '" << ttl_entry->destination_name << "' by TTL rule of table '" << log_name << "' but there is not enough space"); - else if (ttl_entry->destination_type == PartDestinationType::DISK) + else if (ttl_entry->destination_type == DataDestinationType::DISK) LOG_WARNING(log, "Would like to reserve space on disk '" << ttl_entry->destination_name << "' by TTL rule of table '" << log_name << "' but there is not enough space"); @@ -2954,9 +2954,9 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_ SpacePtr MergeTreeData::TTLEntry::getDestination(StoragePolicyPtr policy) const { - if (destination_type == PartDestinationType::VOLUME) + if (destination_type == DataDestinationType::VOLUME) return policy->getVolumeByName(destination_name); - else if (destination_type == PartDestinationType::DISK) + else if (destination_type == DataDestinationType::DISK) return policy->getDiskByName(destination_name); else return {}; @@ -2964,13 +2964,13 @@ SpacePtr MergeTreeData::TTLEntry::getDestination(StoragePolicyPtr policy) const bool MergeTreeData::TTLEntry::isPartInDestination(StoragePolicyPtr policy, const IMergeTreeDataPart & part) const { - if (destination_type == PartDestinationType::VOLUME) + if (destination_type == DataDestinationType::VOLUME) { for (const auto & disk : policy->getVolumeByName(destination_name)->getDisks()) if (disk->getName() == part.volume->getDisk()->getName()) return true; } - else if (destination_type == PartDestinationType::DISK) + else if (destination_type == DataDestinationType::DISK) return policy->getDiskByName(destination_name)->getName() == part.volume->getDisk()->getName(); return false; } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 70d3a1c3514..2b5b75110f1 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include @@ -652,7 +652,7 @@ public: String result_column; /// Name and type of a destination are only valid in table-level context. - PartDestinationType destination_type; + DataDestinationType destination_type; String destination_name; ASTPtr entry_ast; diff --git a/src/Storages/PartitionCommands.cpp b/src/Storages/PartitionCommands.cpp index 9c8fc2cb598..c3bf00187af 100644 --- a/src/Storages/PartitionCommands.cpp +++ b/src/Storages/PartitionCommands.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include @@ -42,13 +42,13 @@ std::optional PartitionCommand::parse(const ASTAlterCommand * res.part = command_ast->part; switch (command_ast->move_destination_type) { - case PartDestinationType::DISK: + case DataDestinationType::DISK: res.move_destination_type = PartitionCommand::MoveDestinationType::DISK; break; - case PartDestinationType::VOLUME: + case DataDestinationType::VOLUME: res.move_destination_type = PartitionCommand::MoveDestinationType::VOLUME; break; - case PartDestinationType::TABLE: + case DataDestinationType::TABLE: res.move_destination_type = PartitionCommand::MoveDestinationType::TABLE; res.to_database = command_ast->to_database; res.to_table = command_ast->to_table; diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 37dce5a78b9..3faaed8d680 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace DB @@ -72,5 +73,17 @@ struct StorageMetadataKeyField static StorageMetadataKeyField getKeyFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context); }; +struct StorageMetadataTTLField +{ + ASTPtr definition_ast; + + ExpressionActionsPtr expression; + + String result_column; + + DataDestinationType destination_type; + + String destination_name; +}; }