From 1d31ac2a0cc35add672a6610331ab903cda06142 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 2 Feb 2024 10:03:31 +0000 Subject: [PATCH 1/8] add `FORGET PARTITION` query to remove old parition nodes from ZooKeeper --- src/Common/ErrorCodes.cpp | 1 + src/Interpreters/InterpreterAlterQuery.cpp | 1 + src/Parsers/ASTAlterQuery.cpp | 6 ++++++ src/Parsers/ASTAlterQuery.h | 3 ++- src/Parsers/ParserAlterQuery.cpp | 8 ++++++++ src/Storages/MergeTree/MergeTreeData.cpp | 22 ++++++++++++++++++++- src/Storages/MergeTree/MergeTreeData.h | 4 +++- src/Storages/PartitionCommands.cpp | 9 +++++++++ src/Storages/PartitionCommands.h | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 14 +++++++++++++ src/Storages/StorageReplicatedMergeTree.h | 2 ++ 11 files changed, 68 insertions(+), 3 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 2e154ddb32d..fa06fef0615 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -593,6 +593,7 @@ M(711, FILECACHE_ACCESS_DENIED) \ M(712, TOO_MANY_MATERIALIZED_VIEWS) \ M(713, BROKEN_PROJECTION) \ + M(714, CANNOT_FORGET_PARTITION) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index bfcb0d6dd39..f1a389d781b 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -412,6 +412,7 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS case ASTAlterCommand::APPLY_DELETED_MASK: case ASTAlterCommand::DROP_PARTITION: case ASTAlterCommand::DROP_DETACHED_PARTITION: + case ASTAlterCommand::FORGET_PARTITION: { required_access.emplace_back(AccessType::ALTER_DELETE, database, table); break; diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index e229095df1b..894f05fce63 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -274,6 +274,12 @@ void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); } + else if (type == ASTAlterCommand::FORGET_PARTITION) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << "FORGET PARTITION " + << (settings.hilite ? hilite_none : ""); + partition->formatImpl(settings, state, frame); + } else if (type == ASTAlterCommand::ATTACH_PARTITION) { settings.ostr << (settings.hilite ? hilite_keyword : "") << "ATTACH " << (part ? "PART " : "PARTITION ") diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index c17f260b660..35c7b3a7766 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -63,6 +63,7 @@ public: DROP_PARTITION, DROP_DETACHED_PARTITION, + FORGET_PARTITION, ATTACH_PARTITION, MOVE_PARTITION, REPLACE_PARTITION, @@ -139,7 +140,7 @@ public: IAST * statistic_decl = nullptr; - /** Used in DROP PARTITION, ATTACH PARTITION FROM, UPDATE, DELETE queries. + /** Used in DROP PARTITION, ATTACH PARTITION FROM, FORGET PARTITION, UPDATE, DELETE queries. * The value or ID of the partition is stored here. */ IAST * partition = nullptr; diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index d72fb493368..7ab70727efd 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -74,6 +74,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_detach_part("DETACH PART"); ParserKeyword s_drop_partition("DROP PARTITION"); ParserKeyword s_drop_part("DROP PART"); + ParserKeyword s_forget_partition("FORGET PARTITION"); ParserKeyword s_move_partition("MOVE PARTITION"); ParserKeyword s_move_part("MOVE PART"); ParserKeyword s_drop_detached_partition("DROP DETACHED PARTITION"); @@ -254,6 +255,13 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->type = ASTAlterCommand::DROP_PARTITION; command->part = true; } + else if (s_forget_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command_partition, expected)) + return false; + + command->type = ASTAlterCommand::FORGET_PARTITION; + } else if (s_drop_detached_partition.ignore(pos, expected)) { if (!parser_partition.parse(pos, command_partition, expected)) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a9f30e6e522..8499f46334c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -197,6 +197,7 @@ namespace ErrorCodes extern const int TOO_MANY_MUTATIONS; extern const int CANNOT_SCHEDULE_TASK; extern const int LIMIT_EXCEEDED; + extern const int CANNOT_FORGET_PARTITION; } static size_t getPartitionAstFieldsCount(const ASTPartition & partition_ast, ASTPtr partition_value_ast) @@ -4945,8 +4946,18 @@ void MergeTreeData::checkAlterPartitionIsPossible( // call to `getPartitionIDFromQuery` using source storage. // Note: `PartitionCommand::REPLACE_PARTITION` is used both for `REPLACE PARTITION` and `ATTACH PARTITION FROM` queries. // But not for `ATTACH PARTITION` queries. + + String partition_id; if (command.type != PartitionCommand::REPLACE_PARTITION) - getPartitionIDFromQuery(command.partition, getContext()); + partition_id = getPartitionIDFromQuery(command.partition, getContext()); + + if (command.type == PartitionCommand::FORGET_PARTITION) + { + DataPartsLock lock = lockParts(); + auto parts_in_partition = getDataPartsPartitionRange(partition_id); + if (!parts_in_partition.empty()) + throw Exception(ErrorCodes::CANNOT_FORGET_PARTITION, "Partition {} is not empty", partition_id); + } } } } @@ -5168,6 +5179,11 @@ void MergeTreeData::fetchPartition( throw Exception(ErrorCodes::NOT_IMPLEMENTED, "FETCH PARTITION is not supported by storage {}", getName()); } +void MergeTreeData::forgetPartition(const ASTPtr & /*partition*/, ContextPtr /*query_context*/) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "FORGET PARTITION is not supported by storage {}", getName()); +} + Pipe MergeTreeData::alterPartition( const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, @@ -5204,6 +5220,10 @@ Pipe MergeTreeData::alterPartition( dropDetached(command.partition, command.part, query_context); break; + case PartitionCommand::FORGET_PARTITION: + forgetPartition(command.partition, query_context); + break; + case PartitionCommand::ATTACH_PARTITION: current_command_results = attachPartition(command.partition, metadata_snapshot, command.part, query_context); break; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index dfdc22baa8f..ef442bb3463 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -832,7 +832,7 @@ public: return secondary_index_sizes; } - /// For ATTACH/DETACH/DROP PARTITION. + /// For ATTACH/DETACH/DROP/FORGET PARTITION. String getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr context, DataPartsLock * acquired_lock = nullptr) const; std::unordered_set getPartitionIDsFromQuery(const ASTs & asts, ContextPtr context) const; std::set getPartitionIdsAffectedByCommands(const MutationCommands & commands, ContextPtr query_context) const; @@ -1357,6 +1357,8 @@ protected: bool fetch_part, ContextPtr query_context); + virtual void forgetPartition(const ASTPtr & partition, ContextPtr context); + virtual void movePartitionToShard(const ASTPtr & partition, bool move_part, const String & to, ContextPtr query_context); void writePartLog( diff --git a/src/Storages/PartitionCommands.cpp b/src/Storages/PartitionCommands.cpp index 6ce66d85ddc..fd5964c7034 100644 --- a/src/Storages/PartitionCommands.cpp +++ b/src/Storages/PartitionCommands.cpp @@ -36,6 +36,13 @@ std::optional PartitionCommand::parse(const ASTAlterCommand * res.part = command_ast->part; return res; } + else if (command_ast->type == ASTAlterCommand::FORGET_PARTITION) + { + PartitionCommand res; + res.type = FORGET_PARTITION; + res.partition = command_ast->partition->clone(); + return res; + } else if (command_ast->type == ASTAlterCommand::ATTACH_PARTITION) { PartitionCommand res; @@ -147,6 +154,8 @@ std::string PartitionCommand::typeToString() const return "DROP DETACHED PART"; else return "DROP DETACHED PARTITION"; + case PartitionCommand::Type::FORGET_PARTITION: + return "FORGET PARTITION"; case PartitionCommand::Type::FETCH_PARTITION: if (part) return "FETCH PART"; diff --git a/src/Storages/PartitionCommands.h b/src/Storages/PartitionCommands.h index b8b2ec47e71..f0ecf91f567 100644 --- a/src/Storages/PartitionCommands.h +++ b/src/Storages/PartitionCommands.h @@ -26,6 +26,7 @@ struct PartitionCommand MOVE_PARTITION, DROP_PARTITION, DROP_DETACHED_PARTITION, + FORGET_PARTITION, FETCH_PARTITION, FREEZE_ALL_PARTITIONS, FREEZE_PARTITION, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1224af4d6cb..f7d84a33f82 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7312,6 +7312,20 @@ void StorageReplicatedMergeTree::fetchPartition( } +void StorageReplicatedMergeTree::forgetPartition(const ASTPtr & partition, ContextPtr query_context) +{ + assertNotReadonly(); + zkutil::ZooKeeperPtr zookeeper = getZooKeeperAndAssertNotReadonly(); + + String partition_id = getPartitionIDFromQuery(partition, query_context); + LOG_INFO(log, "Forget partition {}", partition_id); + + String block_numbers_path = fs::path(zookeeper_path) / "block_numbers"; + String partition_path = fs::path(block_numbers_path) / partition_id; + zookeeper->remove(partition_path); +} + + void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, ContextPtr query_context) { /// Overview of the mutation algorithm. diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index c682b1ec88d..331538c0f8f 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -912,6 +912,8 @@ private: const String & from, bool fetch_part, ContextPtr query_context) override; + void forgetPartition(const ASTPtr & partition, ContextPtr query_context) override; + /// NOTE: there are no guarantees for concurrent merges. Dropping part can /// be concurrently merged into some covering part and dropPart will do From 472145091764047892b0048ec2b5ac2592e91b94 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 20 Feb 2024 18:55:45 +0000 Subject: [PATCH 2/8] add test --- .../02995_forget_partition.reference | 21 ++++++++++++ .../0_stateless/02995_forget_partition.sql | 33 +++++++++++++++++++ 2 files changed, 54 insertions(+) create mode 100644 tests/queries/0_stateless/02995_forget_partition.reference create mode 100644 tests/queries/0_stateless/02995_forget_partition.sql diff --git a/tests/queries/0_stateless/02995_forget_partition.reference b/tests/queries/0_stateless/02995_forget_partition.reference new file mode 100644 index 00000000000..b2bc15b7bcb --- /dev/null +++ b/tests/queries/0_stateless/02995_forget_partition.reference @@ -0,0 +1,21 @@ +---before--- +20240101 +20240102 +20240103 +20240104 +20240105 +20240106 +20240107 +20240108 +20240109 +20240110 +---after--- +20240102 +20240103 +20240104 +20240105 +20240106 +20240107 +20240108 +20240109 +20240110 diff --git a/tests/queries/0_stateless/02995_forget_partition.sql b/tests/queries/0_stateless/02995_forget_partition.sql new file mode 100644 index 00000000000..673052dff00 --- /dev/null +++ b/tests/queries/0_stateless/02995_forget_partition.sql @@ -0,0 +1,33 @@ +-- Tags: zookeeper + +drop table if exists forget_partition; + +create table forget_partition +( + k UInt64, + d Date, + v String +) +engine = ReplicatedMergeTree('/test/02995/{database}/rmt', '1') +order by (k, d) +partition by toYYYYMMDD(d); + +insert into forget_partition select number, '2024-01-01' + interval number day, base64Encode(randomString(20)) from system.numbers limit 10; + +alter table forget_partition drop partition '20240101'; +alter table forget_partition drop partition '20240102'; + +set allow_unrestricted_reads_from_keeper=1; + +select '---before---'; +select name from system.zookeeper where path = '/test/02995/' || currentDatabase() || '/rmt/block_numbers' order by name; + +alter table forget_partition forget partition '20240103'; -- {serverError CANNOT_FORGET_PARTITION} +alter table forget_partition forget partition '20240203'; -- {serverError KEEPER_EXCEPTION} +alter table forget_partition forget partition '20240101'; + + +select '---after---'; +select name from system.zookeeper where path = '/test/02995/' || currentDatabase() || '/rmt/block_numbers' order by name; + +drop table forget_partition; \ No newline at end of file From af0a7a6e000a90efc2f54a54a26a9388c226610a Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 20 Feb 2024 18:56:39 +0000 Subject: [PATCH 3/8] style --- tests/queries/0_stateless/02995_forget_partition.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02995_forget_partition.sql b/tests/queries/0_stateless/02995_forget_partition.sql index 673052dff00..e769b9c8220 100644 --- a/tests/queries/0_stateless/02995_forget_partition.sql +++ b/tests/queries/0_stateless/02995_forget_partition.sql @@ -30,4 +30,4 @@ alter table forget_partition forget partition '20240101'; select '---after---'; select name from system.zookeeper where path = '/test/02995/' || currentDatabase() || '/rmt/block_numbers' order by name; -drop table forget_partition; \ No newline at end of file +drop table forget_partition; From 27371ae4c433d6af69da42ac557fdcb507df4685 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 20 Feb 2024 19:16:47 +0000 Subject: [PATCH 4/8] add docs --- .../sql-reference/statements/alter/partition.md | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 114b8d5ffe3..8227e3a2584 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -9,6 +9,7 @@ The following operations with [partitions](/docs/en/engines/table-engines/merget - [DETACH PARTITION\|PART](#detach-partitionpart) — Moves a partition or part to the `detached` directory and forget it. - [DROP PARTITION\|PART](#drop-partitionpart) — Deletes a partition or part. +- [FORGET PARTITION](#forget-partition) — Deletes a partition completely from ZooKeeper. - [ATTACH PARTITION\|PART](#attach-partitionpart) — Adds a partition or part from the `detached` directory to the table. - [ATTACH PARTITION FROM](#attach-partition-from) — Copies the data partition from one table to another and adds. - [REPLACE PARTITION](#replace-partition) — Copies the data partition from one table to another and replaces. @@ -73,6 +74,22 @@ ALTER TABLE table_name [ON CLUSTER cluster] DROP DETACHED PARTITION|PART partiti Removes the specified part or all parts of the specified partition from `detached`. Read more about setting the partition expression in a section [How to set the partition expression](#how-to-set-partition-expression). +## FORGET PARTITION + +``` sql +ALTER TABLE table_name FORGET PARTITION partition_expr +``` + +Removes all data about empty partition from ZooKeeper. Query fails if partition is not empty. Make sure to execute only for partitions that will never be used again. + +Read about setting the partition expression in a section [How to set the partition expression](#how-to-set-partition-expression). + +Example: + +``` sql +ALTER TABLE mt FORGET PARTITION '20201121'; +``` + ## ATTACH PARTITION\|PART ``` sql From 578bbca03640154749cc88132045d3ae6797b3a8 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 20 Feb 2024 23:57:10 +0000 Subject: [PATCH 5/8] fix test --- tests/queries/0_stateless/02995_forget_partition.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02995_forget_partition.sql b/tests/queries/0_stateless/02995_forget_partition.sql index e769b9c8220..0676ae2afa9 100644 --- a/tests/queries/0_stateless/02995_forget_partition.sql +++ b/tests/queries/0_stateless/02995_forget_partition.sql @@ -12,7 +12,7 @@ engine = ReplicatedMergeTree('/test/02995/{database}/rmt', '1') order by (k, d) partition by toYYYYMMDD(d); -insert into forget_partition select number, '2024-01-01' + interval number day, base64Encode(randomString(20)) from system.numbers limit 10; +insert into forget_partition select number, '2024-01-01' + interval number day, randomString(20) from system.numbers limit 10; alter table forget_partition drop partition '20240101'; alter table forget_partition drop partition '20240102'; From 4dc3a5a46580269cb351ca00c6475337c2c05470 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 21 Feb 2024 12:24:58 +0000 Subject: [PATCH 6/8] fix test --- tests/queries/0_stateless/02995_forget_partition.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02995_forget_partition.sql b/tests/queries/0_stateless/02995_forget_partition.sql index 0676ae2afa9..91627b87052 100644 --- a/tests/queries/0_stateless/02995_forget_partition.sql +++ b/tests/queries/0_stateless/02995_forget_partition.sql @@ -1,4 +1,4 @@ --- Tags: zookeeper +-- Tags: zookeeper, no-replicated-database drop table if exists forget_partition; From 8b7ef7c89585dd35049e2379257ff5b6b78f3403 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 21 Feb 2024 14:53:19 +0000 Subject: [PATCH 7/8] assert is not needed --- src/Storages/StorageReplicatedMergeTree.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index fa59b3fbe35..ad3be6427f6 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7261,7 +7261,6 @@ void StorageReplicatedMergeTree::fetchPartition( void StorageReplicatedMergeTree::forgetPartition(const ASTPtr & partition, ContextPtr query_context) { - assertNotReadonly(); zkutil::ZooKeeperPtr zookeeper = getZooKeeperAndAssertNotReadonly(); String partition_id = getPartitionIDFromQuery(partition, query_context); From 69c5fae1bcb763cbf7ee332eaacc82e49d6f36d0 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 21 Feb 2024 15:11:50 +0000 Subject: [PATCH 8/8] review fixes --- docs/en/sql-reference/statements/alter/partition.md | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 12 +++++++++--- tests/queries/0_stateless/02995_forget_partition.sql | 2 +- 3 files changed, 12 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 8227e3a2584..a21ef4f4af5 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -9,7 +9,7 @@ The following operations with [partitions](/docs/en/engines/table-engines/merget - [DETACH PARTITION\|PART](#detach-partitionpart) — Moves a partition or part to the `detached` directory and forget it. - [DROP PARTITION\|PART](#drop-partitionpart) — Deletes a partition or part. -- [FORGET PARTITION](#forget-partition) — Deletes a partition completely from ZooKeeper. +- [FORGET PARTITION](#forget-partition) — Deletes a partition metadata from zookeeper if it's empty. - [ATTACH PARTITION\|PART](#attach-partitionpart) — Adds a partition or part from the `detached` directory to the table. - [ATTACH PARTITION FROM](#attach-partition-from) — Copies the data partition from one table to another and adds. - [REPLACE PARTITION](#replace-partition) — Copies the data partition from one table to another and replaces. @@ -80,7 +80,7 @@ Read more about setting the partition expression in a section [How to set the pa ALTER TABLE table_name FORGET PARTITION partition_expr ``` -Removes all data about empty partition from ZooKeeper. Query fails if partition is not empty. Make sure to execute only for partitions that will never be used again. +Removes all metadata about an empty partition from ZooKeeper. Query fails if partition is not empty or unknown. Make sure to execute only for partitions that will never be used again. Read about setting the partition expression in a section [How to set the partition expression](#how-to-set-partition-expression). diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ad3be6427f6..ca675051097 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -188,6 +188,7 @@ namespace ErrorCodes extern const int CANNOT_BACKUP_TABLE; extern const int SUPPORT_IS_DISABLED; extern const int FAULT_INJECTED; + extern const int CANNOT_FORGET_PARTITION; } namespace ActionLocks @@ -7264,11 +7265,16 @@ void StorageReplicatedMergeTree::forgetPartition(const ASTPtr & partition, Conte zkutil::ZooKeeperPtr zookeeper = getZooKeeperAndAssertNotReadonly(); String partition_id = getPartitionIDFromQuery(partition, query_context); - LOG_INFO(log, "Forget partition {}", partition_id); - String block_numbers_path = fs::path(zookeeper_path) / "block_numbers"; String partition_path = fs::path(block_numbers_path) / partition_id; - zookeeper->remove(partition_path); + + auto error_code = zookeeper->tryRemove(partition_path); + if (error_code == Coordination::Error::ZOK) + LOG_INFO(log, "Forget partition {}", partition_id); + else if (error_code == Coordination::Error::ZNONODE) + throw Exception(ErrorCodes::CANNOT_FORGET_PARTITION, "Partition {} is unknown", partition_id); + else + throw zkutil::KeeperException::fromPath(error_code, partition_path); } diff --git a/tests/queries/0_stateless/02995_forget_partition.sql b/tests/queries/0_stateless/02995_forget_partition.sql index 91627b87052..269f7932ea4 100644 --- a/tests/queries/0_stateless/02995_forget_partition.sql +++ b/tests/queries/0_stateless/02995_forget_partition.sql @@ -23,7 +23,7 @@ select '---before---'; select name from system.zookeeper where path = '/test/02995/' || currentDatabase() || '/rmt/block_numbers' order by name; alter table forget_partition forget partition '20240103'; -- {serverError CANNOT_FORGET_PARTITION} -alter table forget_partition forget partition '20240203'; -- {serverError KEEPER_EXCEPTION} +alter table forget_partition forget partition '20240203'; -- {serverError CANNOT_FORGET_PARTITION} alter table forget_partition forget partition '20240101';