ClickHouse/src/Parsers/ParserSystemQuery.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

624 lines
20 KiB
C++
Raw Normal View History

#include <Parsers/ParserSystemQuery.h>
#include <Parsers/ASTSystemQuery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/parseDatabaseAndTableName.h>
2023-11-29 02:32:41 +00:00
#include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h>
2021-09-20 18:33:25 +00:00
#include <magic_enum.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int SUPPORT_IS_DISABLED;
}
[[nodiscard]] static bool parseQueryWithOnClusterAndMaybeTable(std::shared_ptr<ASTSystemQuery> & res, IParser::Pos & pos,
SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
Expected & expected, bool require_table, bool allow_string_literal)
{
/// Better form for user: SYSTEM <ACTION> table ON CLUSTER cluster
/// Query rewritten form + form while executing on cluster: SYSTEM <ACTION> ON CLUSTER cluster table
/// Need to support both
String cluster;
bool parsed_on_cluster = false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster, expected))
return false;
parsed_on_cluster = true;
}
bool parsed_table = false;
if (allow_string_literal)
{
ASTPtr ast;
if (ParserStringLiteral{}.parse(pos, ast, expected))
{
2021-10-12 18:11:00 +00:00
res->setTable(ast->as<ASTLiteral &>().value.safeGet<String>());
SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
parsed_table = true;
}
}
if (!parsed_table)
2021-11-16 08:58:28 +00:00
parsed_table = parseDatabaseAndTableAsAST(pos, expected, res->database, res->table);
SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
if (!parsed_table && require_table)
2022-01-30 05:46:11 +00:00
return false;
SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
if (!parsed_on_cluster && ParserKeyword{"ON"}.ignore(pos, expected))
if (!ASTQueryWithOnCluster::parse(pos, cluster, expected))
return false;
res->cluster = cluster;
2021-10-12 18:11:00 +00:00
if (res->database)
res->children.push_back(res->database);
if (res->table)
res->children.push_back(res->table);
SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
return true;
}
enum class SystemQueryTargetType
{
Model,
Function,
Disk,
};
[[nodiscard]] static bool parseQueryWithOnClusterAndTarget(std::shared_ptr<ASTSystemQuery> & res, IParser::Pos & pos, Expected & expected, SystemQueryTargetType target_type)
{
/// Better form for user: SYSTEM <ACTION> target_name ON CLUSTER cluster
/// Query rewritten form + form while executing on cluster: SYSTEM <ACTION> ON CLUSTER cluster target_name
/// Need to support both
String cluster;
bool parsed_on_cluster = false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster, expected))
return false;
parsed_on_cluster = true;
}
String target;
ASTPtr temporary_string_literal;
if (ParserStringLiteral{}.parse(pos, temporary_string_literal, expected))
{
target = temporary_string_literal->as<ASTLiteral &>().value.safeGet<String>();
}
else
{
ParserIdentifier identifier_parser;
ASTPtr identifier;
if (!identifier_parser.parse(pos, identifier, expected))
return false;
if (!tryGetIdentifierNameInto(identifier, target))
return false;
}
if (!parsed_on_cluster && ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster, expected))
return false;
}
res->cluster = cluster;
switch (target_type)
{
case SystemQueryTargetType::Model:
{
res->target_model = std::move(target);
break;
}
case SystemQueryTargetType::Function:
{
res->target_function = std::move(target);
break;
}
case SystemQueryTargetType::Disk:
{
res->disk = std::move(target);
break;
}
}
return true;
}
[[nodiscard]] static bool parseQueryWithOnCluster(std::shared_ptr<ASTSystemQuery> & res, IParser::Pos & pos,
2022-01-30 05:46:11 +00:00
Expected & expected)
{
String cluster_str;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
res->cluster = cluster_str;
return true;
}
[[nodiscard]] static bool parseDropReplica(std::shared_ptr<ASTSystemQuery> & res, IParser::Pos & pos, Expected & expected, bool database)
{
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
ASTPtr ast;
if (!ParserStringLiteral{}.parse(pos, ast, expected))
return false;
res->replica = ast->as<ASTLiteral &>().value.safeGet<String>();
2023-04-07 16:26:23 +00:00
if (ParserKeyword{"FROM SHARD"}.ignore(pos, expected))
{
if (!ParserStringLiteral{}.parse(pos, ast, expected))
return false;
res->shard = ast->as<ASTLiteral &>().value.safeGet<String>();
}
if (ParserKeyword{"FROM"}.ignore(pos, expected))
{
// way 1. parse replica database
// way 2. parse replica table
// way 3. parse replica zkpath
if (ParserKeyword{"DATABASE"}.ignore(pos, expected))
{
ParserIdentifier database_parser;
if (!database_parser.parse(pos, res->database, expected))
return false;
}
else if (!database && ParserKeyword{"TABLE"}.ignore(pos, expected))
{
parseDatabaseAndTableAsAST(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;
return true;
}
bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
2019-12-07 23:05:57 +00:00
if (!ParserKeyword{"SYSTEM"}.ignore(pos, expected))
return false;
using Type = ASTSystemQuery::Type;
auto res = std::make_shared<ASTSystemQuery>();
bool found = false;
2021-09-20 15:15:23 +00:00
for (const auto & type : magic_enum::enum_values<Type>())
{
2021-09-20 15:15:23 +00:00
if (ParserKeyword{ASTSystemQuery::typeToString(type)}.ignore(pos, expected))
{
2021-09-20 15:15:23 +00:00
res->type = type;
found = true;
break;
}
2021-09-20 18:33:25 +00:00
}
if (!found)
return false;
switch (res->type)
{
case Type::RELOAD_DICTIONARY:
{
SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
if (!parseQueryWithOnClusterAndMaybeTable(res, pos, expected, /* require table = */ true, /* allow_string_literal = */ true))
return false;
break;
}
case Type::RELOAD_MODEL:
{
if (!parseQueryWithOnClusterAndTarget(res, pos, expected, SystemQueryTargetType::Model))
return false;
break;
}
case Type::RELOAD_FUNCTION:
{
if (!parseQueryWithOnClusterAndTarget(res, pos, expected, SystemQueryTargetType::Function))
return false;
2021-04-16 09:56:40 +00:00
break;
}
2020-05-17 12:44:22 +00:00
case Type::DROP_REPLICA:
{
if (!parseDropReplica(res, pos, expected, /* database */ false))
return false;
break;
}
case Type::DROP_DATABASE_REPLICA:
{
if (!parseDropReplica(res, pos, expected, /* database */ true))
2020-05-17 12:44:22 +00:00
return false;
break;
}
case Type::ENABLE_FAILPOINT:
case Type::DISABLE_FAILPOINT:
{
ASTPtr ast;
if (ParserIdentifier{}.parse(pos, ast, expected))
res->fail_point_name = ast->as<ASTIdentifier &>().name();
else
return false;
break;
}
2020-05-17 12:44:22 +00:00
case Type::RESTART_REPLICA:
case Type::SYNC_REPLICA:
case Type::WAIT_LOADING_PARTS:
{
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
2021-11-16 08:58:28 +00:00
if (!parseDatabaseAndTableAsAST(pos, expected, res->database, res->table))
return false;
2023-03-27 23:39:36 +00:00
if (res->type == Type::SYNC_REPLICA)
{
if (ParserKeyword{"STRICT"}.ignore(pos, expected))
res->sync_replica_mode = SyncReplicaMode::STRICT;
else if (ParserKeyword{"LIGHTWEIGHT"}.ignore(pos, expected))
{
2024-01-05 15:17:15 +00:00
res->sync_replica_mode = SyncReplicaMode::LIGHTWEIGHT;
if (ParserKeyword{"FROM"}.ignore(pos, expected))
{
2024-01-05 15:17:15 +00:00
do
{
ASTPtr replica_ast;
if (!ParserStringLiteral{}.parse(pos, replica_ast, expected))
return false;
res->src_replicas.insert(replica_ast->as<ASTLiteral &>().value.safeGet<String>());
} while (ParserToken{TokenType::Comma}.ignore(pos, expected));
}
}
2023-03-27 23:39:36 +00:00
else if (ParserKeyword{"PULL"}.ignore(pos, expected))
res->sync_replica_mode = SyncReplicaMode::PULL;
}
break;
}
case Type::SYNC_DATABASE_REPLICA:
{
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
if (!parseDatabaseAsAST(pos, expected, res->database))
return false;
break;
}
case Type::RESTART_DISK:
{
if (!parseQueryWithOnClusterAndTarget(res, pos, expected, SystemQueryTargetType::Disk))
return false;
break;
}
SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
/// FLUSH DISTRIBUTED requires table
/// START/STOP DISTRIBUTED SENDS does not require table
case Type::STOP_DISTRIBUTED_SENDS:
case Type::START_DISTRIBUTED_SENDS:
SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
{
if (!parseQueryWithOnClusterAndMaybeTable(res, pos, expected, /* require table = */ false, /* allow_string_literal = */ false))
return false;
break;
}
case Type::FLUSH_DISTRIBUTED:
SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
case Type::RESTORE_REPLICA:
{
SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov <avtokmakov@yandex-team.ru>
2021-06-20 08:24:43 +00:00
if (!parseQueryWithOnClusterAndMaybeTable(res, pos, expected, /* require table = */ true, /* allow_string_literal = */ false))
return false;
break;
}
case Type::STOP_MERGES:
case Type::START_MERGES:
{
String storage_policy_str;
String volume_str;
2022-01-28 06:30:57 +00:00
auto parse_on_volume = [&]() -> bool
{
ASTPtr ast;
if (ParserIdentifier{}.parse(pos, ast, expected))
storage_policy_str = ast->as<ASTIdentifier &>().name();
else
return false;
if (!ParserToken{TokenType::Dot}.ignore(pos, expected))
return false;
if (ParserIdentifier{}.parse(pos, ast, expected))
volume_str = ast->as<ASTIdentifier &>().name();
else
return false;
2022-01-28 06:30:57 +00:00
return true;
};
if (ParserKeyword{"ON VOLUME"}.ignore(pos, expected))
{
if (!parse_on_volume())
return false;
}
else
{
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
2022-01-28 06:30:57 +00:00
if (ParserKeyword{"ON VOLUME"}.ignore(pos, expected))
{
if (!parse_on_volume())
return false;
}
}
2022-01-28 06:30:57 +00:00
res->storage_policy = storage_policy_str;
res->volume = volume_str;
if (res->volume.empty() && res->storage_policy.empty())
2021-11-16 08:58:28 +00:00
parseDatabaseAndTableAsAST(pos, expected, res->database, res->table);
break;
}
2019-08-01 15:36:12 +00:00
case Type::STOP_TTL_MERGES:
case Type::START_TTL_MERGES:
2019-09-03 14:50:49 +00:00
case Type::STOP_MOVES:
case Type::START_MOVES:
case Type::STOP_FETCHES:
case Type::START_FETCHES:
case Type::STOP_REPLICATED_SENDS:
2019-02-02 11:28:43 +00:00
case Type::START_REPLICATED_SENDS:
case Type::STOP_REPLICATION_QUEUES:
case Type::START_REPLICATION_QUEUES:
case Type::STOP_PULLING_REPLICATION_LOG:
case Type::START_PULLING_REPLICATION_LOG:
2023-09-20 16:10:00 +00:00
case Type::STOP_CLEANUP:
case Type::START_CLEANUP:
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
2021-11-16 08:58:28 +00:00
parseDatabaseAndTableAsAST(pos, expected, res->database, res->table);
break;
2023-02-12 19:17:55 +00:00
case Type::REFRESH_VIEW:
case Type::START_VIEW:
case Type::STOP_VIEW:
case Type::CANCEL_VIEW:
if (!parseDatabaseAndTableAsAST(pos, expected, res->database, res->table))
return false;
break;
case Type::START_VIEWS:
case Type::STOP_VIEWS:
break;
2023-11-29 02:32:41 +00:00
case Type::TEST_VIEW:
{
if (!parseDatabaseAndTableAsAST(pos, expected, res->database, res->table))
return false;
if (ParserKeyword{"UNSET FAKE TIME"}.ignore(pos, expected))
break;
if (!ParserKeyword{"SET FAKE TIME"}.ignore(pos, expected))
return false;
ASTPtr ast;
if (!ParserStringLiteral{}.parse(pos, ast, expected))
return false;
String time_str = ast->as<ASTLiteral &>().value.get<const String &>();
ReadBufferFromString buf(time_str);
time_t time;
readDateTimeText(time, buf);
res->fake_time_for_view = Int64(time);
break;
}
2021-01-07 19:19:33 +00:00
case Type::SUSPEND:
{
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
2021-01-07 19:19:33 +00:00
ASTPtr seconds;
if (!(ParserKeyword{"FOR"}.ignore(pos, expected)
&& ParserUnsignedInteger().parse(pos, seconds, expected)
&& ParserKeyword{"SECOND"}.ignore(pos, expected))) /// SECOND, not SECONDS to be consistent with INTERVAL parsing in SQL
{
return false;
}
res->seconds = seconds->as<ASTLiteral>()->value.get<UInt64>();
break;
}
2022-04-25 16:49:00 +00:00
case Type::DROP_FILESYSTEM_CACHE:
{
ParserLiteral path_parser;
ASTPtr ast;
if (path_parser.parse(pos, ast, expected))
{
2023-04-15 11:08:49 +00:00
res->filesystem_cache_name = ast->as<ASTLiteral>()->value.safeGet<String>();
if (ParserKeyword{"KEY"}.ignore(pos, expected) && ParserIdentifier().parse(pos, ast, expected))
{
res->key_to_drop = ast->as<ASTIdentifier>()->name();
if (ParserKeyword{"OFFSET"}.ignore(pos, expected) && ParserLiteral().parse(pos, ast, expected))
res->offset_to_drop = ast->as<ASTLiteral>()->value.safeGet<UInt64>();
}
}
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
2022-04-25 16:49:00 +00:00
break;
}
2023-06-29 14:41:36 +00:00
case Type::SYNC_FILESYSTEM_CACHE:
{
ParserLiteral path_parser;
ASTPtr ast;
if (path_parser.parse(pos, ast, expected))
res->filesystem_cache_name = ast->as<ASTLiteral>()->value.safeGet<String>();
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
break;
}
case Type::DROP_DISK_METADATA_CACHE:
{
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Not implemented");
}
case Type::DROP_SCHEMA_CACHE:
{
if (ParserKeyword{"FOR"}.ignore(pos, expected))
{
if (ParserKeyword{"FILE"}.ignore(pos, expected))
res->schema_cache_storage = "FILE";
else if (ParserKeyword{"S3"}.ignore(pos, expected))
res->schema_cache_storage = "S3";
else if (ParserKeyword{"HDFS"}.ignore(pos, expected))
res->schema_cache_storage = "HDFS";
else if (ParserKeyword{"URL"}.ignore(pos, expected))
res->schema_cache_storage = "URL";
else if (ParserKeyword{"AZURE"}.ignore(pos, expected))
res->schema_cache_storage = "AZURE";
else
return false;
}
break;
}
case Type::DROP_FORMAT_SCHEMA_CACHE:
{
if (ParserKeyword{"FOR"}.ignore(pos, expected))
{
if (ParserKeyword{"Protobuf"}.ignore(pos, expected))
res->schema_cache_format = "Protobuf";
else
return false;
}
break;
}
2022-06-08 12:09:59 +00:00
case Type::UNFREEZE:
{
ASTPtr ast;
if (ParserKeyword{"WITH NAME"}.ignore(pos, expected) && ParserStringLiteral{}.parse(pos, ast, expected))
{
res->backup_name = ast->as<ASTLiteral &>().value.get<const String &>();
}
else
{
return false;
}
break;
}
case Type::START_LISTEN:
case Type::STOP_LISTEN:
{
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
auto parse_server_type = [&](ServerType::Type & type, std::string & custom_name) -> bool
{
type = ServerType::Type::END;
custom_name = "";
for (const auto & cur_type : magic_enum::enum_values<ServerType::Type>())
{
if (ParserKeyword{ServerType::serverTypeToString(cur_type)}.ignore(pos, expected))
{
type = cur_type;
break;
}
}
if (type == ServerType::Type::END)
return false;
if (type == ServerType::CUSTOM)
{
ASTPtr ast;
if (!ParserStringLiteral{}.parse(pos, ast, expected))
return false;
custom_name = ast->as<ASTLiteral &>().value.get<const String &>();
}
return true;
};
ServerType::Type base_type;
std::string base_custom_name;
ServerType::Types exclude_type;
ServerType::CustomNames exclude_custom_names;
if (!parse_server_type(base_type, base_custom_name))
return false;
if (ParserKeyword{"EXCEPT"}.ignore(pos, expected))
{
if (base_type != ServerType::Type::QUERIES_ALL &&
base_type != ServerType::Type::QUERIES_DEFAULT &&
base_type != ServerType::Type::QUERIES_CUSTOM)
return false;
ServerType::Type current_type;
std::string current_custom_name;
while (true)
{
if (!exclude_type.empty() && !ParserToken(TokenType::Comma).ignore(pos, expected))
break;
if (!parse_server_type(current_type, current_custom_name))
return false;
exclude_type.insert(current_type);
if (current_type == ServerType::Type::CUSTOM)
exclude_custom_names.insert(current_custom_name);
}
}
res->server_type = ServerType(base_type, base_custom_name, exclude_type, exclude_custom_names);
break;
}
default:
{
if (!parseQueryWithOnCluster(res, pos, expected))
return false;
break;
}
}
2021-10-12 18:11:00 +00:00
if (res->database)
res->children.push_back(res->database);
if (res->table)
res->children.push_back(res->table);
node = std::move(res);
return true;
}
}