Merge pull request #35987 from awakeljw/fork_chmaster

[Improvement] improvement in PARTITION ALL
This commit is contained in:
Kseniia Sumarokova 2022-04-07 17:36:35 +02:00 committed by GitHub
commit 5f9bc1c68e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 49 additions and 16 deletions

View File

@ -34,12 +34,17 @@ void ASTPartition::formatImpl(const FormatSettings & settings, FormatState & sta
value->formatImpl(settings, state, frame);
}
else
{
if (all)
settings.ostr << "ALL";
else
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ID " << (settings.hilite ? hilite_none : "");
WriteBufferFromOwnString id_buf;
writeQuoted(id, id_buf);
settings.ostr << id_buf.str();
}
}
}
}

View File

@ -32,10 +32,6 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
else if (s_all.ignore(pos, expected))
{
ASTPtr value = makeASTFunction("tuple");
partition->value = value;
partition->children.push_back(value);
partition->fields_count = 0;
partition->all = true;
}
else

View File

@ -3385,6 +3385,13 @@ void MergeTreeData::checkAlterPartitionIsPossible(
else
{
/// We are able to parse it
const auto * partition_ast = command.partition->as<ASTPartition>();
if (partition_ast && partition_ast->all)
{
if (command.type != PartitionCommand::DROP_PARTITION)
throw DB::Exception("Only support DETACH PARTITION ALL currently", ErrorCodes::SUPPORT_IS_DISABLED);
}
else
getPartitionIDFromQuery(command.partition, getContext());
}
}
@ -3393,14 +3400,15 @@ void MergeTreeData::checkAlterPartitionIsPossible(
void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition)
{
const String partition_id = getPartitionIDFromQuery(partition, getContext());
DataPartsVector parts_to_remove;
const auto * partition_ast = partition->as<ASTPartition>();
if (partition_ast && partition_ast->all)
parts_to_remove = getDataPartsVector();
else
{
const String partition_id = getPartitionIDFromQuery(partition, getContext());
parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id);
}
UInt64 partition_size = 0;
for (const auto & part : parts_to_remove)
@ -3828,6 +3836,9 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc
{
const auto & partition_ast = ast->as<ASTPartition &>();
if (partition_ast.all)
throw Exception("Only Support DETACH PARTITION ALL currently", ErrorCodes::SUPPORT_IS_DISABLED);
if (!partition_ast.value)
{
MergeTreePartInfo::validatePartitionID(partition_ast.id, format_version);
@ -3847,11 +3858,8 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc
}
/// Re-parse partition key fields using the information about expected field types.
auto metadata_snapshot = getInMemoryMetadataPtr();
const Block & key_sample_block = metadata_snapshot->getPartitionKey().sample_block;
if (partition_ast.all)
return "ALL";
size_t fields_count = key_sample_block.columns();
if (partition_ast.fields_count != fields_count)
throw Exception(ErrorCodes::INVALID_PARTITION_VALUE,

View File

@ -1347,13 +1347,14 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont
/// Asks to complete merges and does not allow them to start.
/// This protects against "revival" of data for a removed partition after completion of merge.
auto merge_blocker = stopMergesAndWait();
String partition_id = getPartitionIDFromQuery(partition, local_context);
const auto * partition_ast = partition->as<ASTPartition>();
if (partition_ast && partition_ast->all)
parts_to_remove = getDataPartsVector();
else
{
String partition_id = getPartitionIDFromQuery(partition, local_context);
parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id);
}
/// TODO should we throw an exception if parts_to_remove is empty?
removePartsFromWorkingSet(parts_to_remove, true);
}

View File

@ -66,6 +66,29 @@ select * from replicated_table_detach_all1 order by id;
SYSTEM SYNC REPLICA replicated_table_detach_all2;
select * from replicated_table_detach_all2 order by id;
ALTER TABLE replicated_table_detach_all1 FETCH PARTITION ALL FROM '/clickhouse/tables/test_00753_{database}/replicated_table_detach_all1'; -- { serverError 344 }
DROP TABLE replicated_table_detach_all1;
DROP TABLE replicated_table_detach_all2;
DROP TABLE IF EXISTS partition_all;
DROP TABLE IF EXISTS partition_all2;
CREATE TABLE partition_all (x UInt64, p UInt8, q UInt8) ENGINE = MergeTree ORDER BY tuple() PARTITION BY p;
INSERT INTO partition_all VALUES (4, 1, 2), (5, 1, 3), (3, 1, 4);
CREATE TABLE partition_all2 (x UInt64, p UInt8, q UInt8) ENGINE = MergeTree ORDER BY tuple() PARTITION BY p;
INSERT INTO partition_all2 VALUES (4, 1, 2), (5, 1, 3), (3, 1, 4);
-- test PARTITION ALL
ALTER TABLE partition_all2 REPLACE PARTITION ALL FROM partition_all; -- { serverError 344 }
ALTER TABLE partition_all MOVE PARTITION ALL TO TABLE partition_all2; -- { serverError 344 }
ALTER TABLE partition_all2 CLEAR INDEX p IN PARTITION ALL; -- { serverError 344 }
ALTER TABLE partition_all2 CLEAR COLUMN q IN PARTITION ALL; -- { serverError 344 }
ALTER TABLE partition_all2 UPDATE q = q + 1 IN PARTITION ALL where p = 1; -- { serverError 344 }
ALTER TABLE partition_all2 FREEZE PARTITION ALL; -- { serverError 344 }
CHECK TABLE partition_all2 PARTITION ALL; -- { serverError 344 }
OPTIMIZE TABLE partition_all2 PARTITION ALL; -- { serverError 344 }
DROP TABLE partition_all;
DROP TABLE partition_all2;