mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Merge pull request #72209 from cwurm/fix_min_age_to_force_merge_on_partition_only
Fix min_age_to_force_merge_on_partition_only
This commit is contained in:
commit
020d843058
@ -457,6 +457,7 @@ MergeTreeDataMergerMutator::MergeSelectingInfo MergeTreeDataMergerMutator::getPo
|
|||||||
|
|
||||||
auto & partition_info = partitions_info[partition_id];
|
auto & partition_info = partitions_info[partition_id];
|
||||||
partition_info.min_age = std::min(partition_info.min_age, part_info.age);
|
partition_info.min_age = std::min(partition_info.min_age, part_info.age);
|
||||||
|
++partition_info.num_parts;
|
||||||
|
|
||||||
++res.parts_selected_precondition;
|
++res.parts_selected_precondition;
|
||||||
|
|
||||||
@ -639,11 +640,21 @@ String MergeTreeDataMergerMutator::getBestPartitionToOptimizeEntire(
|
|||||||
auto best_partition_it = std::max_element(
|
auto best_partition_it = std::max_element(
|
||||||
partitions_info.begin(),
|
partitions_info.begin(),
|
||||||
partitions_info.end(),
|
partitions_info.end(),
|
||||||
[](const auto & e1, const auto & e2) { return e1.second.min_age < e2.second.min_age; });
|
[](const auto & e1, const auto & e2)
|
||||||
|
{
|
||||||
|
// If one partition has only a single part, always select the other partition.
|
||||||
|
if (e1.second.num_parts == 1)
|
||||||
|
return true;
|
||||||
|
if (e2.second.num_parts == 1)
|
||||||
|
return false;
|
||||||
|
// If both partitions have more than one part, select the older partition.
|
||||||
|
return e1.second.min_age < e2.second.min_age;
|
||||||
|
});
|
||||||
|
|
||||||
assert(best_partition_it != partitions_info.end());
|
assert(best_partition_it != partitions_info.end());
|
||||||
|
|
||||||
if (static_cast<size_t>(best_partition_it->second.min_age) < (*data_settings)[MergeTreeSetting::min_age_to_force_merge_seconds])
|
if ((static_cast<size_t>(best_partition_it->second.min_age) < (*data_settings)[MergeTreeSetting::min_age_to_force_merge_seconds])
|
||||||
|
|| static_cast<size_t>(best_partition_it->second.num_parts) == 1)
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
return best_partition_it->first;
|
return best_partition_it->first;
|
||||||
|
@ -65,6 +65,7 @@ public:
|
|||||||
struct PartitionInfo
|
struct PartitionInfo
|
||||||
{
|
{
|
||||||
time_t min_age{std::numeric_limits<time_t>::max()};
|
time_t min_age{std::numeric_limits<time_t>::max()};
|
||||||
|
size_t num_parts = 0;
|
||||||
};
|
};
|
||||||
using PartitionsInfo = std::unordered_map<std::string, PartitionInfo>;
|
using PartitionsInfo = std::unordered_map<std::string, PartitionInfo>;
|
||||||
|
|
||||||
|
@ -3,5 +3,6 @@ Without merge
|
|||||||
With merge any part range
|
With merge any part range
|
||||||
1
|
1
|
||||||
With merge partition only
|
With merge partition only
|
||||||
|
2
|
||||||
1
|
1
|
||||||
1
|
1
|
||||||
|
@ -52,13 +52,13 @@ DROP TABLE test_with_merge;
|
|||||||
|
|
||||||
SELECT 'With merge partition only';
|
SELECT 'With merge partition only';
|
||||||
|
|
||||||
CREATE TABLE test_with_merge (i Int64) ENGINE = MergeTree ORDER BY i
|
CREATE TABLE test_with_merge (i Int64) ENGINE = MergeTree ORDER BY i PARTITION BY i
|
||||||
SETTINGS min_age_to_force_merge_seconds=1, merge_selecting_sleep_ms=1000, min_age_to_force_merge_on_partition_only=true;
|
SETTINGS min_age_to_force_merge_seconds=1, merge_selecting_sleep_ms=1000, min_age_to_force_merge_on_partition_only=true;
|
||||||
INSERT INTO test_with_merge SELECT 1;
|
INSERT INTO test_with_merge SELECT 1;
|
||||||
INSERT INTO test_with_merge SELECT 2;
|
INSERT INTO test_with_merge SELECT 2;
|
||||||
INSERT INTO test_with_merge SELECT 3;"
|
INSERT INTO test_with_merge SELECT 2 SETTINGS insert_deduplicate = 0;"
|
||||||
|
|
||||||
wait_for_number_of_parts 'test_with_merge' 1 100
|
wait_for_number_of_parts 'test_with_merge' 2 100
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -mq "
|
$CLICKHOUSE_CLIENT -mq "
|
||||||
SELECT sleepEachRow(1) FROM numbers(9) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; -- Sleep for 9 seconds and verify that we keep the old part because it's the only one
|
SELECT sleepEachRow(1) FROM numbers(9) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; -- Sleep for 9 seconds and verify that we keep the old part because it's the only one
|
||||||
|
@ -3,5 +3,6 @@ Without merge
|
|||||||
With merge replicated any part range
|
With merge replicated any part range
|
||||||
1
|
1
|
||||||
With merge replicated partition only
|
With merge replicated partition only
|
||||||
|
2
|
||||||
1
|
1
|
||||||
1
|
1
|
||||||
|
@ -53,12 +53,14 @@ DROP TABLE test_replicated;
|
|||||||
SELECT 'With merge replicated partition only';
|
SELECT 'With merge replicated partition only';
|
||||||
|
|
||||||
CREATE TABLE test_replicated (i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test02676_partition_only', 'node') ORDER BY i
|
CREATE TABLE test_replicated (i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test02676_partition_only', 'node') ORDER BY i
|
||||||
|
PARTITION BY i
|
||||||
SETTINGS min_age_to_force_merge_seconds=1, merge_selecting_sleep_ms=1000, min_age_to_force_merge_on_partition_only=true;
|
SETTINGS min_age_to_force_merge_seconds=1, merge_selecting_sleep_ms=1000, min_age_to_force_merge_on_partition_only=true;
|
||||||
INSERT INTO test_replicated SELECT 1;
|
INSERT INTO test_replicated SELECT 1;
|
||||||
INSERT INTO test_replicated SELECT 2;
|
INSERT INTO test_replicated SELECT 2;
|
||||||
INSERT INTO test_replicated SELECT 3;"
|
SELECT sleep(3) FORMAT Null; -- Sleep so the first partition is older
|
||||||
|
INSERT INTO test_replicated SELECT 2 SETTINGS insert_deduplicate = 0;"
|
||||||
|
|
||||||
wait_for_number_of_parts 'test_replicated' 1 100
|
wait_for_number_of_parts 'test_replicated' 2 100
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -mq "
|
$CLICKHOUSE_CLIENT -mq "
|
||||||
SELECT sleepEachRow(1) FROM numbers(9) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; -- Sleep for 9 seconds and verify that we keep the old part because it's the only one
|
SELECT sleepEachRow(1) FROM numbers(9) SETTINGS function_sleep_max_microseconds_per_block = 10000000 FORMAT Null; -- Sleep for 9 seconds and verify that we keep the old part because it's the only one
|
||||||
|
Loading…
Reference in New Issue
Block a user