mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Rename PartDestinationType to DataDestionationType, because there is no part-specific information
This commit is contained in:
parent
f4b0d475af
commit
f026fa7de1
@ -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);
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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
|
||||
|
||||
|
@ -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.
|
||||
}
|
||||
|
@ -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_)
|
||||
{
|
||||
|
@ -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))
|
||||
|
@ -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))
|
||||
|
@ -4,7 +4,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
enum class PartDestinationType
|
||||
enum class DataDestinationType
|
||||
{
|
||||
DISK,
|
||||
VOLUME,
|
@ -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;
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user