2017-08-06 20:26:23 +00:00
|
|
|
#include <Interpreters/InterpreterSystemQuery.h>
|
2018-04-19 13:56:14 +00:00
|
|
|
#include <Common/DNSResolver.h>
|
2018-05-21 13:49:54 +00:00
|
|
|
#include <Common/ActionLock.h>
|
|
|
|
#include <Common/typeid_cast.h>
|
2018-06-05 19:46:49 +00:00
|
|
|
#include <Common/getNumberOfPhysicalCPUCores.h>
|
2019-01-11 19:12:36 +00:00
|
|
|
#include <Common/ThreadPool.h>
|
2019-10-25 19:07:47 +00:00
|
|
|
#include <Common/escapeForFileName.h>
|
2017-08-07 17:01:04 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2020-02-10 13:10:17 +00:00
|
|
|
#include <Interpreters/DatabaseCatalog.h>
|
2019-09-26 10:41:33 +00:00
|
|
|
#include <Interpreters/ExternalDictionariesLoader.h>
|
2017-08-24 18:19:06 +00:00
|
|
|
#include <Interpreters/EmbeddedDictionaries.h>
|
2018-05-21 13:49:54 +00:00
|
|
|
#include <Interpreters/ActionLocksManager.h>
|
|
|
|
#include <Interpreters/InterpreterDropQuery.h>
|
|
|
|
#include <Interpreters/InterpreterCreateQuery.h>
|
2020-03-30 23:36:23 +00:00
|
|
|
#include <Interpreters/InterpreterRenameQuery.h>
|
2018-06-13 19:01:07 +00:00
|
|
|
#include <Interpreters/QueryLog.h>
|
2019-12-19 07:54:43 +00:00
|
|
|
#include <Interpreters/DDLWorker.h>
|
2018-06-13 19:01:07 +00:00
|
|
|
#include <Interpreters/PartLog.h>
|
|
|
|
#include <Interpreters/QueryThreadLog.h>
|
2019-02-03 21:30:45 +00:00
|
|
|
#include <Interpreters/TraceLog.h>
|
2019-07-31 14:03:23 +00:00
|
|
|
#include <Interpreters/TextLog.h>
|
2019-08-13 14:31:46 +00:00
|
|
|
#include <Interpreters/MetricLog.h>
|
2020-06-10 19:17:30 +00:00
|
|
|
#include <Interpreters/AsynchronousMetricLog.h>
|
2020-03-07 17:37:38 +00:00
|
|
|
#include <Access/ContextAccess.h>
|
2020-04-30 22:29:47 +00:00
|
|
|
#include <Access/AllowedClientHosts.h>
|
2018-05-21 13:49:54 +00:00
|
|
|
#include <Databases/IDatabase.h>
|
2019-04-08 05:13:16 +00:00
|
|
|
#include <Storages/StorageDistributed.h>
|
2018-05-21 13:49:54 +00:00
|
|
|
#include <Storages/StorageReplicatedMergeTree.h>
|
|
|
|
#include <Storages/StorageFactory.h>
|
2017-08-06 20:26:23 +00:00
|
|
|
#include <Parsers/ASTSystemQuery.h>
|
2018-05-21 13:49:54 +00:00
|
|
|
#include <Parsers/ASTDropQuery.h>
|
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
2017-08-06 20:26:23 +00:00
|
|
|
#include <csignal>
|
2019-01-09 15:44:20 +00:00
|
|
|
#include <algorithm>
|
2017-08-06 20:26:23 +00:00
|
|
|
|
2020-04-16 12:31:57 +00:00
|
|
|
#if !defined(ARCADIA_BUILD)
|
|
|
|
# include "config_core.h"
|
|
|
|
#endif
|
|
|
|
|
2017-08-04 15:54:00 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2017-08-06 20:26:23 +00:00
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2020-02-25 18:02:41 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2017-08-06 20:26:23 +00:00
|
|
|
extern const int BAD_ARGUMENTS;
|
|
|
|
extern const int CANNOT_KILL;
|
2017-08-07 17:01:04 +00:00
|
|
|
extern const int NOT_IMPLEMENTED;
|
2019-09-19 11:04:57 +00:00
|
|
|
extern const int TIMEOUT_EXCEEDED;
|
2020-06-23 12:01:51 +00:00
|
|
|
extern const int TABLE_WAS_NOT_DROPPED;
|
2017-08-06 20:26:23 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
namespace ActionLocks
|
|
|
|
{
|
|
|
|
extern StorageActionBlockType PartsMerge;
|
|
|
|
extern StorageActionBlockType PartsFetch;
|
|
|
|
extern StorageActionBlockType PartsSend;
|
|
|
|
extern StorageActionBlockType ReplicationQueue;
|
2019-04-22 15:11:16 +00:00
|
|
|
extern StorageActionBlockType DistributedSend;
|
2019-08-01 15:36:12 +00:00
|
|
|
extern StorageActionBlockType PartsTTLMerge;
|
2019-09-03 14:50:49 +00:00
|
|
|
extern StorageActionBlockType PartsMove;
|
2018-05-21 13:49:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-08-31 12:55:19 +00:00
|
|
|
namespace
|
|
|
|
{
|
2017-08-04 15:54:00 +00:00
|
|
|
|
2017-08-24 18:19:06 +00:00
|
|
|
ExecutionStatus getOverallExecutionStatusOfCommands()
|
|
|
|
{
|
|
|
|
return ExecutionStatus(0);
|
|
|
|
}
|
|
|
|
|
2019-01-22 19:56:53 +00:00
|
|
|
/// Consequently tries to execute all commands and generates final exception message for failed commands
|
2017-08-24 18:19:06 +00:00
|
|
|
template <typename Callable, typename ... Callables>
|
|
|
|
ExecutionStatus getOverallExecutionStatusOfCommands(Callable && command, Callables && ... commands)
|
|
|
|
{
|
|
|
|
ExecutionStatus status_head(0);
|
|
|
|
try
|
|
|
|
{
|
|
|
|
command();
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
status_head = ExecutionStatus::fromCurrentException();
|
|
|
|
}
|
|
|
|
|
|
|
|
ExecutionStatus status_tail = getOverallExecutionStatusOfCommands(std::forward<Callables>(commands)...);
|
|
|
|
|
|
|
|
auto res_status = status_head.code != 0 ? status_head.code : status_tail.code;
|
|
|
|
auto res_message = status_head.message + (status_tail.message.empty() ? "" : ("\n" + status_tail.message));
|
|
|
|
|
|
|
|
return ExecutionStatus(res_status, res_message);
|
|
|
|
}
|
|
|
|
|
2018-06-13 19:01:07 +00:00
|
|
|
/// Consequently tries to execute all commands and throws exception with info about failed commands
|
|
|
|
template <typename ... Callables>
|
|
|
|
void executeCommandsAndThrowIfError(Callables && ... commands)
|
|
|
|
{
|
|
|
|
auto status = getOverallExecutionStatusOfCommands(std::forward<Callables>(commands)...);
|
|
|
|
if (status.code != 0)
|
|
|
|
throw Exception(status.message, status.code);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-01-24 16:20:36 +00:00
|
|
|
AccessType getRequiredAccessType(StorageActionBlockType action_type)
|
|
|
|
{
|
|
|
|
if (action_type == ActionLocks::PartsMerge)
|
2020-04-03 11:54:50 +00:00
|
|
|
return AccessType::SYSTEM_MERGES;
|
2020-01-24 16:20:36 +00:00
|
|
|
else if (action_type == ActionLocks::PartsFetch)
|
2020-04-03 11:54:50 +00:00
|
|
|
return AccessType::SYSTEM_FETCHES;
|
2020-01-24 16:20:36 +00:00
|
|
|
else if (action_type == ActionLocks::PartsSend)
|
2020-04-03 11:54:50 +00:00
|
|
|
return AccessType::SYSTEM_REPLICATED_SENDS;
|
2020-01-24 16:20:36 +00:00
|
|
|
else if (action_type == ActionLocks::ReplicationQueue)
|
2020-04-03 11:54:50 +00:00
|
|
|
return AccessType::SYSTEM_REPLICATION_QUEUES;
|
2020-01-24 16:20:36 +00:00
|
|
|
else if (action_type == ActionLocks::DistributedSend)
|
2020-04-03 11:54:50 +00:00
|
|
|
return AccessType::SYSTEM_DISTRIBUTED_SENDS;
|
2020-01-24 16:20:36 +00:00
|
|
|
else if (action_type == ActionLocks::PartsTTLMerge)
|
2020-04-03 11:54:50 +00:00
|
|
|
return AccessType::SYSTEM_TTL_MERGES;
|
2020-01-24 16:20:36 +00:00
|
|
|
else if (action_type == ActionLocks::PartsMove)
|
2020-04-03 11:54:50 +00:00
|
|
|
return AccessType::SYSTEM_MOVES;
|
2020-01-24 16:20:36 +00:00
|
|
|
else
|
|
|
|
throw Exception("Unknown action type: " + std::to_string(action_type), ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
|
|
|
|
2020-03-04 20:29:52 +00:00
|
|
|
}
|
2020-01-24 16:20:36 +00:00
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
/// Implements SYSTEM [START|STOP] <something action from ActionLocks>
|
2020-03-04 20:29:52 +00:00
|
|
|
void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type, bool start)
|
2018-05-21 13:49:54 +00:00
|
|
|
{
|
|
|
|
auto manager = context.getActionLocksManager();
|
|
|
|
manager->cleanExpired();
|
|
|
|
|
2020-03-04 20:29:52 +00:00
|
|
|
if (table_id)
|
2018-05-21 13:49:54 +00:00
|
|
|
{
|
2020-03-04 20:29:52 +00:00
|
|
|
context.checkAccess(getRequiredAccessType(action_type), table_id);
|
2018-05-21 13:49:54 +00:00
|
|
|
if (start)
|
2020-03-04 20:29:52 +00:00
|
|
|
manager->remove(table_id, action_type);
|
2018-05-21 13:49:54 +00:00
|
|
|
else
|
2020-03-04 20:29:52 +00:00
|
|
|
manager->add(table_id, action_type);
|
2018-05-21 13:49:54 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2020-03-07 17:37:38 +00:00
|
|
|
auto access = context.getAccess();
|
2020-02-10 13:10:17 +00:00
|
|
|
for (auto & elem : DatabaseCatalog::instance().getDatabases())
|
2020-01-24 16:20:36 +00:00
|
|
|
{
|
2020-05-28 20:10:45 +00:00
|
|
|
for (auto iterator = elem.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
2020-01-24 16:20:36 +00:00
|
|
|
{
|
2020-06-02 01:53:22 +00:00
|
|
|
StoragePtr table = iterator->table();
|
|
|
|
if (!table)
|
|
|
|
continue;
|
|
|
|
|
2020-06-20 22:44:52 +00:00
|
|
|
if (!access->isGranted(getRequiredAccessType(action_type), elem.first, iterator->name()))
|
|
|
|
{
|
|
|
|
LOG_INFO(
|
|
|
|
log,
|
|
|
|
"Access {} denied, skipping {}.{}",
|
|
|
|
toString(getRequiredAccessType(action_type)),
|
|
|
|
elem.first,
|
|
|
|
iterator->name());
|
2020-03-07 17:37:38 +00:00
|
|
|
continue;
|
2020-06-20 22:44:52 +00:00
|
|
|
}
|
2020-06-02 01:53:22 +00:00
|
|
|
|
2020-08-21 15:44:29 +00:00
|
|
|
if (start)
|
2020-06-02 01:53:22 +00:00
|
|
|
manager->remove(table, action_type);
|
2020-03-07 17:37:38 +00:00
|
|
|
else
|
2020-06-02 01:53:22 +00:00
|
|
|
manager->add(table, action_type);
|
2020-01-24 16:20:36 +00:00
|
|
|
}
|
|
|
|
}
|
2018-05-21 13:49:54 +00:00
|
|
|
}
|
|
|
|
}
|
2020-01-24 16:20:36 +00:00
|
|
|
|
2020-03-04 20:29:52 +00:00
|
|
|
|
2017-08-31 12:55:19 +00:00
|
|
|
InterpreterSystemQuery::InterpreterSystemQuery(const ASTPtr & query_ptr_, Context & context_)
|
2019-11-11 01:11:32 +00:00
|
|
|
: query_ptr(query_ptr_->clone()), context(context_), log(&Poco::Logger::get("InterpreterSystemQuery"))
|
|
|
|
{
|
|
|
|
}
|
2017-08-31 12:55:19 +00:00
|
|
|
|
2017-08-06 20:26:23 +00:00
|
|
|
|
2017-08-04 15:54:00 +00:00
|
|
|
BlockIO InterpreterSystemQuery::execute()
|
|
|
|
{
|
2019-03-15 16:14:13 +00:00
|
|
|
auto & query = query_ptr->as<ASTSystemQuery &>();
|
2017-08-06 20:26:23 +00:00
|
|
|
|
2019-12-19 07:54:43 +00:00
|
|
|
if (!query.cluster.empty())
|
2020-01-24 16:20:36 +00:00
|
|
|
return executeDDLQueryOnCluster(query_ptr, context, getRequiredAccessForDDLOnCluster());
|
2019-12-19 07:54:43 +00:00
|
|
|
|
2017-08-06 20:26:23 +00:00
|
|
|
using Type = ASTSystemQuery::Type;
|
|
|
|
|
2018-03-26 14:12:07 +00:00
|
|
|
/// Use global context with fresh system profile settings
|
|
|
|
Context system_context = context.getGlobalContext();
|
|
|
|
system_context.setSetting("profile", context.getSystemProfileName());
|
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
/// Make canonical query for simpler processing
|
2020-03-04 20:29:52 +00:00
|
|
|
if (!query.table.empty())
|
2020-06-05 07:03:51 +00:00
|
|
|
table_id = context.resolveStorageID(StorageID(query.database, query.table), Context::ResolveOrdinary);
|
2018-05-21 13:49:54 +00:00
|
|
|
|
2019-12-23 03:48:34 +00:00
|
|
|
if (!query.target_dictionary.empty() && !query.database.empty())
|
|
|
|
query.target_dictionary = query.database + "." + query.target_dictionary;
|
2019-12-15 13:48:11 +00:00
|
|
|
|
2017-08-06 20:26:23 +00:00
|
|
|
switch (query.type)
|
|
|
|
{
|
|
|
|
case Type::SHUTDOWN:
|
2020-04-03 11:54:50 +00:00
|
|
|
context.checkAccess(AccessType::SYSTEM_SHUTDOWN);
|
2017-08-06 20:26:23 +00:00
|
|
|
if (kill(0, SIGTERM))
|
2017-08-31 12:55:19 +00:00
|
|
|
throwFromErrno("System call kill(0, SIGTERM) failed", ErrorCodes::CANNOT_KILL);
|
2017-08-06 20:26:23 +00:00
|
|
|
break;
|
|
|
|
case Type::KILL:
|
2020-04-03 11:54:50 +00:00
|
|
|
context.checkAccess(AccessType::SYSTEM_SHUTDOWN);
|
2017-08-06 20:26:23 +00:00
|
|
|
if (kill(0, SIGKILL))
|
2017-08-31 12:55:19 +00:00
|
|
|
throwFromErrno("System call kill(0, SIGKILL) failed", ErrorCodes::CANNOT_KILL);
|
2017-08-06 20:26:23 +00:00
|
|
|
break;
|
|
|
|
case Type::DROP_DNS_CACHE:
|
2020-04-03 11:54:50 +00:00
|
|
|
context.checkAccess(AccessType::SYSTEM_DROP_DNS_CACHE);
|
2018-04-19 13:56:14 +00:00
|
|
|
DNSResolver::instance().dropCache();
|
2018-01-15 14:13:19 +00:00
|
|
|
/// Reinitialize clusters to update their resolved_addresses
|
2018-03-26 14:12:07 +00:00
|
|
|
system_context.reloadClusterConfig();
|
2017-08-06 20:26:23 +00:00
|
|
|
break;
|
|
|
|
case Type::DROP_MARK_CACHE:
|
2020-04-03 11:54:50 +00:00
|
|
|
context.checkAccess(AccessType::SYSTEM_DROP_MARK_CACHE);
|
2018-03-26 14:12:07 +00:00
|
|
|
system_context.dropMarkCache();
|
2017-08-06 20:26:23 +00:00
|
|
|
break;
|
|
|
|
case Type::DROP_UNCOMPRESSED_CACHE:
|
2020-04-03 11:54:50 +00:00
|
|
|
context.checkAccess(AccessType::SYSTEM_DROP_UNCOMPRESSED_CACHE);
|
2018-03-26 14:12:07 +00:00
|
|
|
system_context.dropUncompressedCache();
|
2017-08-06 20:26:23 +00:00
|
|
|
break;
|
2018-08-30 16:31:20 +00:00
|
|
|
#if USE_EMBEDDED_COMPILER
|
2018-09-05 11:37:41 +00:00
|
|
|
case Type::DROP_COMPILED_EXPRESSION_CACHE:
|
2020-04-03 11:54:50 +00:00
|
|
|
context.checkAccess(AccessType::SYSTEM_DROP_COMPILED_EXPRESSION_CACHE);
|
2018-09-05 11:37:41 +00:00
|
|
|
system_context.dropCompiledExpressionCache();
|
2018-08-30 16:31:20 +00:00
|
|
|
break;
|
|
|
|
#endif
|
2017-08-24 18:19:06 +00:00
|
|
|
case Type::RELOAD_DICTIONARY:
|
2020-04-03 11:54:50 +00:00
|
|
|
context.checkAccess(AccessType::SYSTEM_RELOAD_DICTIONARY);
|
2020-07-15 19:25:31 +00:00
|
|
|
system_context.getExternalDictionariesLoader().loadOrReload(
|
|
|
|
DatabaseCatalog::instance().resolveDictionaryName(query.target_dictionary));
|
2020-03-03 08:32:58 +00:00
|
|
|
ExternalDictionariesLoader::resetAll();
|
2017-08-24 18:19:06 +00:00
|
|
|
break;
|
|
|
|
case Type::RELOAD_DICTIONARIES:
|
2020-04-03 11:54:50 +00:00
|
|
|
context.checkAccess(AccessType::SYSTEM_RELOAD_DICTIONARY);
|
2018-06-13 19:01:07 +00:00
|
|
|
executeCommandsAndThrowIfError(
|
2019-12-12 18:33:43 +00:00
|
|
|
[&] () { system_context.getExternalDictionariesLoader().reloadAllTriedToLoad(); },
|
2018-06-13 19:01:07 +00:00
|
|
|
[&] () { system_context.getEmbeddedDictionaries().reload(); }
|
2017-08-24 18:19:06 +00:00
|
|
|
);
|
2020-03-03 08:32:58 +00:00
|
|
|
ExternalDictionariesLoader::resetAll();
|
2017-08-24 18:19:06 +00:00
|
|
|
break;
|
2018-03-26 14:12:07 +00:00
|
|
|
case Type::RELOAD_EMBEDDED_DICTIONARIES:
|
2020-04-03 11:54:50 +00:00
|
|
|
context.checkAccess(AccessType::SYSTEM_RELOAD_EMBEDDED_DICTIONARIES);
|
2018-03-26 14:12:07 +00:00
|
|
|
system_context.getEmbeddedDictionaries().reload();
|
|
|
|
break;
|
2018-03-13 10:41:47 +00:00
|
|
|
case Type::RELOAD_CONFIG:
|
2020-04-03 11:54:50 +00:00
|
|
|
context.checkAccess(AccessType::SYSTEM_RELOAD_CONFIG);
|
2018-03-26 14:12:07 +00:00
|
|
|
system_context.reloadConfig();
|
2018-03-13 10:41:47 +00:00
|
|
|
break;
|
2017-08-07 17:01:04 +00:00
|
|
|
case Type::STOP_MERGES:
|
2020-03-04 20:29:52 +00:00
|
|
|
startStopAction(ActionLocks::PartsMerge, false);
|
2018-05-21 13:49:54 +00:00
|
|
|
break;
|
2017-08-07 17:01:04 +00:00
|
|
|
case Type::START_MERGES:
|
2020-03-04 20:29:52 +00:00
|
|
|
startStopAction(ActionLocks::PartsMerge, true);
|
2018-05-21 13:49:54 +00:00
|
|
|
break;
|
2019-08-01 15:36:12 +00:00
|
|
|
case Type::STOP_TTL_MERGES:
|
2020-03-04 20:29:52 +00:00
|
|
|
startStopAction(ActionLocks::PartsTTLMerge, false);
|
2019-08-01 15:36:12 +00:00
|
|
|
break;
|
|
|
|
case Type::START_TTL_MERGES:
|
2020-03-04 20:29:52 +00:00
|
|
|
startStopAction(ActionLocks::PartsTTLMerge, true);
|
2019-08-01 15:36:12 +00:00
|
|
|
break;
|
2019-09-03 14:50:49 +00:00
|
|
|
case Type::STOP_MOVES:
|
2020-03-04 20:29:52 +00:00
|
|
|
startStopAction(ActionLocks::PartsMove, false);
|
2019-09-03 14:50:49 +00:00
|
|
|
break;
|
|
|
|
case Type::START_MOVES:
|
2020-03-04 20:29:52 +00:00
|
|
|
startStopAction(ActionLocks::PartsMove, true);
|
2019-09-03 14:50:49 +00:00
|
|
|
break;
|
2018-05-21 13:49:54 +00:00
|
|
|
case Type::STOP_FETCHES:
|
2020-03-04 20:29:52 +00:00
|
|
|
startStopAction(ActionLocks::PartsFetch, false);
|
2018-05-21 13:49:54 +00:00
|
|
|
break;
|
|
|
|
case Type::START_FETCHES:
|
2020-03-04 20:29:52 +00:00
|
|
|
startStopAction(ActionLocks::PartsFetch, true);
|
2018-05-21 13:49:54 +00:00
|
|
|
break;
|
|
|
|
case Type::STOP_REPLICATED_SENDS:
|
2020-03-04 20:29:52 +00:00
|
|
|
startStopAction(ActionLocks::PartsSend, false);
|
2018-05-21 13:49:54 +00:00
|
|
|
break;
|
2019-02-02 11:28:43 +00:00
|
|
|
case Type::START_REPLICATED_SENDS:
|
2020-03-04 20:29:52 +00:00
|
|
|
startStopAction(ActionLocks::PartsSend, true);
|
2018-05-21 13:49:54 +00:00
|
|
|
break;
|
2017-08-07 17:01:04 +00:00
|
|
|
case Type::STOP_REPLICATION_QUEUES:
|
2020-03-04 20:29:52 +00:00
|
|
|
startStopAction(ActionLocks::ReplicationQueue, false);
|
2018-05-21 13:49:54 +00:00
|
|
|
break;
|
2017-08-07 17:01:04 +00:00
|
|
|
case Type::START_REPLICATION_QUEUES:
|
2020-03-04 20:29:52 +00:00
|
|
|
startStopAction(ActionLocks::ReplicationQueue, true);
|
2018-05-21 13:49:54 +00:00
|
|
|
break;
|
2019-04-22 15:11:16 +00:00
|
|
|
case Type::STOP_DISTRIBUTED_SENDS:
|
2020-03-04 20:29:52 +00:00
|
|
|
startStopAction(ActionLocks::DistributedSend, false);
|
2019-04-22 15:11:16 +00:00
|
|
|
break;
|
|
|
|
case Type::START_DISTRIBUTED_SENDS:
|
2020-03-04 20:29:52 +00:00
|
|
|
startStopAction(ActionLocks::DistributedSend, true);
|
2019-04-22 15:11:16 +00:00
|
|
|
break;
|
2020-05-17 12:44:22 +00:00
|
|
|
case Type::DROP_REPLICA:
|
|
|
|
dropReplica(query);
|
|
|
|
break;
|
2018-05-21 13:49:54 +00:00
|
|
|
case Type::SYNC_REPLICA:
|
|
|
|
syncReplica(query);
|
|
|
|
break;
|
2019-05-10 04:19:02 +00:00
|
|
|
case Type::FLUSH_DISTRIBUTED:
|
|
|
|
flushDistributed(query);
|
2019-04-22 15:11:16 +00:00
|
|
|
break;
|
2018-05-21 13:49:54 +00:00
|
|
|
case Type::RESTART_REPLICAS:
|
|
|
|
restartReplicas(system_context);
|
|
|
|
break;
|
|
|
|
case Type::RESTART_REPLICA:
|
2020-03-04 20:29:52 +00:00
|
|
|
if (!tryRestartReplica(table_id, system_context))
|
2019-12-19 09:27:12 +00:00
|
|
|
throw Exception("There is no " + query.database + "." + query.table + " replicated table",
|
2018-05-21 13:49:54 +00:00
|
|
|
ErrorCodes::BAD_ARGUMENTS);
|
|
|
|
break;
|
2018-10-09 10:05:27 +00:00
|
|
|
case Type::FLUSH_LOGS:
|
2020-04-03 11:54:50 +00:00
|
|
|
context.checkAccess(AccessType::SYSTEM_FLUSH_LOGS);
|
2018-06-13 19:01:07 +00:00
|
|
|
executeCommandsAndThrowIfError(
|
2020-06-09 18:11:08 +00:00
|
|
|
[&] () { if (auto query_log = context.getQueryLog()) query_log->flush(true); },
|
|
|
|
[&] () { if (auto part_log = context.getPartLog("")) part_log->flush(true); },
|
|
|
|
[&] () { if (auto query_thread_log = context.getQueryThreadLog()) query_thread_log->flush(true); },
|
|
|
|
[&] () { if (auto trace_log = context.getTraceLog()) trace_log->flush(true); },
|
|
|
|
[&] () { if (auto text_log = context.getTextLog()) text_log->flush(true); },
|
|
|
|
[&] () { if (auto metric_log = context.getMetricLog()) metric_log->flush(true); },
|
|
|
|
[&] () { if (auto asynchronous_metric_log = context.getAsynchronousMetricLog()) asynchronous_metric_log->flush(true); }
|
2018-06-13 19:01:07 +00:00
|
|
|
);
|
|
|
|
break;
|
2018-05-21 13:49:54 +00:00
|
|
|
case Type::STOP_LISTEN_QUERIES:
|
|
|
|
case Type::START_LISTEN_QUERIES:
|
2017-08-07 17:01:04 +00:00
|
|
|
throw Exception(String(ASTSystemQuery::typeToString(query.type)) + " is not supported yet", ErrorCodes::NOT_IMPLEMENTED);
|
2017-08-06 20:26:23 +00:00
|
|
|
default:
|
|
|
|
throw Exception("Unknown type of SYSTEM query", ErrorCodes::BAD_ARGUMENTS);
|
|
|
|
}
|
|
|
|
|
2017-08-04 15:54:00 +00:00
|
|
|
return BlockIO();
|
|
|
|
}
|
|
|
|
|
2017-08-06 20:26:23 +00:00
|
|
|
|
2020-03-30 23:36:23 +00:00
|
|
|
StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, Context & system_context, bool need_ddl_guard)
|
2018-05-21 13:49:54 +00:00
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
context.checkAccess(AccessType::SYSTEM_RESTART_REPLICA, replica);
|
2020-01-24 16:20:36 +00:00
|
|
|
|
2020-03-30 23:36:23 +00:00
|
|
|
auto table_ddl_guard = need_ddl_guard ? DatabaseCatalog::instance().getDDLGuard(replica.getDatabaseName(), replica.getTableName()) : nullptr;
|
2020-05-28 23:01:18 +00:00
|
|
|
auto [database, table] = DatabaseCatalog::instance().tryGetDatabaseAndTable(replica, context);
|
2018-05-21 13:49:54 +00:00
|
|
|
ASTPtr create_ast;
|
|
|
|
|
|
|
|
/// Detach actions
|
2020-03-04 20:29:52 +00:00
|
|
|
if (!table || !dynamic_cast<const StorageReplicatedMergeTree *>(table.get()))
|
|
|
|
return nullptr;
|
2018-05-21 13:49:54 +00:00
|
|
|
|
2020-03-04 20:29:52 +00:00
|
|
|
table->shutdown();
|
|
|
|
{
|
2018-05-21 13:49:54 +00:00
|
|
|
/// If table was already dropped by anyone, an exception will be thrown
|
2020-04-09 18:10:27 +00:00
|
|
|
auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
2020-05-28 20:10:45 +00:00
|
|
|
create_ast = database->getCreateTableQuery(replica.table_name, context);
|
2018-05-21 13:49:54 +00:00
|
|
|
|
2020-03-04 20:29:52 +00:00
|
|
|
database->detachTable(replica.table_name);
|
2018-05-21 13:49:54 +00:00
|
|
|
}
|
2020-03-04 20:29:52 +00:00
|
|
|
table.reset();
|
2018-05-21 13:49:54 +00:00
|
|
|
|
|
|
|
/// Attach actions
|
2020-03-04 20:29:52 +00:00
|
|
|
/// getCreateTableQuery must return canonical CREATE query representation, there are no need for AST postprocessing
|
|
|
|
auto & create = create_ast->as<ASTCreateQuery &>();
|
|
|
|
create.attach = true;
|
2018-05-21 13:49:54 +00:00
|
|
|
|
2020-05-04 00:11:49 +00:00
|
|
|
auto columns = InterpreterCreateQuery::getColumnsDescription(*create.columns_list->columns, system_context, false);
|
2020-03-04 20:29:52 +00:00
|
|
|
auto constraints = InterpreterCreateQuery::getConstraintsDescription(create.columns_list->constraints);
|
2020-04-02 01:00:21 +00:00
|
|
|
auto data_path = database->getTableDataPath(create);
|
2018-05-21 13:49:54 +00:00
|
|
|
|
2020-03-04 20:29:52 +00:00
|
|
|
table = StorageFactory::instance().get(create,
|
2020-04-02 01:00:21 +00:00
|
|
|
data_path,
|
2020-03-04 20:29:52 +00:00
|
|
|
system_context,
|
|
|
|
system_context.getGlobalContext(),
|
|
|
|
columns,
|
|
|
|
constraints,
|
|
|
|
false);
|
2018-05-21 13:49:54 +00:00
|
|
|
|
2020-04-02 01:00:21 +00:00
|
|
|
database->attachTable(replica.table_name, table, data_path);
|
2018-05-21 13:49:54 +00:00
|
|
|
|
2020-03-04 20:29:52 +00:00
|
|
|
table->startup();
|
|
|
|
return table;
|
2018-05-21 13:49:54 +00:00
|
|
|
}
|
|
|
|
|
2018-08-27 18:05:28 +00:00
|
|
|
void InterpreterSystemQuery::restartReplicas(Context & system_context)
|
2018-05-21 13:49:54 +00:00
|
|
|
{
|
2020-03-04 20:29:52 +00:00
|
|
|
std::vector<StorageID> replica_names;
|
2020-03-30 23:36:23 +00:00
|
|
|
auto & catalog = DatabaseCatalog::instance();
|
2018-05-21 13:49:54 +00:00
|
|
|
|
2020-03-30 23:36:23 +00:00
|
|
|
for (auto & elem : catalog.getDatabases())
|
2018-05-21 13:49:54 +00:00
|
|
|
{
|
|
|
|
DatabasePtr & database = elem.second;
|
2020-05-28 20:10:45 +00:00
|
|
|
for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next())
|
2018-05-21 13:49:54 +00:00
|
|
|
{
|
2020-06-02 02:06:16 +00:00
|
|
|
if (auto table = iterator->table())
|
|
|
|
{
|
|
|
|
if (dynamic_cast<const StorageReplicatedMergeTree *>(table.get()))
|
2020-07-07 12:11:58 +00:00
|
|
|
replica_names.emplace_back(StorageID{iterator->databaseName(), iterator->name()});
|
2020-06-02 02:06:16 +00:00
|
|
|
}
|
2018-05-21 13:49:54 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (replica_names.empty())
|
|
|
|
return;
|
|
|
|
|
2020-03-30 23:36:23 +00:00
|
|
|
TableGuards guards;
|
|
|
|
for (const auto & name : replica_names)
|
|
|
|
guards.emplace(UniqueTableName{name.database_name, name.table_name}, nullptr);
|
|
|
|
for (auto & guard : guards)
|
|
|
|
guard.second = catalog.getDDLGuard(guard.first.database_name, guard.first.table_name);
|
|
|
|
|
2019-01-09 15:44:20 +00:00
|
|
|
ThreadPool pool(std::min(size_t(getNumberOfPhysicalCPUCores()), replica_names.size()));
|
2018-05-21 13:49:54 +00:00
|
|
|
for (auto & table : replica_names)
|
2020-03-30 23:36:23 +00:00
|
|
|
pool.scheduleOrThrowOnError([&]() { tryRestartReplica(table, system_context, false); });
|
2018-05-21 13:49:54 +00:00
|
|
|
pool.wait();
|
|
|
|
}
|
|
|
|
|
2020-05-17 12:44:22 +00:00
|
|
|
void InterpreterSystemQuery::dropReplica(ASTSystemQuery & query)
|
|
|
|
{
|
2020-06-23 12:01:51 +00:00
|
|
|
if (query.replica.empty())
|
|
|
|
throw Exception("Replica name is empty", ErrorCodes::BAD_ARGUMENTS);
|
2020-06-22 15:01:40 +00:00
|
|
|
|
2020-05-17 12:44:22 +00:00
|
|
|
if (!table_id.empty())
|
|
|
|
{
|
|
|
|
context.checkAccess(AccessType::SYSTEM_DROP_REPLICA, table_id);
|
2020-06-05 07:03:51 +00:00
|
|
|
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context);
|
2020-05-17 12:44:22 +00:00
|
|
|
|
2020-06-23 12:01:51 +00:00
|
|
|
if (!dropReplicaImpl(query, table))
|
2020-05-17 12:44:22 +00:00
|
|
|
throw Exception("Table " + table_id.getNameForLogs() + " is not replicated", ErrorCodes::BAD_ARGUMENTS);
|
|
|
|
}
|
2020-06-05 07:03:51 +00:00
|
|
|
else if (!query.database.empty())
|
2020-05-17 12:44:22 +00:00
|
|
|
{
|
2020-06-23 12:01:51 +00:00
|
|
|
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)
|
2020-05-24 13:53:23 +00:00
|
|
|
{
|
2020-06-23 12:01:51 +00:00
|
|
|
DatabasePtr & database = elem.second;
|
2020-06-05 07:03:51 +00:00
|
|
|
for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next())
|
2020-06-23 12:01:51 +00:00
|
|
|
dropReplicaImpl(query, iterator->table());
|
|
|
|
LOG_TRACE(log, "Dropped replica {} from database {}", query.replica, backQuoteIfNeed(database->getDatabaseName()));
|
2020-05-24 13:53:23 +00:00
|
|
|
}
|
2020-06-05 07:03:51 +00:00
|
|
|
}
|
|
|
|
else if (!query.replica_zk_path.empty())
|
|
|
|
{
|
2020-06-23 12:01:51 +00:00
|
|
|
context.checkAccess(AccessType::SYSTEM_DROP_REPLICA);
|
2020-06-22 15:01:40 +00:00
|
|
|
auto remote_replica_path = query.replica_zk_path + "/replicas/" + query.replica;
|
|
|
|
|
2020-06-23 12:01:51 +00:00
|
|
|
/// This check is actually redundant, but it may prevent from some user mistakes
|
|
|
|
for (auto & elem : DatabaseCatalog::instance().getDatabases())
|
2020-06-22 15:01:40 +00:00
|
|
|
{
|
|
|
|
DatabasePtr & database = elem.second;
|
|
|
|
for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next())
|
|
|
|
{
|
|
|
|
if (auto * storage_replicated = dynamic_cast<StorageReplicatedMergeTree *>(iterator->table().get()))
|
|
|
|
{
|
2020-06-23 12:01:51 +00:00
|
|
|
StorageReplicatedMergeTree::Status status;
|
2020-06-22 15:01:40 +00:00
|
|
|
storage_replicated->getStatus(status);
|
2020-06-23 12:01:51 +00:00
|
|
|
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);
|
2020-06-22 15:01:40 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-06-23 12:01:51 +00:00
|
|
|
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);
|
|
|
|
|
2020-06-22 15:01:40 +00:00
|
|
|
if (zookeeper->exists(remote_replica_path + "/is_active"))
|
|
|
|
throw Exception("Can't remove replica: " + query.replica + ", because it's active",
|
2020-06-23 12:01:51 +00:00
|
|
|
ErrorCodes::TABLE_WAS_NOT_DROPPED);
|
2020-06-22 15:01:40 +00:00
|
|
|
|
2020-06-23 12:01:51 +00:00
|
|
|
StorageReplicatedMergeTree::dropReplica(zookeeper, query.replica_zk_path, query.replica, log);
|
|
|
|
LOG_INFO(log, "Dropped replica {}", remote_replica_path);
|
2020-06-05 07:03:51 +00:00
|
|
|
}
|
2020-06-23 12:01:51 +00:00
|
|
|
else
|
|
|
|
throw Exception("Invalid query", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
2020-05-24 13:53:23 +00:00
|
|
|
|
2020-06-23 12:01:51 +00:00
|
|
|
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;
|
2020-05-17 12:44:22 +00:00
|
|
|
}
|
|
|
|
|
2020-03-04 20:29:52 +00:00
|
|
|
void InterpreterSystemQuery::syncReplica(ASTSystemQuery &)
|
2018-05-21 13:49:54 +00:00
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
context.checkAccess(AccessType::SYSTEM_SYNC_REPLICA, table_id);
|
2020-05-28 23:01:18 +00:00
|
|
|
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context);
|
2018-05-21 13:49:54 +00:00
|
|
|
|
2020-04-22 06:01:33 +00:00
|
|
|
if (auto * storage_replicated = dynamic_cast<StorageReplicatedMergeTree *>(table.get()))
|
2019-09-19 11:04:57 +00:00
|
|
|
{
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_TRACE(log, "Synchronizing entries in replica's queue with table's log and waiting for it to become empty");
|
2019-09-19 11:04:57 +00:00
|
|
|
if (!storage_replicated->waitForShrinkingQueueSize(0, context.getSettingsRef().receive_timeout.totalMilliseconds()))
|
2019-09-19 21:20:58 +00:00
|
|
|
{
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_ERROR(log, "SYNC REPLICA {}: Timed out!", table_id.getNameForLogs());
|
2019-09-19 11:04:57 +00:00
|
|
|
throw Exception(
|
2020-03-04 20:29:52 +00:00
|
|
|
"SYNC REPLICA " + table_id.getNameForLogs() + ": command timed out! "
|
2019-09-19 11:04:57 +00:00
|
|
|
"See the 'receive_timeout' setting", ErrorCodes::TIMEOUT_EXCEEDED);
|
2019-09-19 21:20:58 +00:00
|
|
|
}
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_TRACE(log, "SYNC REPLICA {}: OK", table_id.getNameForLogs());
|
2019-09-19 11:04:57 +00:00
|
|
|
}
|
2019-04-08 05:13:16 +00:00
|
|
|
else
|
2020-03-04 20:29:52 +00:00
|
|
|
throw Exception("Table " + table_id.getNameForLogs() + " is not replicated", ErrorCodes::BAD_ARGUMENTS);
|
2018-05-21 13:49:54 +00:00
|
|
|
}
|
|
|
|
|
2020-03-04 20:29:52 +00:00
|
|
|
void InterpreterSystemQuery::flushDistributed(ASTSystemQuery &)
|
2019-04-22 15:11:16 +00:00
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
context.checkAccess(AccessType::SYSTEM_FLUSH_DISTRIBUTED, table_id);
|
2019-04-22 15:11:16 +00:00
|
|
|
|
2020-05-28 23:01:18 +00:00
|
|
|
if (auto * storage_distributed = dynamic_cast<StorageDistributed *>(DatabaseCatalog::instance().getTable(table_id, context).get()))
|
2019-05-10 04:19:02 +00:00
|
|
|
storage_distributed->flushClusterNodesAllData();
|
2019-04-22 15:11:16 +00:00
|
|
|
else
|
2020-03-04 20:29:52 +00:00
|
|
|
throw Exception("Table " + table_id.getNameForLogs() + " is not distributed", ErrorCodes::BAD_ARGUMENTS);
|
2018-05-21 13:49:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-01-24 16:20:36 +00:00
|
|
|
AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() const
|
|
|
|
{
|
|
|
|
const auto & query = query_ptr->as<const ASTSystemQuery &>();
|
|
|
|
using Type = ASTSystemQuery::Type;
|
|
|
|
AccessRightsElements required_access;
|
|
|
|
switch (query.type)
|
|
|
|
{
|
|
|
|
case Type::SHUTDOWN: [[fallthrough]];
|
|
|
|
case Type::KILL:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_SHUTDOWN);
|
2020-01-24 16:20:36 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case Type::DROP_DNS_CACHE: [[fallthrough]];
|
|
|
|
case Type::DROP_MARK_CACHE: [[fallthrough]];
|
|
|
|
#if USE_EMBEDDED_COMPILER
|
|
|
|
case Type::DROP_COMPILED_EXPRESSION_CACHE: [[fallthrough]];
|
|
|
|
#endif
|
|
|
|
case Type::DROP_UNCOMPRESSED_CACHE:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_DROP_CACHE);
|
2020-01-24 16:20:36 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case Type::RELOAD_DICTIONARY: [[fallthrough]];
|
|
|
|
case Type::RELOAD_DICTIONARIES: [[fallthrough]];
|
|
|
|
case Type::RELOAD_EMBEDDED_DICTIONARIES:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_RELOAD_DICTIONARY);
|
2020-01-24 16:20:36 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case Type::RELOAD_CONFIG:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_RELOAD_CONFIG);
|
2020-01-24 16:20:36 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case Type::STOP_MERGES: [[fallthrough]];
|
|
|
|
case Type::START_MERGES:
|
|
|
|
{
|
|
|
|
if (query.table.empty())
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_MERGES);
|
2020-01-24 16:20:36 +00:00
|
|
|
else
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_MERGES, query.database, query.table);
|
2020-01-24 16:20:36 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case Type::STOP_TTL_MERGES: [[fallthrough]];
|
|
|
|
case Type::START_TTL_MERGES:
|
|
|
|
{
|
|
|
|
if (query.table.empty())
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_TTL_MERGES);
|
2020-01-24 16:20:36 +00:00
|
|
|
else
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_TTL_MERGES, query.database, query.table);
|
2020-01-24 16:20:36 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case Type::STOP_MOVES: [[fallthrough]];
|
|
|
|
case Type::START_MOVES:
|
|
|
|
{
|
|
|
|
if (query.table.empty())
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_MOVES);
|
2020-01-24 16:20:36 +00:00
|
|
|
else
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_MOVES, query.database, query.table);
|
2020-01-24 16:20:36 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case Type::STOP_FETCHES: [[fallthrough]];
|
|
|
|
case Type::START_FETCHES:
|
|
|
|
{
|
|
|
|
if (query.table.empty())
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_FETCHES);
|
2020-01-24 16:20:36 +00:00
|
|
|
else
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_FETCHES, query.database, query.table);
|
2020-01-24 16:20:36 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case Type::STOP_DISTRIBUTED_SENDS: [[fallthrough]];
|
|
|
|
case Type::START_DISTRIBUTED_SENDS:
|
|
|
|
{
|
|
|
|
if (query.table.empty())
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_DISTRIBUTED_SENDS);
|
2020-01-24 16:20:36 +00:00
|
|
|
else
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_DISTRIBUTED_SENDS, query.database, query.table);
|
2020-01-24 16:20:36 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case Type::STOP_REPLICATED_SENDS: [[fallthrough]];
|
|
|
|
case Type::START_REPLICATED_SENDS:
|
|
|
|
{
|
|
|
|
if (query.table.empty())
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_REPLICATED_SENDS);
|
2020-01-24 16:20:36 +00:00
|
|
|
else
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_REPLICATED_SENDS, query.database, query.table);
|
2020-01-24 16:20:36 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case Type::STOP_REPLICATION_QUEUES: [[fallthrough]];
|
|
|
|
case Type::START_REPLICATION_QUEUES:
|
|
|
|
{
|
|
|
|
if (query.table.empty())
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_REPLICATION_QUEUES);
|
2020-01-24 16:20:36 +00:00
|
|
|
else
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_REPLICATION_QUEUES, query.database, query.table);
|
2020-01-24 16:20:36 +00:00
|
|
|
break;
|
|
|
|
}
|
2020-05-17 12:44:22 +00:00
|
|
|
case Type::DROP_REPLICA:
|
|
|
|
{
|
|
|
|
required_access.emplace_back(AccessType::SYSTEM_DROP_REPLICA, query.database, query.table);
|
|
|
|
break;
|
|
|
|
}
|
2020-01-24 16:20:36 +00:00
|
|
|
case Type::SYNC_REPLICA:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_SYNC_REPLICA, query.database, query.table);
|
2020-01-24 16:20:36 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case Type::RESTART_REPLICA:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_RESTART_REPLICA, query.database, query.table);
|
2020-01-24 16:20:36 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case Type::RESTART_REPLICAS:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_RESTART_REPLICA);
|
2020-01-24 16:20:36 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case Type::FLUSH_DISTRIBUTED:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_FLUSH_DISTRIBUTED, query.database, query.table);
|
2020-01-24 16:20:36 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case Type::FLUSH_LOGS:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SYSTEM_FLUSH_LOGS);
|
2020-01-24 16:20:36 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case Type::STOP_LISTEN_QUERIES: break;
|
|
|
|
case Type::START_LISTEN_QUERIES: break;
|
|
|
|
case Type::UNKNOWN: break;
|
|
|
|
case Type::END: break;
|
|
|
|
}
|
|
|
|
return required_access;
|
|
|
|
}
|
|
|
|
|
2017-08-30 21:25:44 +00:00
|
|
|
}
|