Make OPTIMIZE be able to merge all parts with FINAL.

This commit is contained in:
Amos Bird 2018-07-06 01:32:14 +08:00 committed by alexey-milovidov
parent d5f8e4a1a5
commit 78beacb3a5
4 changed files with 57 additions and 7 deletions

View File

@ -18,9 +18,6 @@ BlockIO InterpreterOptimizeQuery::execute()
{
const ASTOptimizeQuery & ast = typeid_cast<const ASTOptimizeQuery &>(*query_ptr);
if (ast.final && !ast.partition)
throw Exception("FINAL flag for OPTIMIZE query is meaningful only with specified PARTITION", ErrorCodes::BAD_ARGUMENTS);
StoragePtr table = context.getTable(ast.database, ast.table);
auto table_lock = table->lockStructure(true, __PRETTY_FUNCTION__);
table->optimize(query_ptr, ast.partition, ast.final, ast.deduplicate, context);

View File

@ -479,11 +479,40 @@ bool StorageMergeTree::optimize(
partition_id = data.getPartitionIDFromQuery(partition, context);
String disable_reason;
if (!merge(context.getSettingsRef().min_bytes_to_use_direct_io, true, partition_id, final, deduplicate, &disable_reason))
if (!partition && final)
{
if (context.getSettingsRef().optimize_throw_if_noop)
throw Exception(disable_reason.empty() ? "Can't OPTIMIZE by some reason" : disable_reason, ErrorCodes::CANNOT_ASSIGN_OPTIMIZE);
return false;
MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector();
std::unordered_set<String> partition_ids;
const String * prev_partition_id = nullptr;
for (const MergeTreeData::DataPartPtr & part : data_parts)
{
const String & partition_id = part->info.partition_id;
if (!prev_partition_id || partition_id != *prev_partition_id)
{
prev_partition_id = &partition_id;
partition_ids.emplace(partition_id);
}
}
for (const String & partition_id : partition_ids)
{
if (!merge(context.getSettingsRef().min_bytes_to_use_direct_io, true, partition_id, true, deduplicate, &disable_reason))
{
if (context.getSettingsRef().optimize_throw_if_noop)
throw Exception(disable_reason.empty() ? "Can't OPTIMIZE by some reason" : disable_reason, ErrorCodes::CANNOT_ASSIGN_OPTIMIZE);
return false;
}
}
}
else
{
if (!merge(context.getSettingsRef().min_bytes_to_use_direct_io, true, partition_id, final, deduplicate, &disable_reason))
{
if (context.getSettingsRef().optimize_throw_if_noop)
throw Exception(disable_reason.empty() ? "Can't OPTIMIZE by some reason" : disable_reason, ErrorCodes::CANNOT_ASSIGN_OPTIMIZE);
return false;
}
}
return true;

View File

@ -0,0 +1,7 @@
2000-01-01 1 first 1
2000-01-01 1 first 2
2000-01-01 2 first 2
2000-01-02 1 first 3
2000-01-01 1 first 3
2000-01-01 2 first 2
2000-01-02 1 first 3

View File

@ -0,0 +1,17 @@
DROP TABLE IF EXISTS test.partitioned_by_tuple;
CREATE TABLE test.partitioned_by_tuple (d Date, x UInt8, w String, y UInt8) ENGINE SummingMergeTree (y) PARTITION BY (d, x) ORDER BY (d, x, w);
INSERT INTO test.partitioned_by_tuple VALUES ('2000-01-02', 1, 'first', 3);
INSERT INTO test.partitioned_by_tuple VALUES ('2000-01-01', 2, 'first', 2);
INSERT INTO test.partitioned_by_tuple VALUES ('2000-01-01', 1, 'first', 1), ('2000-01-01', 1, 'first', 2);
OPTIMIZE TABLE test.partitioned_by_tuple;
SELECT * FROM test.partitioned_by_tuple ORDER BY d, x, w, y;
OPTIMIZE TABLE test.partitioned_by_tuple FINAL;
SELECT * FROM test.partitioned_by_tuple ORDER BY d, x, w, y;
DROP TABLE test.partitioned_by_tuple;