Remove redundant and duplicated code

This commit is contained in:
alesapin 2020-07-13 19:19:08 +03:00
parent 1952be94ea
commit 4a53264a86
9 changed files with 22 additions and 50 deletions

View File

@ -102,6 +102,10 @@ void IStorage::checkAlterIsPossible(const AlterCommands & commands, const Settin
}
}
void IStorage::checkAlterPartitionIsPossible(const PartitionCommands & /*commands*/, const Settings & /*settings*/) const
{
///TODO alesap
}
StorageID IStorage::getStorageID() const
{

View File

@ -360,6 +360,9 @@ public:
throw Exception("Partition operations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/// Checks that partition commands can be applied to storage.
virtual void checkAlterPartitionIsPossible(const PartitionCommands & commands, const Settings & settings) const;
/** Perform any background work. For example, combining parts in a MergeTree type table.
* Returns whether any work has been done.
*/

View File

@ -2525,6 +2525,19 @@ void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const StorageM
context);
}
void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition)
{
const String partition_id = getPartitionIDFromQuery(partition, global_context);
auto parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id);
UInt64 partition_size = 0;
for (const auto & part : parts_to_remove)
partition_size += part->getBytesOnDisk();
auto table_id = getStorageID();
global_context.checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, partition_size);
}
void MergeTreeData::movePartitionToDisk(const ASTPtr & partition, const String & name, bool moving_part, const Context & context)
{

View File

@ -547,6 +547,8 @@ public:
/// Moves partition to specified Volume
void movePartitionToVolume(const ASTPtr & partition, const String & name, bool moving_part, const Context & context);
void checkPartitionCanBeDropped(const ASTPtr & partition) override;
size_t getColumnCompressedSize(const std::string & name) const
{
auto lock = lockParts();

View File

@ -1068,20 +1068,6 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry(
}
}
/// TODO: it makes sense to check DROP_RANGE also
if (entry.type == LogEntry::CLEAR_COLUMN || entry.type == LogEntry::REPLACE_RANGE)
{
String conflicts_description;
String range_name = (entry.type == LogEntry::REPLACE_RANGE) ? entry.replace_range_entry->drop_range_part_name : entry.new_part_name;
auto range = MergeTreePartInfo::fromPartName(range_name, format_version);
if (0 != getConflictsCountForRange(range, entry, &conflicts_description, state_lock))
{
LOG_DEBUG(log, conflicts_description);
return false;
}
}
/// Alters must be executed one by one. First metadata change, and after that data alter (MUTATE_PART entries with).
/// corresponding alter_version.
if (entry.type == LogEntry::ALTER_METADATA)

View File

@ -209,22 +209,6 @@ void StorageMergeTree::checkTableCanBeDropped() const
global_context.checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes());
}
void StorageMergeTree::checkPartitionCanBeDropped(const ASTPtr & partition)
{
auto table_id = getStorageID();
const String partition_id = getPartitionIDFromQuery(partition, global_context);
auto parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id);
UInt64 partition_size = 0;
for (const auto & part : parts_to_remove)
{
partition_size += part->getBytesOnDisk();
}
global_context.checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, partition_size);
}
void StorageMergeTree::drop()
{
shutdown();

View File

@ -81,8 +81,6 @@ public:
void checkTableCanBeDropped() const override;
void checkPartitionCanBeDropped(const ASTPtr & partition) override;
ActionLock getActionLock(StorageActionBlockType action_type) override;
CheckResults checkData(const ASTPtr & query, const Context & context) override;

View File

@ -4054,22 +4054,6 @@ void StorageReplicatedMergeTree::checkTableCanBeDropped() const
global_context.checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes());
}
void StorageReplicatedMergeTree::checkPartitionCanBeDropped(const ASTPtr & partition)
{
const String partition_id = getPartitionIDFromQuery(partition, global_context);
auto parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id);
UInt64 partition_size = 0;
for (const auto & part : parts_to_remove)
partition_size += part->getBytesOnDisk();
auto table_id = getStorageID();
global_context.checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, partition_size);
}
void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
{
MergeTreeData::rename(new_path_to_table_data, new_table_id);

View File

@ -134,8 +134,6 @@ public:
void checkTableCanBeDropped() const override;
void checkPartitionCanBeDropped(const ASTPtr & partition) override;
ActionLock getActionLock(StorageActionBlockType action_type) override;
/// Wait when replication queue size becomes less or equal than queue_size