mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Remove redundant and duplicated code
This commit is contained in:
parent
1952be94ea
commit
4a53264a86
@ -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
|
||||
{
|
||||
|
@ -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.
|
||||
*/
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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();
|
||||
|
@ -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)
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
|
Loading…
Reference in New Issue
Block a user