Apply TTL if it's not calculated for part

This commit is contained in:
alesapin 2020-09-03 11:59:41 +03:00
parent 821add088e
commit acc0ee0657
6 changed files with 90 additions and 3 deletions

View File

@ -1042,6 +1042,37 @@ void IMergeTreeDataPart::accumulateColumnSizes(ColumnToSize & column_to_size) co
column_to_size[column_name] = size.data_compressed;
}
bool IMergeTreeDataPart::checkAllTTLCalculated(const StorageMetadataPtr & metadata_snapshot) const
{
if (!metadata_snapshot->hasAnyTTL())
return false;
if (metadata_snapshot->hasRowsTTL())
{
if (isEmpty()) /// All rows were finally deleted and we don't store TTL
return true;
else if (ttl_infos.table_ttl.min == 0)
return false;
}
for (const auto & [column, desc] : metadata_snapshot->getColumnTTLs())
{
/// Part has this column, but we don't calculated TTL for it
if (!ttl_infos.columns_ttl.count(column) && getColumns().contains(column))
return false;
}
for (const auto & move_desc : metadata_snapshot->getMoveTTLs())
{
/// Move TTL is not calculated
if (!ttl_infos.moves_ttl.count(move_desc.result_column))
return false;
}
return true;
}
bool isCompactPart(const MergeTreeDataPartPtr & data_part)
{
return (data_part && data_part->getType() == MergeTreeDataPartType::COMPACT);

View File

@ -344,6 +344,11 @@ public:
static inline constexpr auto DELETE_ON_DESTROY_MARKER_FILE_NAME = "delete-on-destroy.txt";
/// Checks that all TTLs (table min/max, column ttls, so on) for part
/// calculated. Part without calculated TTL may exist if TTL was added after
/// part creation (using alter query with materialize_ttl setting).
bool checkAllTTLCalculated(const StorageMetadataPtr & metadata_snapshot) const;
protected:
/// Total size of all columns, calculated once in calcuateColumnSizesOnDisk

View File

@ -635,8 +635,17 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
new_data_part->is_temp = true;
bool need_remove_expired_values = false;
bool force_ttl = false;
for (const auto & part : parts)
{
new_data_part->ttl_infos.update(part->ttl_infos);
if (metadata_snapshot->hasAnyTTL() && !part->checkAllTTLCalculated(metadata_snapshot))
{
LOG_INFO(log, "Some TTL values were not calculated for part {}. Will calculate them forcefully during merge.", part->name);
need_remove_expired_values = true;
force_ttl = true;
}
}
const auto & part_min_ttl = new_data_part->ttl_infos.part_min_ttl;
if (part_min_ttl && part_min_ttl <= time_of_merge)
@ -809,7 +818,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
merged_stream = std::make_shared<DistinctSortedBlockInputStream>(merged_stream, sort_description, SizeLimits(), 0 /*limit_hint*/, Names());
if (need_remove_expired_values)
merged_stream = std::make_shared<TTLBlockInputStream>(merged_stream, data, metadata_snapshot, new_data_part, time_of_merge, false);
merged_stream = std::make_shared<TTLBlockInputStream>(merged_stream, data, metadata_snapshot, new_data_part, time_of_merge, force_ttl);
if (metadata_snapshot->hasSecondaryIndices())

View File

@ -58,7 +58,7 @@ struct MergeTreeDataPartTTLInfos
part_max_ttl = time_max;
}
bool empty()
bool empty() const
{
return !part_min_ttl && moves_ttl.empty();
}

View File

@ -0,0 +1,5 @@
3000
3000
2000
2000
1001

View File

@ -0,0 +1,37 @@
DROP TABLE IF EXISTS table_for_ttl;
CREATE TABLE table_for_ttl(
d DateTime,
key UInt64,
value String)
ENGINE = MergeTree()
ORDER BY tuple()
PARTITION BY key;
INSERT INTO table_for_ttl SELECT now() - INTERVAL 2 YEAR, 1, toString(number) from numbers(1000);
INSERT INTO table_for_ttl SELECT now() - INTERVAL 2 DAY, 3, toString(number) from numbers(2000, 1000);
INSERT INTO table_for_ttl SELECT now(), 4, toString(number) from numbers(3000, 1000);
SELECT count() FROM table_for_ttl;
ALTER TABLE table_for_ttl MODIFY TTL d + INTERVAL 1 YEAR SETTINGS materialize_ttl_after_modify = 0;
SELECT count() FROM table_for_ttl;
OPTIMIZE TABLE table_for_ttl FINAL;
SELECT count() FROM table_for_ttl;
ALTER TABLE table_for_ttl MODIFY COLUMN value String TTL d + INTERVAL 1 DAY SETTINGS materialize_ttl_after_modify = 0;
SELECT count(distinct value) FROM table_for_ttl;
OPTIMIZE TABLE table_for_ttl FINAL;
SELECT count(distinct value) FROM table_for_ttl;
OPTIMIZE TABLE table_for_ttl FINAL; -- Just check in logs, that it doesn't run with force again
DROP TABLE IF EXISTS table_for_ttl;