mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-13 18:02:24 +00:00
Check expired ttl
This commit is contained in:
parent
51807599e7
commit
25c39689c3
@ -359,6 +359,7 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition(
|
||||
const String & partition_id,
|
||||
bool final,
|
||||
bool * is_single_merged_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
String * out_disable_reason)
|
||||
{
|
||||
MergeTreeData::DataPartsVector parts = selectAllPartsFromPartition(partition_id);
|
||||
@ -374,8 +375,9 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition(
|
||||
}
|
||||
|
||||
/// If final, optimize_skip_merged_partitions is true and we have only one part in partition with level > 0
|
||||
/// than we don't select it to merge
|
||||
if (final && data.getSettings()->optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0)
|
||||
/// than we don't select it to merge. But if there are some expired TTL then merge is needed
|
||||
if (final && data.getSettings()->optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0 &&
|
||||
(!metadata_snapshot->hasAnyTTL() || parts[0]->checkAllTTLCalculated(metadata_snapshot)))
|
||||
{
|
||||
*is_single_merged_part = true;
|
||||
return false;
|
||||
@ -639,6 +641,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
|
||||
{
|
||||
static const String TMP_PREFIX = "tmp_merge_";
|
||||
|
||||
|
||||
|
||||
if (merges_blocker.isCancelled())
|
||||
throw Exception("Cancelled merging parts", ErrorCodes::ABORTED);
|
||||
|
||||
|
@ -99,6 +99,7 @@ public:
|
||||
const String & partition_id,
|
||||
bool final,
|
||||
bool * is_single_merged_part,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
String * out_disable_reason = nullptr);
|
||||
|
||||
/** Merge the parts.
|
||||
|
@ -686,7 +686,7 @@ bool StorageMergeTree::merge(
|
||||
{
|
||||
UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace();
|
||||
selected = merger_mutator.selectAllPartsToMergeWithinPartition(
|
||||
future_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, out_disable_reason);
|
||||
future_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, metadata_snapshot, out_disable_reason);
|
||||
|
||||
/// If final - we will wait for currently processing merges to finish and continue.
|
||||
/// TODO Respect query settings for timeout
|
||||
|
@ -3739,6 +3739,7 @@ bool StorageReplicatedMergeTree::optimize(
|
||||
};
|
||||
|
||||
const auto storage_settings_ptr = getSettings();
|
||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||
|
||||
if (!partition && final)
|
||||
{
|
||||
@ -3766,7 +3767,7 @@ bool StorageReplicatedMergeTree::optimize(
|
||||
/// and setting optimize_skip_merged_partitions is true
|
||||
bool is_single_merged_part = false;
|
||||
bool selected = merger_mutator.selectAllPartsToMergeWithinPartition(
|
||||
future_merged_part, disk_space, can_merge, partition_id, true, &is_single_merged_part, nullptr);
|
||||
future_merged_part, disk_space, can_merge, partition_id, true, &is_single_merged_part, metadata_snapshot, nullptr);
|
||||
|
||||
if (!selected)
|
||||
{
|
||||
@ -3821,7 +3822,7 @@ bool StorageReplicatedMergeTree::optimize(
|
||||
UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace();
|
||||
String partition_id = getPartitionIDFromQuery(partition, query_context);
|
||||
selected = merger_mutator.selectAllPartsToMergeWithinPartition(
|
||||
future_merged_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, &disable_reason);
|
||||
future_merged_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, metadata_snapshot, &disable_reason);
|
||||
}
|
||||
|
||||
if (!selected)
|
||||
|
Loading…
Reference in New Issue
Block a user