Rename PartDestinationType to DataDestionationType, because there is no part-specific information

This commit is contained in:
alesapin 2020-05-22 16:29:33 +03:00
parent f4b0d475af
commit f026fa7de1
12 changed files with 57 additions and 44 deletions

View File

@ -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);

View File

@ -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);
}

View File

@ -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

View File

@ -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.
}

View File

@ -1,7 +1,7 @@
#pragma once
#include <Parsers/IAST.h>
#include <Storages/MergeTree/PartDestinationType.h>
#include <Storages/DataDestinationType.h>
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_)
{

View File

@ -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))

View File

@ -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))

View File

@ -4,7 +4,7 @@
namespace DB
{
enum class PartDestinationType
enum class DataDestinationType
{
DISK,
VOLUME,

View File

@ -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;
}

View File

@ -8,7 +8,7 @@
#include <Storages/MergeTree/MergeTreeSettings.h>
#include <Storages/MergeTree/MergeTreeMutationStatus.h>
#include <Storages/MergeTree/MergeList.h>
#include <Storages/MergeTree/PartDestinationType.h>
#include <Storages/DataDestinationType.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/ReadBufferFromFile.h>
@ -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;

View File

@ -1,6 +1,6 @@
#include <Storages/PartitionCommands.h>
#include <Storages/IStorage.h>
#include <Storages/MergeTree/PartDestinationType.h>
#include <Storages/DataDestinationType.h>
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTIdentifier.h>
@ -42,13 +42,13 @@ std::optional<PartitionCommand> 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;

View File

@ -3,6 +3,7 @@
#include <Storages/ColumnsDescription.h>
#include <Storages/IndicesDescription.h>
#include <Storages/ConstraintsDescription.h>
#include <Storages/DataDestinationType.h>
#include <Parsers/IAST_fwd.h>
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;
};
}