mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #11896 from ClickHouse/merging_drop_replica
Merging #10679
This commit is contained in:
commit
2401bc9143
@ -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](#query_language-system-drop-replica)
|
||||
- [FLUSH LOGS](#query_language-system-flush_logs)
|
||||
- [RELOAD CONFIG](#query_language-system-reload-config)
|
||||
- [SHUTDOWN](#query_language-system-shutdown)
|
||||
@ -67,6 +68,24 @@ For more convenient (automatic) cache management, see disable\_internal\_dns\_ca
|
||||
|
||||
Resets the mark cache. Used in development of ClickHouse and performance tests.
|
||||
|
||||
## DROP REPLICA {#query_language-system-drop-replica}
|
||||
|
||||
Dead replicas can be dropped using following syntax:
|
||||
|
||||
```sql
|
||||
SYSTEM DROP REPLICA 'replica_name' FROM TABLE database.table;
|
||||
SYSTEM DROP REPLICA 'replica_name' FROM DATABASE database;
|
||||
SYSTEM DROP REPLICA 'replica_name';
|
||||
SYSTEM DROP REPLICA 'replica_name' FROM ZKPATH '/path/to/table/in/zk';
|
||||
```
|
||||
|
||||
Queries will remove the replica path in ZooKeeper. It's useful when replica is dead and its metadata cannot be removed from ZooKeeper by `DROP TABLE` because there is no such table anymore. It will only drop the inactive/stale replica, and it can't drop local replica, please use `DROP TABLE` for that. `DROP REPLICA` does not drop any tables and does not remove any data or metadata from disk.
|
||||
|
||||
The first one removes metadata of `'replica_name'` replica of `database.table` table.
|
||||
The second one does the same for all replicated tables in the database.
|
||||
The third one does the same for all replicated tables on local server.
|
||||
The forth one is useful to remove metadata of dead replica when all other replicas of a table were dropped. It requires the table path to be specified explicitly. It must be the same path as was passed to the first argument of `ReplicatedMergeTree` engine on table creation.
|
||||
|
||||
## DROP UNCOMPRESSED CACHE {#query_language-system-drop-uncompressed-cache}
|
||||
|
||||
Reset the uncompressed data cache. Used in development of ClickHouse and performance tests.
|
||||
|
@ -133,6 +133,7 @@ enum class AccessType
|
||||
M(SYSTEM_REPLICATED_SENDS, "SYSTEM STOP REPLICATED SENDS, SYSTEM START REPLICATED SENDS, STOP_REPLICATED_SENDS, START REPLICATED SENDS", TABLE, SYSTEM_SENDS) \
|
||||
M(SYSTEM_SENDS, "SYSTEM STOP SENDS, SYSTEM START SENDS, STOP SENDS, START SENDS", GROUP, SYSTEM) \
|
||||
M(SYSTEM_REPLICATION_QUEUES, "SYSTEM STOP REPLICATION QUEUES, SYSTEM START REPLICATION QUEUES, STOP_REPLICATION_QUEUES, START REPLICATION QUEUES", TABLE, SYSTEM) \
|
||||
M(SYSTEM_DROP_REPLICA, "DROP REPLICA", TABLE, SYSTEM) \
|
||||
M(SYSTEM_SYNC_REPLICA, "SYNC REPLICA", TABLE, SYSTEM) \
|
||||
M(SYSTEM_RESTART_REPLICA, "RESTART REPLICA", TABLE, SYSTEM) \
|
||||
M(SYSTEM_FLUSH_DISTRIBUTED, "FLUSH DISTRIBUTED", TABLE, SYSTEM_FLUSH) \
|
||||
|
@ -49,6 +49,7 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_KILL;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int TIMEOUT_EXCEEDED;
|
||||
extern const int TABLE_WAS_NOT_DROPPED;
|
||||
}
|
||||
|
||||
|
||||
@ -185,7 +186,7 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
|
||||
/// Make canonical query for simpler processing
|
||||
if (!query.table.empty())
|
||||
table_id = context.resolveStorageID(StorageID(query.database, query.table), Context::ResolveOrdinary);
|
||||
table_id = context.resolveStorageID(StorageID(query.database, query.table), Context::ResolveOrdinary);
|
||||
|
||||
if (!query.target_dictionary.empty() && !query.database.empty())
|
||||
query.target_dictionary = query.database + "." + query.target_dictionary;
|
||||
@ -285,6 +286,9 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
case Type::START_DISTRIBUTED_SENDS:
|
||||
startStopAction(ActionLocks::DistributedSend, true);
|
||||
break;
|
||||
case Type::DROP_REPLICA:
|
||||
dropReplica(query);
|
||||
break;
|
||||
case Type::SYNC_REPLICA:
|
||||
syncReplica(query);
|
||||
break;
|
||||
@ -400,6 +404,111 @@ void InterpreterSystemQuery::restartReplicas(Context & system_context)
|
||||
pool.wait();
|
||||
}
|
||||
|
||||
void InterpreterSystemQuery::dropReplica(ASTSystemQuery & query)
|
||||
{
|
||||
if (query.replica.empty())
|
||||
throw Exception("Replica name is empty", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (!table_id.empty())
|
||||
{
|
||||
context.checkAccess(AccessType::SYSTEM_DROP_REPLICA, table_id);
|
||||
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context);
|
||||
|
||||
if (!dropReplicaImpl(query, table))
|
||||
throw Exception("Table " + table_id.getNameForLogs() + " is not replicated", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
else if (!query.database.empty())
|
||||
{
|
||||
context.checkAccess(AccessType::SYSTEM_DROP_REPLICA, query.database);
|
||||
DatabasePtr database = DatabaseCatalog::instance().getDatabase(query.database);
|
||||
for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
dropReplicaImpl(query, iterator->table());
|
||||
LOG_TRACE(log, "Dropped replica {} from database {}", query.replica, backQuoteIfNeed(database->getDatabaseName()));
|
||||
}
|
||||
else if (query.is_drop_whole_replica)
|
||||
{
|
||||
context.checkAccess(AccessType::SYSTEM_DROP_REPLICA);
|
||||
auto databases = DatabaseCatalog::instance().getDatabases();
|
||||
|
||||
for (auto & elem : databases)
|
||||
{
|
||||
DatabasePtr & database = elem.second;
|
||||
for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
dropReplicaImpl(query, iterator->table());
|
||||
LOG_TRACE(log, "Dropped replica {} from database {}", query.replica, backQuoteIfNeed(database->getDatabaseName()));
|
||||
}
|
||||
}
|
||||
else if (!query.replica_zk_path.empty())
|
||||
{
|
||||
context.checkAccess(AccessType::SYSTEM_DROP_REPLICA);
|
||||
auto remote_replica_path = query.replica_zk_path + "/replicas/" + query.replica;
|
||||
|
||||
/// This check is actually redundant, but it may prevent from some user mistakes
|
||||
for (auto & elem : DatabaseCatalog::instance().getDatabases())
|
||||
{
|
||||
DatabasePtr & database = elem.second;
|
||||
for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
{
|
||||
if (auto * storage_replicated = dynamic_cast<StorageReplicatedMergeTree *>(iterator->table().get()))
|
||||
{
|
||||
StorageReplicatedMergeTree::Status status;
|
||||
storage_replicated->getStatus(status);
|
||||
if (status.zookeeper_path == query.replica_zk_path)
|
||||
throw Exception("There is a local table " + storage_replicated->getStorageID().getNameForLogs() +
|
||||
", which has the same table path in ZooKeeper. Please check the path in query. "
|
||||
"If you want to drop replica of this table, use `DROP TABLE` "
|
||||
"or `SYSTEM DROP REPLICA 'name' FROM db.table`", ErrorCodes::TABLE_WAS_NOT_DROPPED);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
auto zookeeper = context.getZooKeeper();
|
||||
|
||||
bool looks_like_table_path = zookeeper->exists(query.replica_zk_path + "/replicas") ||
|
||||
zookeeper->exists(query.replica_zk_path + "/dropped");
|
||||
if (!looks_like_table_path)
|
||||
throw Exception("Specified path " + query.replica_zk_path + " does not look like a table path",
|
||||
ErrorCodes::TABLE_WAS_NOT_DROPPED);
|
||||
|
||||
if (zookeeper->exists(remote_replica_path + "/is_active"))
|
||||
throw Exception("Can't remove replica: " + query.replica + ", because it's active",
|
||||
ErrorCodes::TABLE_WAS_NOT_DROPPED);
|
||||
|
||||
StorageReplicatedMergeTree::dropReplica(zookeeper, query.replica_zk_path, query.replica, log);
|
||||
LOG_INFO(log, "Dropped replica {}", remote_replica_path);
|
||||
}
|
||||
else
|
||||
throw Exception("Invalid query", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
bool InterpreterSystemQuery::dropReplicaImpl(ASTSystemQuery & query, const StoragePtr & table)
|
||||
{
|
||||
auto * storage_replicated = dynamic_cast<StorageReplicatedMergeTree *>(table.get());
|
||||
if (!storage_replicated)
|
||||
return false;
|
||||
|
||||
StorageReplicatedMergeTree::Status status;
|
||||
auto zookeeper = context.getZooKeeper();
|
||||
storage_replicated->getStatus(status);
|
||||
|
||||
/// Do not allow to drop local replicas and active remote replicas
|
||||
if (query.replica == status.replica_name)
|
||||
throw Exception("We can't drop local replica, please use `DROP TABLE` "
|
||||
"if you want to clean the data and drop this replica", ErrorCodes::TABLE_WAS_NOT_DROPPED);
|
||||
|
||||
/// NOTE it's not atomic: replica may become active after this check, but before dropReplica(...)
|
||||
/// However, the main usecase is to drop dead replica, which cannot become active.
|
||||
/// This check prevents only from accidental drop of some other replica.
|
||||
if (zookeeper->exists(status.zookeeper_path + "/replicas/" + query.replica + "/is_active"))
|
||||
throw Exception("Can't drop replica: " + query.replica + ", because it's active",
|
||||
ErrorCodes::TABLE_WAS_NOT_DROPPED);
|
||||
|
||||
storage_replicated->dropReplica(zookeeper, status.zookeeper_path, query.replica, log);
|
||||
LOG_TRACE(log, "Dropped replica {} of {}", query.replica, table->getStorageID().getNameForLogs());
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
void InterpreterSystemQuery::syncReplica(ASTSystemQuery &)
|
||||
{
|
||||
context.checkAccess(AccessType::SYSTEM_SYNC_REPLICA, table_id);
|
||||
@ -530,6 +639,11 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster()
|
||||
required_access.emplace_back(AccessType::SYSTEM_REPLICATION_QUEUES, query.database, query.table);
|
||||
break;
|
||||
}
|
||||
case Type::DROP_REPLICA:
|
||||
{
|
||||
required_access.emplace_back(AccessType::SYSTEM_DROP_REPLICA, query.database, query.table);
|
||||
break;
|
||||
}
|
||||
case Type::SYNC_REPLICA:
|
||||
{
|
||||
required_access.emplace_back(AccessType::SYSTEM_SYNC_REPLICA, query.database, query.table);
|
||||
|
@ -51,6 +51,8 @@ private:
|
||||
|
||||
void restartReplicas(Context & system_context);
|
||||
void syncReplica(ASTSystemQuery & query);
|
||||
void dropReplica(ASTSystemQuery & query);
|
||||
bool dropReplicaImpl(ASTSystemQuery & query, const StoragePtr & table);
|
||||
void flushDistributed(ASTSystemQuery & query);
|
||||
|
||||
AccessRightsElements getRequiredAccessForDDLOnCluster() const;
|
||||
|
@ -9,7 +9,7 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_TYPE_OF_FIELD;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
@ -39,6 +39,8 @@ const char * ASTSystemQuery::typeToString(Type type)
|
||||
return "RESTART REPLICAS";
|
||||
case Type::RESTART_REPLICA:
|
||||
return "RESTART REPLICA";
|
||||
case Type::DROP_REPLICA:
|
||||
return "DROP REPLICA";
|
||||
case Type::SYNC_REPLICA:
|
||||
return "SYNC REPLICA";
|
||||
case Type::FLUSH_DISTRIBUTED:
|
||||
@ -82,15 +84,15 @@ const char * ASTSystemQuery::typeToString(Type type)
|
||||
case Type::FLUSH_LOGS:
|
||||
return "FLUSH LOGS";
|
||||
default:
|
||||
throw Exception("Unknown SYSTEM query command", ErrorCodes::BAD_TYPE_OF_FIELD);
|
||||
throw Exception("Unknown SYSTEM query command", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SYSTEM " << (settings.hilite ? hilite_none : "");
|
||||
settings.ostr << typeToString(type);
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SYSTEM ";
|
||||
settings.ostr << typeToString(type) << (settings.hilite ? hilite_none : "");
|
||||
|
||||
auto print_database_table = [&]
|
||||
{
|
||||
@ -116,6 +118,28 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
};
|
||||
|
||||
auto print_drop_replica = [&] {
|
||||
settings.ostr << " " << quoteString(replica);
|
||||
if (!table.empty())
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM TABLE"
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
print_database_table();
|
||||
}
|
||||
else if (!replica_zk_path.empty())
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM ZKPATH "
|
||||
<< (settings.hilite ? hilite_none : "") << quoteString(replica_zk_path);
|
||||
}
|
||||
else if (!database.empty())
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM DATABASE "
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(database)
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
}
|
||||
};
|
||||
|
||||
if (!cluster.empty())
|
||||
formatOnCluster(settings);
|
||||
|
||||
@ -143,6 +167,8 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
|
||||
}
|
||||
else if (type == Type::RELOAD_DICTIONARY)
|
||||
print_database_dictionary();
|
||||
else if (type == Type::DROP_REPLICA)
|
||||
print_drop_replica();
|
||||
}
|
||||
|
||||
|
||||
|
@ -30,6 +30,7 @@ public:
|
||||
START_LISTEN_QUERIES,
|
||||
RESTART_REPLICAS,
|
||||
RESTART_REPLICA,
|
||||
DROP_REPLICA,
|
||||
SYNC_REPLICA,
|
||||
RELOAD_DICTIONARY,
|
||||
RELOAD_DICTIONARIES,
|
||||
@ -61,6 +62,9 @@ public:
|
||||
String target_dictionary;
|
||||
String database;
|
||||
String table;
|
||||
String replica;
|
||||
String replica_zk_path;
|
||||
bool is_drop_whole_replica;
|
||||
|
||||
String getID(char) const override { return "SYSTEM query"; }
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Parsers/ASTSystemQuery.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/parseDatabaseAndTableName.h>
|
||||
|
||||
@ -57,6 +58,48 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
|
||||
break;
|
||||
}
|
||||
|
||||
case Type::DROP_REPLICA:
|
||||
{
|
||||
ASTPtr ast;
|
||||
if (!ParserStringLiteral{}.parse(pos, ast, expected))
|
||||
return false;
|
||||
res->replica = ast->as<ASTLiteral &>().value.safeGet<String>();
|
||||
if (ParserKeyword{"FROM"}.ignore(pos, expected))
|
||||
{
|
||||
// way 1. parse replica database
|
||||
// way 2. parse replica tables
|
||||
// way 3. parse replica zkpath
|
||||
if (ParserKeyword{"DATABASE"}.ignore(pos, expected))
|
||||
{
|
||||
ParserIdentifier database_parser;
|
||||
ASTPtr database;
|
||||
if (!database_parser.parse(pos, database, expected))
|
||||
return false;
|
||||
tryGetIdentifierNameInto(database, res->database);
|
||||
}
|
||||
else if (ParserKeyword{"TABLE"}.ignore(pos, expected))
|
||||
{
|
||||
parseDatabaseAndTableName(pos, expected, res->database, res->table);
|
||||
}
|
||||
else if (ParserKeyword{"ZKPATH"}.ignore(pos, expected))
|
||||
{
|
||||
ASTPtr path_ast;
|
||||
if (!ParserStringLiteral{}.parse(pos, path_ast, expected))
|
||||
return false;
|
||||
String zk_path = path_ast->as<ASTLiteral &>().value.safeGet<String>();
|
||||
if (!zk_path.empty() && zk_path[zk_path.size() - 1] == '/')
|
||||
zk_path.pop_back();
|
||||
res->replica_zk_path = zk_path;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
else
|
||||
res->is_drop_whole_replica = true;
|
||||
|
||||
break;
|
||||
}
|
||||
|
||||
case Type::RESTART_REPLICA:
|
||||
case Type::SYNC_REPLICA:
|
||||
if (!parseDatabaseAndTableName(pos, expected, res->database, res->table))
|
||||
|
@ -622,7 +622,8 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada
|
||||
|
||||
void StorageReplicatedMergeTree::drop()
|
||||
{
|
||||
/// There is also the case when user has configured ClickHouse to wrong ZooKeeper cluster,
|
||||
/// There is also the case when user has configured ClickHouse to wrong ZooKeeper cluster
|
||||
/// or metadata of staled replica were removed manually,
|
||||
/// in this case, has_metadata_in_zookeeper = false, and we also permit to drop the table.
|
||||
|
||||
if (has_metadata_in_zookeeper)
|
||||
@ -634,95 +635,99 @@ void StorageReplicatedMergeTree::drop()
|
||||
throw Exception("Can't drop readonly replicated table (need to drop data in ZooKeeper as well)", ErrorCodes::TABLE_IS_READ_ONLY);
|
||||
|
||||
shutdown();
|
||||
dropReplica(zookeeper, zookeeper_path, replica_name, log);
|
||||
}
|
||||
|
||||
if (zookeeper->expired())
|
||||
throw Exception("Table was not dropped because ZooKeeper session has expired.", ErrorCodes::TABLE_WAS_NOT_DROPPED);
|
||||
dropAllData();
|
||||
}
|
||||
|
||||
LOG_INFO(log, "Removing replica {}", replica_path);
|
||||
replica_is_active_node = nullptr;
|
||||
/// It may left some garbage if replica_path subtree are concurently modified
|
||||
zookeeper->tryRemoveRecursive(replica_path);
|
||||
if (zookeeper->exists(replica_path))
|
||||
LOG_ERROR(log, "Replica was not completely removed from ZooKeeper, {} still exists and may contain some garbage.", replica_path);
|
||||
void StorageReplicatedMergeTree::dropReplica(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, const String & replica, Poco::Logger * logger)
|
||||
{
|
||||
if (zookeeper->expired())
|
||||
throw Exception("Table was not dropped because ZooKeeper session has expired.", ErrorCodes::TABLE_WAS_NOT_DROPPED);
|
||||
|
||||
/// Check that `zookeeper_path` exists: it could have been deleted by another replica after execution of previous line.
|
||||
Strings replicas;
|
||||
if (Coordination::Error::ZOK == zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) && replicas.empty())
|
||||
auto remote_replica_path = zookeeper_path + "/replicas/" + replica;
|
||||
LOG_INFO(logger, "Removing replica {}", remote_replica_path);
|
||||
/// It may left some garbage if replica_path subtree are concurrently modified
|
||||
zookeeper->tryRemoveRecursive(remote_replica_path);
|
||||
if (zookeeper->exists(remote_replica_path))
|
||||
LOG_ERROR(logger, "Replica was not completely removed from ZooKeeper, {} still exists and may contain some garbage.", remote_replica_path);
|
||||
|
||||
/// Check that `zookeeper_path` exists: it could have been deleted by another replica after execution of previous line.
|
||||
Strings replicas;
|
||||
if (Coordination::Error::ZOK != zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) || !replicas.empty())
|
||||
return;
|
||||
|
||||
LOG_INFO(logger, "{} is the last replica, will remove table", remote_replica_path);
|
||||
|
||||
/** At this moment, another replica can be created and we cannot remove the table.
|
||||
* Try to remove /replicas node first. If we successfully removed it,
|
||||
* it guarantees that we are the only replica that proceed to remove the table
|
||||
* and no new replicas can be created after that moment (it requires the existence of /replicas node).
|
||||
* and table cannot be recreated with new /replicas node on another servers while we are removing data,
|
||||
* because table creation is executed in single transaction that will conflict with remaining nodes.
|
||||
*/
|
||||
|
||||
Coordination::Requests ops;
|
||||
Coordination::Responses responses;
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/replicas", -1));
|
||||
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/dropped", "", zkutil::CreateMode::Persistent));
|
||||
Coordination::Error code = zookeeper->tryMulti(ops, responses);
|
||||
|
||||
if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS)
|
||||
{
|
||||
LOG_WARNING(logger, "Table {} is already started to be removing by another replica right now", remote_replica_path);
|
||||
}
|
||||
else if (code == Coordination::Error::ZNOTEMPTY)
|
||||
{
|
||||
LOG_WARNING(logger, "Another replica was suddenly created, will keep the table {}", remote_replica_path);
|
||||
}
|
||||
else if (code != Coordination::Error::ZOK)
|
||||
{
|
||||
zkutil::KeeperMultiException::check(code, ops, responses);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_INFO(logger, "Removing table {} (this might take several minutes)", zookeeper_path);
|
||||
|
||||
Strings children;
|
||||
code = zookeeper->tryGetChildren(zookeeper_path, children);
|
||||
if (code == Coordination::Error::ZNONODE)
|
||||
{
|
||||
LOG_INFO(log, "{} is the last replica, will remove table", replica_path);
|
||||
LOG_WARNING(logger, "Table {} is already finished removing by another replica right now", remote_replica_path);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (const auto & child : children)
|
||||
if (child != "dropped")
|
||||
zookeeper->tryRemoveRecursive(zookeeper_path + "/" + child);
|
||||
|
||||
/** At this moment, another replica can be created and we cannot remove the table.
|
||||
* Try to remove /replicas node first. If we successfully removed it,
|
||||
* it guarantees that we are the only replica that proceed to remove the table
|
||||
* and no new replicas can be created after that moment (it requires the existence of /replicas node).
|
||||
* and table cannot be recreated with new /replicas node on another servers while we are removing data,
|
||||
* because table creation is executed in single transaction that will conflict with remaining nodes.
|
||||
*/
|
||||
ops.clear();
|
||||
responses.clear();
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/dropped", -1));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path, -1));
|
||||
code = zookeeper->tryMulti(ops, responses);
|
||||
|
||||
Coordination::Requests ops;
|
||||
Coordination::Responses responses;
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/replicas", -1));
|
||||
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/dropped", "", zkutil::CreateMode::Persistent));
|
||||
Coordination::Error code = zookeeper->tryMulti(ops, responses);
|
||||
|
||||
if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS)
|
||||
if (code == Coordination::Error::ZNONODE)
|
||||
{
|
||||
LOG_WARNING(log, "Table {} is already started to be removing by another replica right now", replica_path);
|
||||
LOG_WARNING(logger, "Table {} is already finished removing by another replica right now", remote_replica_path);
|
||||
}
|
||||
else if (code == Coordination::Error::ZNOTEMPTY)
|
||||
{
|
||||
LOG_WARNING(log, "Another replica was suddenly created, will keep the table {}", replica_path);
|
||||
LOG_ERROR(logger, "Table was not completely removed from ZooKeeper, {} still exists and may contain some garbage.",
|
||||
zookeeper_path);
|
||||
}
|
||||
else if (code != Coordination::Error::ZOK)
|
||||
{
|
||||
/// It is still possible that ZooKeeper session is expired or server is killed in the middle of the delete operation.
|
||||
zkutil::KeeperMultiException::check(code, ops, responses);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_INFO(log, "Removing table {} (this might take several minutes)", zookeeper_path);
|
||||
|
||||
Strings children;
|
||||
code = zookeeper->tryGetChildren(zookeeper_path, children);
|
||||
if (code == Coordination::Error::ZNONODE)
|
||||
{
|
||||
LOG_WARNING(log, "Table {} is already finished removing by another replica right now", replica_path);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (const auto & child : children)
|
||||
if (child != "dropped")
|
||||
zookeeper->tryRemoveRecursive(zookeeper_path + "/" + child);
|
||||
|
||||
ops.clear();
|
||||
responses.clear();
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/dropped", -1));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path, -1));
|
||||
code = zookeeper->tryMulti(ops, responses);
|
||||
|
||||
if (code == Coordination::Error::ZNONODE)
|
||||
{
|
||||
LOG_WARNING(log, "Table {} is already finished removing by another replica right now", replica_path);
|
||||
}
|
||||
else if (code == Coordination::Error::ZNOTEMPTY)
|
||||
{
|
||||
LOG_ERROR(log, "Table was not completely removed from ZooKeeper, {} still exists and may contain some garbage.",
|
||||
zookeeper_path);
|
||||
}
|
||||
else if (code != Coordination::Error::ZOK)
|
||||
{
|
||||
/// It is still possible that ZooKeeper session is expired or server is killed in the middle of the delete operation.
|
||||
zkutil::KeeperMultiException::check(code, ops, responses);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_INFO(log, "Table {} was successfully removed from ZooKeeper", zookeeper_path);
|
||||
}
|
||||
}
|
||||
LOG_INFO(logger, "Table {} was successfully removed from ZooKeeper", zookeeper_path);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
dropAllData();
|
||||
}
|
||||
|
||||
|
||||
|
@ -185,6 +185,10 @@ public:
|
||||
|
||||
int getMetadataVersion() const { return metadata_version; }
|
||||
|
||||
/** Remove a specific replica from zookeeper.
|
||||
*/
|
||||
static void dropReplica(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, const String & replica, Poco::Logger * logger);
|
||||
|
||||
private:
|
||||
|
||||
/// Get a sequential consistent view of current parts.
|
||||
|
0
tests/integration/test_drop_replica/__init__.py
Normal file
0
tests/integration/test_drop_replica/__init__.py
Normal file
@ -0,0 +1,21 @@
|
||||
<yandex>
|
||||
<remote_servers>
|
||||
<test_cluster>
|
||||
<shard>
|
||||
<internal_replication>true</internal_replication>
|
||||
<replica>
|
||||
<host>node_1_1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>node_1_2</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>node_1_3</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_cluster>
|
||||
</remote_servers>
|
||||
</yandex>
|
130
tests/integration/test_drop_replica/test.py
Normal file
130
tests/integration/test_drop_replica/test.py
Normal file
@ -0,0 +1,130 @@
|
||||
import time
|
||||
import pytest
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.cluster import ClickHouseKiller
|
||||
from helpers.test_tools import assert_eq_with_retry
|
||||
from helpers.network import PartitionManager
|
||||
|
||||
def fill_nodes(nodes, shard):
|
||||
for node in nodes:
|
||||
node.query(
|
||||
'''
|
||||
CREATE DATABASE test;
|
||||
|
||||
CREATE TABLE test.test_table(date Date, id UInt32)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0;
|
||||
'''.format(shard=shard, replica=node.name))
|
||||
|
||||
node.query(
|
||||
'''
|
||||
CREATE DATABASE test1;
|
||||
|
||||
CREATE TABLE test1.test_table(date Date, id UInt32)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test1/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0;
|
||||
'''.format(shard=shard, replica=node.name))
|
||||
|
||||
node.query(
|
||||
'''
|
||||
CREATE DATABASE test2;
|
||||
|
||||
CREATE TABLE test2.test_table(date Date, id UInt32)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test2/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0;
|
||||
'''.format(shard=shard, replica=node.name))
|
||||
|
||||
|
||||
node.query(
|
||||
'''
|
||||
CREATE DATABASE test3;
|
||||
|
||||
CREATE TABLE test3.test_table(date Date, id UInt32)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test3/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0;
|
||||
'''.format(shard=shard, replica=node.name))
|
||||
|
||||
node.query(
|
||||
'''
|
||||
CREATE DATABASE test4;
|
||||
|
||||
CREATE TABLE test4.test_table(date Date, id UInt32)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test4/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date) SETTINGS min_replicated_logs_to_keep=3, max_replicated_logs_to_keep=5, cleanup_delay_period=0, cleanup_delay_period_random_add=0;
|
||||
'''.format(shard=shard, replica=node.name))
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
node_1_1 = cluster.add_instance('node_1_1', with_zookeeper=True, main_configs=['configs/remote_servers.xml'])
|
||||
node_1_2 = cluster.add_instance('node_1_2', with_zookeeper=True, main_configs=['configs/remote_servers.xml'])
|
||||
node_1_3 = cluster.add_instance('node_1_3', with_zookeeper=True, main_configs=['configs/remote_servers.xml'])
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def start_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
fill_nodes([node_1_1, node_1_2], 1)
|
||||
|
||||
yield cluster
|
||||
|
||||
except Exception as ex:
|
||||
print ex
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
def test_drop_replica(start_cluster):
|
||||
for i in range(100):
|
||||
node_1_1.query("INSERT INTO test.test_table VALUES (1, {})".format(i))
|
||||
node_1_1.query("INSERT INTO test1.test_table VALUES (1, {})".format(i))
|
||||
node_1_1.query("INSERT INTO test2.test_table VALUES (1, {})".format(i))
|
||||
node_1_1.query("INSERT INTO test3.test_table VALUES (1, {})".format(i))
|
||||
node_1_1.query("INSERT INTO test4.test_table VALUES (1, {})".format(i))
|
||||
|
||||
zk = cluster.get_kazoo_client('zoo1')
|
||||
assert "can't drop local replica" in node_1_1.query_and_get_error("SYSTEM DROP REPLICA 'node_1_1'")
|
||||
assert "can't drop local replica" in node_1_1.query_and_get_error("SYSTEM DROP REPLICA 'node_1_1' FROM DATABASE test")
|
||||
assert "can't drop local replica" in node_1_1.query_and_get_error("SYSTEM DROP REPLICA 'node_1_1' FROM TABLE test.test_table")
|
||||
assert "it's active" in node_1_2.query_and_get_error("SYSTEM DROP REPLICA 'node_1_1'")
|
||||
assert "it's active" in node_1_2.query_and_get_error("SYSTEM DROP REPLICA 'node_1_1' FROM DATABASE test")
|
||||
assert "it's active" in node_1_2.query_and_get_error("SYSTEM DROP REPLICA 'node_1_1' FROM TABLE test.test_table")
|
||||
assert "it's active" in \
|
||||
node_1_3.query_and_get_error("SYSTEM DROP REPLICA 'node_1_1' FROM ZKPATH '/clickhouse/tables/test/{shard}/replicated/test_table'".format(shard=1))
|
||||
assert "There is a local table" in \
|
||||
node_1_2.query_and_get_error("SYSTEM DROP REPLICA 'node_1_1' FROM ZKPATH '/clickhouse/tables/test/{shard}/replicated/test_table'".format(shard=1))
|
||||
assert "There is a local table" in \
|
||||
node_1_1.query_and_get_error("SYSTEM DROP REPLICA 'node_1_1' FROM ZKPATH '/clickhouse/tables/test/{shard}/replicated/test_table'".format(shard=1))
|
||||
assert "does not look like a table path" in \
|
||||
node_1_3.query_and_get_error("SYSTEM DROP REPLICA 'node_1_1' FROM ZKPATH '/clickhouse/tables/test'")
|
||||
|
||||
with PartitionManager() as pm:
|
||||
## make node_1_1 dead
|
||||
pm.drop_instance_zk_connections(node_1_1)
|
||||
time.sleep(10)
|
||||
|
||||
assert "doesn't exist" in node_1_3.query_and_get_error("SYSTEM DROP REPLICA 'node_1_1' FROM TABLE test.test_table")
|
||||
|
||||
assert "doesn't exist" in node_1_3.query_and_get_error("SYSTEM DROP REPLICA 'node_1_1' FROM DATABASE test1")
|
||||
|
||||
node_1_3.query("SYSTEM DROP REPLICA 'node_1_1'")
|
||||
exists_replica_1_1 = zk.exists("/clickhouse/tables/test3/{shard}/replicated/test_table/replicas/{replica}".format(shard=1, replica='node_1_1'))
|
||||
assert (exists_replica_1_1 != None)
|
||||
|
||||
## If you want to drop a inactive/stale replicate table that does not have a local replica, you can following syntax(ZKPATH):
|
||||
node_1_3.query("SYSTEM DROP REPLICA 'node_1_1' FROM ZKPATH '/clickhouse/tables/test2/{shard}/replicated/test_table'".format(shard=1))
|
||||
exists_replica_1_1 = zk.exists("/clickhouse/tables/test2/{shard}/replicated/test_table/replicas/{replica}".format(shard=1, replica='node_1_1'))
|
||||
assert (exists_replica_1_1 == None)
|
||||
|
||||
node_1_2.query("SYSTEM DROP REPLICA 'node_1_1' FROM TABLE test.test_table")
|
||||
exists_replica_1_1 = zk.exists("/clickhouse/tables/test/{shard}/replicated/test_table/replicas/{replica}".format(shard=1, replica='node_1_1'))
|
||||
assert (exists_replica_1_1 == None)
|
||||
|
||||
node_1_2.query("SYSTEM DROP REPLICA 'node_1_1' FROM DATABASE test1")
|
||||
exists_replica_1_1 = zk.exists("/clickhouse/tables/test1/{shard}/replicated/test_table/replicas/{replica}".format(shard=1, replica='node_1_1'))
|
||||
assert (exists_replica_1_1 == None)
|
||||
|
||||
node_1_3.query("SYSTEM DROP REPLICA 'node_1_1' FROM ZKPATH '/clickhouse/tables/test3/{shard}/replicated/test_table'".format(shard=1))
|
||||
exists_replica_1_1 = zk.exists("/clickhouse/tables/test3/{shard}/replicated/test_table/replicas/{replica}".format(shard=1, replica='node_1_1'))
|
||||
assert (exists_replica_1_1 == None)
|
||||
|
||||
node_1_2.query("SYSTEM DROP REPLICA 'node_1_1'")
|
||||
exists_replica_1_1 = zk.exists("/clickhouse/tables/test4/{shard}/replicated/test_table/replicas/{replica}".format(shard=1, replica='node_1_1'))
|
||||
assert (exists_replica_1_1 == None)
|
@ -89,6 +89,7 @@ SYSTEM DISTRIBUTED SENDS ['SYSTEM STOP DISTRIBUTED SENDS','SYSTEM START DISTRIBU
|
||||
SYSTEM REPLICATED SENDS ['SYSTEM STOP REPLICATED SENDS','SYSTEM START REPLICATED SENDS','STOP_REPLICATED_SENDS','START REPLICATED SENDS'] TABLE SYSTEM SENDS
|
||||
SYSTEM SENDS ['SYSTEM STOP SENDS','SYSTEM START SENDS','STOP SENDS','START SENDS'] \N SYSTEM
|
||||
SYSTEM REPLICATION QUEUES ['SYSTEM STOP REPLICATION QUEUES','SYSTEM START REPLICATION QUEUES','STOP_REPLICATION_QUEUES','START REPLICATION QUEUES'] TABLE SYSTEM
|
||||
SYSTEM DROP REPLICA ['DROP REPLICA'] TABLE SYSTEM
|
||||
SYSTEM SYNC REPLICA ['SYNC REPLICA'] TABLE SYSTEM
|
||||
SYSTEM RESTART REPLICA ['RESTART REPLICA'] TABLE SYSTEM
|
||||
SYSTEM FLUSH DISTRIBUTED ['FLUSH DISTRIBUTED'] TABLE SYSTEM FLUSH
|
||||
|
Loading…
Reference in New Issue
Block a user