mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
fix: can not drop local table and replace ALTER with SYSTEM
fix test_drop_replica fix drop replica '/path/to/zk/' ending in '/' and update doc
This commit is contained in:
parent
0a4af8f0a7
commit
2e6a3eff8e
@ -197,16 +197,6 @@ Constraint check *will not be executed* on existing data if it was added.
|
||||
|
||||
All changes on replicated tables are broadcasting to ZooKeeper so will be applied on other replicas.
|
||||
|
||||
### Manipulations with Replicas {#manipulations-with-replicas}
|
||||
|
||||
Replicas can be dropped using following syntax:
|
||||
|
||||
```sql
|
||||
ALTER TABLE [db].name DROP REPLICA replica_name;
|
||||
```
|
||||
|
||||
Queries will remove the replica path in zookeeper, it's useful when you want to decrease your replica factor. It will only drop the inactive/stale replica, and it can't drop local replica, please use `DROP TABLE` for that.
|
||||
|
||||
### Manipulations with Partitions and Parts {#alter_manipulations-with-partitions}
|
||||
|
||||
The following operations with [partitions](../../engines/table-engines/mergetree-family/custom-partitioning-key.md) are available:
|
||||
|
@ -12,6 +12,7 @@ toc_title: SYSTEM
|
||||
- [DROP MARK CACHE](#query_language-system-drop-mark-cache)
|
||||
- [DROP UNCOMPRESSED CACHE](#query_language-system-drop-uncompressed-cache)
|
||||
- [DROP COMPILED EXPRESSION CACHE](#query_language-system-drop-compiled-expression-cache)
|
||||
- [DROP REPLICA TABLE](#query_language-system-drop-replica-table)
|
||||
- [FLUSH LOGS](#query_language-system-flush_logs)
|
||||
- [RELOAD CONFIG](#query_language-system-reload-config)
|
||||
- [SHUTDOWN](#query_language-system-shutdown)
|
||||
@ -77,6 +78,17 @@ For manage uncompressed data cache parameters use following server level setting
|
||||
Reset the compiled expression cache. Used in development of ClickHouse and performance tests.
|
||||
Complied expression cache used when query/user/profile enable option [compile](../../operations/settings/settings.md#compile)
|
||||
|
||||
## DROP REPLICA TABLE {query_language-system-drop-replica-table}
|
||||
|
||||
Replicas can be dropped using following syntax:
|
||||
|
||||
```sql
|
||||
SYSTEM DROP REPLICA replica_name FROM [db].name;
|
||||
SYSTEM DROP REPLICA replica_name '/path/to/table/in/zk';
|
||||
```
|
||||
|
||||
Queries will remove the replica path in zookeeper, it's useful when you want to decrease your replica factor. It will only drop the inactive/stale replica, and it can't drop local replica, please use `SYSTEM DROP REPLICA` for that.
|
||||
|
||||
## FLUSH LOGS {#query_language-system-flush_logs}
|
||||
|
||||
Flushes buffers of log messages to system tables (e.g. system.query\_log). Allows you to not wait 7.5 seconds when debugging.
|
||||
|
@ -9,8 +9,6 @@
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/MutationCommands.h>
|
||||
#include <Storages/PartitionCommands.h>
|
||||
#include <Storages/ReplicaCommands.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/LiveView/LiveViewCommands.h>
|
||||
#include <Storages/LiveView/StorageLiveView.h>
|
||||
#include <Access/AccessRightsElement.h>
|
||||
@ -54,7 +52,6 @@ BlockIO InterpreterAlterQuery::execute()
|
||||
|
||||
AlterCommands alter_commands;
|
||||
PartitionCommands partition_commands;
|
||||
ReplicaCommands replica_commands;
|
||||
MutationCommands mutation_commands;
|
||||
LiveViewCommands live_view_commands;
|
||||
for (ASTAlterCommand * command_ast : alter.command_list->commands)
|
||||
@ -69,10 +66,6 @@ BlockIO InterpreterAlterQuery::execute()
|
||||
"(see allow_drop_detached setting)", ErrorCodes::SUPPORT_IS_DISABLED);
|
||||
partition_commands.emplace_back(std::move(*partition_command));
|
||||
}
|
||||
else if (auto replica_command = ReplicaCommand::parse(command_ast))
|
||||
{
|
||||
replica_commands.emplace_back(std::move(*replica_command));
|
||||
}
|
||||
else if (auto mut_command = MutationCommand::parse(command_ast))
|
||||
{
|
||||
if (mut_command->type == MutationCommand::MATERIALIZE_TTL && !table->hasAnyTTL())
|
||||
@ -101,18 +94,6 @@ BlockIO InterpreterAlterQuery::execute()
|
||||
table->alterPartition(query_ptr, partition_commands, context);
|
||||
}
|
||||
|
||||
if (!replica_commands.empty())
|
||||
{
|
||||
replica_commands.validate(*table);
|
||||
auto replicate_table = std::dynamic_pointer_cast<StorageReplicatedMergeTree>(table);
|
||||
|
||||
auto table_lock_holder = table->lockAlterIntention(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
for (auto & command : replica_commands)
|
||||
{
|
||||
replicate_table->dropReplica(command.replica_name);
|
||||
}
|
||||
}
|
||||
|
||||
if (!live_view_commands.empty())
|
||||
{
|
||||
live_view_commands.validate(*table);
|
||||
@ -292,11 +273,6 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS
|
||||
required_access.emplace_back(AccessType::ALTER_FREEZE_PARTITION, database, table);
|
||||
break;
|
||||
}
|
||||
case ASTAlterCommand::DROP_REPLICA:
|
||||
{
|
||||
required_access.emplace_back(AccessType::ALTER_DELETE, database, table);
|
||||
break;
|
||||
}
|
||||
case ASTAlterCommand::MODIFY_QUERY:
|
||||
{
|
||||
required_access.emplace_back(AccessType::ALTER_VIEW_MODIFY_QUERY, database, table);
|
||||
|
@ -421,8 +421,26 @@ void InterpreterSystemQuery::dropReplica(ASTSystemQuery & query)
|
||||
else
|
||||
{
|
||||
context.checkAccess(AccessType::SYSTEM_DROP_REPLICA);
|
||||
auto zookeeper = context.getZooKeeper();
|
||||
auto to_drop_path = query.replica_zk_path + "/replicas/" + query.replica;
|
||||
auto & catalog = DatabaseCatalog::instance();
|
||||
StorageReplicatedMergeTree::Status status;
|
||||
|
||||
for (auto & elem : catalog.getDatabases())
|
||||
{
|
||||
DatabasePtr & database = elem.second;
|
||||
for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next())
|
||||
{
|
||||
if (auto * storage_replicated = dynamic_cast<StorageReplicatedMergeTree *>(iterator->table().get()))
|
||||
{
|
||||
storage_replicated->getStatus(status);
|
||||
if (to_drop_path.compare(status.replica_path) == 0)
|
||||
throw Exception("We can't drop local replica, please use `DROP TABLE` if you want to clean the data and drop this replica",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
auto zookeeper = context.getZooKeeper();
|
||||
|
||||
// TODO check if local table have this this replica_path
|
||||
//check if is active replica if we drop other replicas
|
||||
|
@ -61,11 +61,6 @@ ASTPtr ASTAlterCommand::clone() const
|
||||
res->rename_to = rename_to->clone();
|
||||
res->children.push_back(res->rename_to);
|
||||
}
|
||||
if (replica)
|
||||
{
|
||||
res->replica = replica->clone();
|
||||
res->children.push_back(res->replica);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
@ -304,10 +299,6 @@ void ASTAlterCommand::formatImpl(
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " TO ";
|
||||
rename_to->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (type == ASTAlterCommand::DROP_REPLICA)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "DROP REPLICA" << (settings.hilite ? hilite_none : "") << " " << std::quoted(replica_name, '\'');
|
||||
}
|
||||
else
|
||||
throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||
}
|
||||
|
@ -58,8 +58,6 @@ public:
|
||||
NO_TYPE,
|
||||
|
||||
LIVE_VIEW_REFRESH,
|
||||
|
||||
DROP_REPLICA,
|
||||
};
|
||||
|
||||
Type type = NO_TYPE;
|
||||
@ -104,9 +102,6 @@ public:
|
||||
*/
|
||||
ASTPtr partition;
|
||||
|
||||
// FOR DROP REPLICA queries
|
||||
ASTPtr replica;
|
||||
|
||||
/// For DELETE/UPDATE WHERE: the predicate that filters the rows to delete/update.
|
||||
ASTPtr predicate;
|
||||
|
||||
@ -164,8 +159,6 @@ public:
|
||||
|
||||
/// Target column name
|
||||
ASTPtr rename_to;
|
||||
/// DROP REPLICA
|
||||
String replica_name;
|
||||
|
||||
String getID(char delim) const override { return "AlterCommand" + (delim + std::to_string(static_cast<int>(type))); }
|
||||
|
||||
|
@ -43,8 +43,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserKeyword s_add_constraint("ADD CONSTRAINT");
|
||||
ParserKeyword s_drop_constraint("DROP CONSTRAINT");
|
||||
|
||||
ParserKeyword s_drop_replica("DROP REPLICA");
|
||||
|
||||
ParserKeyword s_add("ADD");
|
||||
ParserKeyword s_drop("DROP");
|
||||
ParserKeyword s_suspend("SUSPEND");
|
||||
@ -423,14 +421,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
command->with_name = ast_with_name->as<ASTLiteral &>().value.get<const String &>();
|
||||
}
|
||||
}
|
||||
else if (s_drop_replica.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_string_literal.parse(pos, command->replica, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_REPLICA;
|
||||
command->replica_name = command->replica->as<ASTLiteral &>().value.get<const String &>();
|
||||
}
|
||||
else if (s_modify_column.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
@ -520,8 +510,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
command->children.push_back(command->column);
|
||||
if (command->partition)
|
||||
command->children.push_back(command->partition);
|
||||
if (command->replica)
|
||||
command->children.push_back(command->replica);
|
||||
if (command->order_by)
|
||||
command->children.push_back(command->order_by);
|
||||
if (command->predicate)
|
||||
|
@ -17,7 +17,6 @@ namespace DB
|
||||
* [MODIFY SETTING setting_name=setting_value, ...]
|
||||
* [COMMENT COLUMN [IF EXISTS] col_name string]
|
||||
* [DROP|DETACH|ATTACH PARTITION|PART partition, ...]
|
||||
* [DROP REPLICA replica, ...]
|
||||
* [FETCH PARTITION partition FROM ...]
|
||||
* [FREEZE [PARTITION] [WITH NAME name]]
|
||||
* [DELETE WHERE ...]
|
||||
|
@ -72,7 +72,10 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
|
||||
ASTPtr path_ast;
|
||||
if (!ParserStringLiteral{}.parse(pos, path_ast, expected))
|
||||
return false;
|
||||
res->replica_zk_path = path_ast->as<ASTLiteral &>().value.safeGet<String>();
|
||||
String zk_path = path_ast->as<ASTLiteral &>().value.safeGet<String>();
|
||||
if (zk_path[zk_path.size()-1] == '/')
|
||||
zk_path.pop_back();
|
||||
res->replica_zk_path = zk_path;
|
||||
}
|
||||
|
||||
break;
|
||||
|
@ -1,41 +0,0 @@
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
|
||||
#include <Storages/ReplicaCommands.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_STORAGE;
|
||||
}
|
||||
|
||||
std::optional<ReplicaCommand> ReplicaCommand::parse(const ASTAlterCommand * command_ast)
|
||||
{
|
||||
if (command_ast->type == ASTAlterCommand::DROP_REPLICA)
|
||||
{
|
||||
ReplicaCommand res;
|
||||
res.type = DROP_REPLICA;
|
||||
res.replica = command_ast->replica;
|
||||
res.replica_name = command_ast->replica_name;
|
||||
return res;
|
||||
}
|
||||
else
|
||||
return {};
|
||||
}
|
||||
|
||||
void ReplicaCommands::validate(const IStorage & table)
|
||||
{
|
||||
for (const ReplicaCommand & command : *this)
|
||||
{
|
||||
if (command.type == ReplicaCommand::DROP_REPLICA)
|
||||
{
|
||||
if (!empty() && !dynamic_cast<const StorageReplicatedMergeTree *>(&table))
|
||||
throw Exception("Wrong storage type. Must be StorageReplicateMergeTree", DB::ErrorCodes::UNKNOWN_STORAGE);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -1,39 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
|
||||
#include <optional>
|
||||
#include <vector>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ASTAlterCommand;
|
||||
|
||||
struct ReplicaCommand
|
||||
{
|
||||
enum Type
|
||||
{
|
||||
DROP_REPLICA,
|
||||
};
|
||||
|
||||
Type type;
|
||||
|
||||
ASTPtr replica;
|
||||
String replica_name;
|
||||
|
||||
static std::optional<ReplicaCommand> parse(const ASTAlterCommand * command);
|
||||
};
|
||||
|
||||
class ReplicaCommands : public std::vector<ReplicaCommand>
|
||||
{
|
||||
public:
|
||||
void validate(const IStorage & table);
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -115,12 +115,12 @@ public:
|
||||
*/
|
||||
void drop() override;
|
||||
|
||||
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
||||
|
||||
/** Removes a specific replica from Zookeeper.
|
||||
*/
|
||||
void dropReplica(const String & replica_name);
|
||||
|
||||
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
||||
|
||||
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||
|
||||
bool supportsIndexForIn() const override { return true; }
|
||||
@ -311,7 +311,6 @@ private:
|
||||
*/
|
||||
void removeReplica(const String & replica);
|
||||
|
||||
|
||||
/** Create nodes in the ZK, which must always be, but which might not exist when older versions of the server are running.
|
||||
*/
|
||||
void createNewZooKeeperNodes();
|
||||
|
@ -28,7 +28,7 @@ def start_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
fill_nodes([node_1_1, node_1_2], 1)
|
||||
fill_nodes([node_1_1, node_1_2, node_1_3], 1)
|
||||
|
||||
yield cluster
|
||||
|
||||
@ -46,6 +46,7 @@ def test_drop_replica(start_cluster):
|
||||
|
||||
assert "can't drop local replica" in node_1_1.query_and_get_error("SYSTEM DROP REPLICA 'node_1_1' FROM test.test_table")
|
||||
assert "can't drop local replica" in node_1_2.query_and_get_error("SYSTEM DROP REPLICA 'node_1_2' FROM test.test_table")
|
||||
assert "can't drop local replica" in node_1_3.query_and_get_error("SYSTEM DROP REPLICA 'node_1_3' FROM '/clickhouse/tables/test/{shard}/replicated'".format(shard=1))
|
||||
assert "it's active" in node_1_1.query_and_get_error("SYSTEM DROP REPLICA 'node_1_2' FROM test.test_table")
|
||||
|
||||
with PartitionManager() as pm:
|
||||
|
Loading…
Reference in New Issue
Block a user