mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
support ALTER TABLE t DETACH PARTITION ALL syntax
This commit is contained in:
parent
5d8988ad4e
commit
9a4cc78dfb
@ -15,6 +15,7 @@ public:
|
||||
size_t fields_count = 0;
|
||||
|
||||
String id;
|
||||
bool all = false;
|
||||
|
||||
String getID(char) const override;
|
||||
ASTPtr clone() const override;
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Parsers/ASTPartition.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
namespace DB
|
||||
@ -13,6 +14,7 @@ namespace DB
|
||||
bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserKeyword s_id("ID");
|
||||
ParserKeyword s_all("ALL");
|
||||
ParserStringLiteral parser_string_literal;
|
||||
ParserExpression parser_expr;
|
||||
|
||||
@ -28,6 +30,14 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
|
||||
partition->id = partition_id->as<ASTLiteral &>().value.get<String>();
|
||||
}
|
||||
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
|
||||
{
|
||||
ASTPtr value;
|
||||
|
@ -3373,7 +3373,12 @@ void MergeTreeData::checkAlterPartitionIsPossible(
|
||||
void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition)
|
||||
{
|
||||
const String partition_id = getPartitionIDFromQuery(partition, getContext());
|
||||
auto parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id);
|
||||
DataPartsVector parts_to_remove;
|
||||
const auto * partition_ast = partition->as<ASTPartition>();
|
||||
if (partition_ast && partition_ast->all)
|
||||
parts_to_remove = getDataPartsVector();
|
||||
else
|
||||
parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id);
|
||||
|
||||
UInt64 partition_size = 0;
|
||||
|
||||
@ -3824,6 +3829,8 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc
|
||||
|
||||
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,
|
||||
|
@ -1348,7 +1348,11 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont
|
||||
/// 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);
|
||||
parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id);
|
||||
const auto * partition_ast = partition->as<ASTPartition>();
|
||||
if (partition_ast && partition_ast->all)
|
||||
parts_to_remove = getDataPartsVector();
|
||||
else
|
||||
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);
|
||||
|
@ -43,6 +43,7 @@
|
||||
#include <Parsers/ASTDropQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTOptimizeQuery.h>
|
||||
#include <Parsers/ASTPartition.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/ASTCheckQuery.h>
|
||||
@ -4941,15 +4942,37 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & partition, bool de
|
||||
throw Exception("DROP PARTITION cannot be done on this replica because it is not a leader", ErrorCodes::NOT_A_LEADER);
|
||||
|
||||
zkutil::ZooKeeperPtr zookeeper = getZooKeeperAndAssertNotReadonly();
|
||||
LogEntry entry;
|
||||
|
||||
String partition_id = getPartitionIDFromQuery(partition, query_context);
|
||||
bool did_drop = dropAllPartsInPartition(*zookeeper, partition_id, entry, query_context, detach);
|
||||
|
||||
if (did_drop)
|
||||
const auto * partition_ast = partition->as<ASTPartition>();
|
||||
if (partition_ast && partition_ast->all)
|
||||
{
|
||||
waitForLogEntryToBeProcessedIfNecessary(entry, query_context);
|
||||
cleanLastPartNode(partition_id);
|
||||
Strings partitions = zookeeper->getChildren(fs::path(zookeeper_path) / "block_numbers");
|
||||
|
||||
std::vector<std::pair<String, std::unique_ptr<LogEntry>>> entries_with_partitionid_to_drop;
|
||||
entries_with_partitionid_to_drop.reserve(partitions.size());
|
||||
for (String & partition_id : partitions)
|
||||
{
|
||||
auto entry = std::make_unique<LogEntry>();
|
||||
if (dropAllPartsInPartition(*zookeeper, partition_id, *entry, query_context, detach))
|
||||
entries_with_partitionid_to_drop.emplace_back(partition_id, std::move(entry));
|
||||
}
|
||||
|
||||
for (const auto & entry : entries_with_partitionid_to_drop)
|
||||
{
|
||||
waitForLogEntryToBeProcessedIfNecessary(*entry.second, query_context);
|
||||
cleanLastPartNode(entry.first);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
LogEntry entry;
|
||||
String partition_id = getPartitionIDFromQuery(partition, query_context);
|
||||
bool did_drop = dropAllPartsInPartition(*zookeeper, partition_id, entry, query_context, detach);
|
||||
if (did_drop)
|
||||
{
|
||||
waitForLogEntryToBeProcessedIfNecessary(entry, query_context);
|
||||
cleanLastPartNode(partition_id);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -10,3 +10,15 @@
|
||||
5 2
|
||||
6 3
|
||||
7 3
|
||||
4 2
|
||||
5 2
|
||||
1 1
|
||||
2 1
|
||||
3 1
|
||||
1 1
|
||||
2 1
|
||||
3 1
|
||||
1 1
|
||||
2 2
|
||||
1 1
|
||||
1 1
|
||||
|
@ -19,4 +19,53 @@ INSERT INTO alter_attach VALUES (6, 3), (7, 3);
|
||||
ALTER TABLE alter_attach ATTACH PARTITION 2;
|
||||
SELECT * FROM alter_attach ORDER BY x;
|
||||
|
||||
ALTER TABLE alter_attach DETACH PARTITION ALL;
|
||||
SELECT * FROM alter_attach ORDER BY x;
|
||||
|
||||
ALTER TABLE alter_attach ATTACH PARTITION 2;
|
||||
SELECT * FROM alter_attach ORDER BY x;
|
||||
|
||||
DROP TABLE IF EXISTS detach_all_no_partition;
|
||||
CREATE TABLE detach_all_no_partition (x UInt64, p UInt8) ENGINE = MergeTree ORDER BY tuple();
|
||||
INSERT INTO detach_all_no_partition VALUES (1, 1), (2, 1), (3, 1);
|
||||
SELECT * FROM detach_all_no_partition ORDER BY x;
|
||||
|
||||
ALTER TABLE detach_all_no_partition DETACH PARTITION ALL;
|
||||
SELECT * FROM detach_all_no_partition ORDER BY x;
|
||||
|
||||
ALTER TABLE detach_all_no_partition ATTACH PARTITION tuple();
|
||||
SELECT * FROM detach_all_no_partition ORDER BY x;
|
||||
|
||||
DROP TABLE alter_attach;
|
||||
DROP TABLE detach_all_no_partition;
|
||||
|
||||
DROP TABLE IF EXISTS replicated_table_detach_all1;
|
||||
DROP TABLE IF EXISTS replicated_table_detach_all2;
|
||||
|
||||
CREATE TABLE replicated_table_detach_all1 (
|
||||
id UInt64,
|
||||
Data String
|
||||
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00753_{database}/replicated_table_detach_all', '1') ORDER BY id PARTITION BY id;
|
||||
|
||||
CREATE TABLE replicated_table_detach_all2 (
|
||||
id UInt64,
|
||||
Data String
|
||||
) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00753_{database}/replicated_table_detach_all', '2') ORDER BY id PARTITION BY id;
|
||||
|
||||
|
||||
INSERT INTO replicated_table_detach_all1 VALUES (1, '1'), (2, '2');
|
||||
select * from replicated_table_detach_all1 order by id;
|
||||
|
||||
ALTER TABLE replicated_table_detach_all1 DETACH PARTITION ALL;
|
||||
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 ATTACH PARTITION tuple(1);
|
||||
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;
|
||||
|
||||
DROP TABLE replicated_table_detach_all1;
|
||||
DROP TABLE replicated_table_detach_all2;
|
||||
|
||||
|
@ -1,3 +1,4 @@
|
||||
1000
|
||||
0
|
||||
1000
|
||||
0
|
||||
|
@ -21,4 +21,8 @@ ALTER TABLE table_01 ATTACH PART '20191001_1_1_0';
|
||||
|
||||
SELECT COUNT() FROM table_01;
|
||||
|
||||
ALTER TABLE table_01 DETACH PARTITION ALL;
|
||||
|
||||
SELECT COUNT() FROM table_01;
|
||||
|
||||
DROP TABLE IF EXISTS table_01;
|
||||
|
Loading…
Reference in New Issue
Block a user