2024-03-02 15:28:45 +00:00
|
|
|
#include <Interpreters/ApplyWithSubqueryVisitor.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/InterpreterAlterQuery.h>
|
2024-01-09 06:33:48 +00:00
|
|
|
#include <Interpreters/InterpreterFactory.h>
|
2021-04-10 23:33:54 +00:00
|
|
|
|
2021-10-31 08:51:20 +00:00
|
|
|
#include <Access/Common/AccessRightsElement.h>
|
2021-04-10 23:33:54 +00:00
|
|
|
#include <Databases/DatabaseFactory.h>
|
|
|
|
#include <Databases/DatabaseReplicated.h>
|
|
|
|
#include <Databases/IDatabase.h>
|
2018-12-05 16:13:52 +00:00
|
|
|
#include <Interpreters/AddDefaultDatabaseVisitor.h>
|
2018-12-25 23:11:36 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2021-11-05 04:21:52 +00:00
|
|
|
#include <Interpreters/FunctionNameNormalizer.h>
|
2024-02-28 00:00:17 +00:00
|
|
|
#include <Interpreters/InterpreterCreateQuery.h>
|
2021-04-10 23:33:54 +00:00
|
|
|
#include <Interpreters/MutationsInterpreter.h>
|
2023-05-25 22:54:54 +00:00
|
|
|
#include <Interpreters/MutationsNonDeterministicHelpers.h>
|
2020-12-14 03:30:39 +00:00
|
|
|
#include <Interpreters/QueryLog.h>
|
2021-04-10 23:33:54 +00:00
|
|
|
#include <Interpreters/executeDDLQueryOnCluster.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/ASTAlterQuery.h>
|
2020-01-24 16:20:36 +00:00
|
|
|
#include <Parsers/ASTAssignment.h>
|
2021-11-26 15:49:40 +00:00
|
|
|
#include <Parsers/ASTIdentifier_fwd.h>
|
2022-01-10 19:01:41 +00:00
|
|
|
#include <Parsers/ASTColumnDeclaration.h>
|
2023-05-25 22:54:54 +00:00
|
|
|
#include <Parsers/queryToString.h>
|
2018-12-25 23:11:36 +00:00
|
|
|
#include <Storages/AlterCommands.h>
|
2021-04-10 23:33:54 +00:00
|
|
|
#include <Storages/IStorage.h>
|
|
|
|
#include <Storages/MutationCommands.h>
|
|
|
|
#include <Storages/PartitionCommands.h>
|
2023-04-03 13:56:03 +00:00
|
|
|
#include <Storages/StorageKeeperMap.h>
|
2018-06-13 13:49:27 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2021-04-10 23:33:54 +00:00
|
|
|
|
2022-11-25 10:16:20 +00:00
|
|
|
#include <Functions/UserDefined/UserDefinedSQLFunctionFactory.h>
|
|
|
|
#include <Functions/UserDefined/UserDefinedSQLFunctionVisitor.h>
|
|
|
|
|
2020-03-06 17:13:28 +00:00
|
|
|
#include <boost/range/algorithm_ext/push_back.hpp>
|
2021-04-10 23:33:54 +00:00
|
|
|
|
2013-08-07 13:07:42 +00:00
|
|
|
#include <algorithm>
|
|
|
|
|
2013-08-09 00:12:59 +00:00
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
2020-01-22 13:24:20 +00:00
|
|
|
extern const int INCORRECT_QUERY;
|
2021-02-08 19:36:17 +00:00
|
|
|
extern const int NOT_IMPLEMENTED;
|
2021-08-23 11:26:54 +00:00
|
|
|
extern const int TABLE_IS_READ_ONLY;
|
2023-04-03 13:56:03 +00:00
|
|
|
extern const int BAD_ARGUMENTS;
|
2023-04-03 15:17:06 +00:00
|
|
|
extern const int UNKNOWN_TABLE;
|
2023-11-08 23:30:35 +00:00
|
|
|
extern const int UNKNOWN_DATABASE;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
2013-09-23 12:01:19 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
InterpreterAlterQuery::InterpreterAlterQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_)
|
2013-08-07 13:07:42 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2021-08-27 06:30:21 +00:00
|
|
|
|
2015-06-18 02:11:05 +00:00
|
|
|
BlockIO InterpreterAlterQuery::execute()
|
2013-12-18 11:19:37 +00:00
|
|
|
{
|
2024-04-07 09:51:45 +00:00
|
|
|
FunctionNameNormalizer::visit(query_ptr.get());
|
2019-03-15 16:14:13 +00:00
|
|
|
const auto & alter = query_ptr->as<ASTAlterQuery &>();
|
2021-08-27 06:30:21 +00:00
|
|
|
if (alter.alter_object == ASTAlterQuery::AlterObjectType::DATABASE)
|
2021-12-28 16:41:57 +00:00
|
|
|
{
|
2021-08-27 06:30:21 +00:00
|
|
|
return executeToDatabase(alter);
|
2021-12-28 16:41:57 +00:00
|
|
|
}
|
2024-02-25 23:01:34 +00:00
|
|
|
else if (alter.alter_object == ASTAlterQuery::AlterObjectType::TABLE)
|
2021-12-28 16:41:57 +00:00
|
|
|
{
|
2021-08-27 06:30:21 +00:00
|
|
|
return executeToTable(alter);
|
2021-12-28 16:41:57 +00:00
|
|
|
}
|
|
|
|
|
2021-08-28 16:09:35 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown alter object type");
|
2021-08-27 06:30:21 +00:00
|
|
|
}
|
2017-04-25 15:21:03 +00:00
|
|
|
|
2021-08-27 06:30:21 +00:00
|
|
|
BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
|
|
|
|
{
|
2024-03-02 15:28:45 +00:00
|
|
|
ASTSelectWithUnionQuery * modify_query = nullptr;
|
|
|
|
|
2024-02-28 00:00:17 +00:00
|
|
|
for (auto & child : alter.command_list->children)
|
|
|
|
{
|
|
|
|
auto * command_ast = child->as<ASTAlterCommand>();
|
|
|
|
if (command_ast->sql_security)
|
|
|
|
InterpreterCreateQuery::processSQLSecurityOption(getContext(), command_ast->sql_security->as<ASTSQLSecurity &>());
|
2024-03-02 15:28:45 +00:00
|
|
|
else if (command_ast->type == ASTAlterCommand::MODIFY_QUERY)
|
|
|
|
modify_query = command_ast->select->as<ASTSelectWithUnionQuery>();
|
2024-02-28 00:00:17 +00:00
|
|
|
}
|
|
|
|
|
2021-08-27 06:30:21 +00:00
|
|
|
BlockIO res;
|
2020-05-11 12:55:17 +00:00
|
|
|
|
2022-11-25 10:16:20 +00:00
|
|
|
if (!UserDefinedSQLFunctionFactory::instance().empty())
|
|
|
|
UserDefinedSQLFunctionVisitor::visit(query_ptr);
|
|
|
|
|
2023-11-16 21:24:58 +00:00
|
|
|
auto table_id = getContext()->tryResolveStorageID(alter);
|
2023-11-08 23:30:35 +00:00
|
|
|
StoragePtr table;
|
|
|
|
|
|
|
|
if (table_id)
|
|
|
|
{
|
|
|
|
query_ptr->as<ASTAlterQuery &>().setDatabase(table_id.database_name);
|
|
|
|
table = DatabaseCatalog::instance().tryGetTable(table_id, getContext());
|
|
|
|
}
|
2023-04-03 13:56:03 +00:00
|
|
|
|
2022-07-06 17:54:05 +00:00
|
|
|
if (!alter.cluster.empty() && !maybeRemoveOnCluster(query_ptr, getContext()))
|
2022-04-22 12:15:29 +00:00
|
|
|
{
|
2023-04-03 14:54:14 +00:00
|
|
|
if (table && table->as<StorageKeeperMap>())
|
2023-04-03 13:56:03 +00:00
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Mutations with ON CLUSTER are not allowed for KeeperMap tables");
|
|
|
|
|
2022-04-22 12:15:29 +00:00
|
|
|
DDLQueryOnClusterParams params;
|
|
|
|
params.access_to_check = getRequiredAccess();
|
|
|
|
return executeDDLQueryOnCluster(query_ptr, getContext(), params);
|
|
|
|
}
|
2020-01-24 16:20:36 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
getContext()->checkAccess(getRequiredAccess());
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2023-11-08 23:30:35 +00:00
|
|
|
if (!table_id)
|
|
|
|
throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} does not exist", backQuoteIfNeed(alter.getDatabase()));
|
|
|
|
|
2020-05-11 12:55:17 +00:00
|
|
|
DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name);
|
2022-09-15 19:15:57 +00:00
|
|
|
if (database->shouldReplicateQuery(getContext(), query_ptr))
|
2021-02-01 19:29:47 +00:00
|
|
|
{
|
2021-02-02 19:39:04 +00:00
|
|
|
auto guard = DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name);
|
|
|
|
guard->releaseTableLock();
|
2022-09-16 14:25:32 +00:00
|
|
|
return database->tryEnqueueReplicatedDDL(query_ptr, getContext());
|
2021-02-01 19:29:47 +00:00
|
|
|
}
|
2020-05-11 12:55:17 +00:00
|
|
|
|
2023-04-03 14:54:14 +00:00
|
|
|
if (!table)
|
2023-04-03 14:56:45 +00:00
|
|
|
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Could not find table: {}", table_id.table_name);
|
2023-04-03 14:54:14 +00:00
|
|
|
|
2022-01-31 22:27:55 +00:00
|
|
|
checkStorageSupportsTransactionsIfNeeded(table, getContext());
|
2021-09-04 09:02:07 +00:00
|
|
|
if (table->isStaticStorage())
|
2021-08-23 11:26:54 +00:00
|
|
|
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only");
|
2021-10-25 17:49:49 +00:00
|
|
|
auto table_lock = table->lockForShare(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout);
|
2021-02-02 19:39:04 +00:00
|
|
|
|
2024-03-02 15:28:45 +00:00
|
|
|
if (modify_query)
|
|
|
|
{
|
|
|
|
// Expand CTE before filling default database
|
2024-04-07 09:51:45 +00:00
|
|
|
ApplyWithSubqueryVisitor::visit(*modify_query);
|
2024-03-02 15:28:45 +00:00
|
|
|
}
|
|
|
|
|
2021-04-14 15:35:52 +00:00
|
|
|
/// Add default database to table identifiers that we can encounter in e.g. default expressions, mutation expression, etc.
|
2021-10-01 19:35:17 +00:00
|
|
|
AddDefaultDatabaseVisitor visitor(getContext(), table_id.getDatabaseName());
|
2019-03-15 16:14:13 +00:00
|
|
|
ASTPtr command_list_ptr = alter.command_list->ptr();
|
2018-12-05 16:13:52 +00:00
|
|
|
visitor.visit(command_list_ptr);
|
|
|
|
|
2014-08-07 09:23:55 +00:00
|
|
|
AlterCommands alter_commands;
|
|
|
|
PartitionCommands partition_commands;
|
2018-02-02 16:02:43 +00:00
|
|
|
MutationCommands mutation_commands;
|
2020-12-04 02:15:44 +00:00
|
|
|
for (const auto & child : alter.command_list->children)
|
2018-06-13 13:49:27 +00:00
|
|
|
{
|
2020-12-04 02:15:44 +00:00
|
|
|
auto * command_ast = child->as<ASTAlterCommand>();
|
2020-08-28 17:40:45 +00:00
|
|
|
if (auto alter_command = AlterCommand::parse(command_ast))
|
2021-08-27 06:30:21 +00:00
|
|
|
{
|
2018-06-13 13:49:27 +00:00
|
|
|
alter_commands.emplace_back(std::move(*alter_command));
|
2021-08-27 06:30:21 +00:00
|
|
|
}
|
2018-06-13 13:49:27 +00:00
|
|
|
else if (auto partition_command = PartitionCommand::parse(command_ast))
|
2019-07-22 11:23:11 +00:00
|
|
|
{
|
2018-06-13 13:49:27 +00:00
|
|
|
partition_commands.emplace_back(std::move(*partition_command));
|
2019-07-22 11:23:11 +00:00
|
|
|
}
|
2018-06-13 13:49:27 +00:00
|
|
|
else if (auto mut_command = MutationCommand::parse(command_ast))
|
2020-01-22 13:24:20 +00:00
|
|
|
{
|
2023-08-07 17:16:15 +00:00
|
|
|
if (mut_command->type == MutationCommand::UPDATE || mut_command->type == MutationCommand::DELETE)
|
|
|
|
{
|
|
|
|
/// TODO: add a check for result query size.
|
|
|
|
auto rewritten_command_ast = replaceNonDeterministicToScalars(*command_ast, getContext());
|
|
|
|
if (rewritten_command_ast)
|
|
|
|
{
|
|
|
|
auto * new_alter_command = rewritten_command_ast->as<ASTAlterCommand>();
|
|
|
|
mut_command = MutationCommand::parse(new_alter_command);
|
|
|
|
if (!mut_command)
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
|
|
|
"Alter command '{}' is rewritten to invalid command '{}'",
|
|
|
|
queryToString(*command_ast), queryToString(*rewritten_command_ast));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-06-13 13:49:27 +00:00
|
|
|
mutation_commands.emplace_back(std::move(*mut_command));
|
2020-01-22 13:24:20 +00:00
|
|
|
}
|
2018-06-13 13:49:27 +00:00
|
|
|
else
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong parameter type in ALTER query");
|
2023-12-01 19:12:05 +00:00
|
|
|
|
2023-09-26 17:16:01 +00:00
|
|
|
if (!getContext()->getSettings().allow_experimental_statistic && (
|
2023-09-19 15:08:15 +00:00
|
|
|
command_ast->type == ASTAlterCommand::ADD_STATISTIC ||
|
|
|
|
command_ast->type == ASTAlterCommand::DROP_STATISTIC ||
|
2023-09-26 17:16:01 +00:00
|
|
|
command_ast->type == ASTAlterCommand::MATERIALIZE_STATISTIC))
|
2023-09-19 15:08:15 +00:00
|
|
|
throw Exception(ErrorCodes::INCORRECT_QUERY, "Alter table with statistic is now disabled. Turn on allow_experimental_statistic");
|
2018-06-13 13:49:27 +00:00
|
|
|
}
|
2018-02-02 16:02:43 +00:00
|
|
|
|
2021-02-08 19:36:17 +00:00
|
|
|
if (typeid_cast<DatabaseReplicated *>(database.get()))
|
|
|
|
{
|
2023-06-22 18:32:29 +00:00
|
|
|
int command_types_count = !mutation_commands.empty() + !partition_commands.empty() + !alter_commands.empty();
|
2023-09-06 12:17:46 +00:00
|
|
|
bool mixed_settings_amd_metadata_alter = alter_commands.hasNonReplicatedAlterCommand() && !alter_commands.areNonReplicatedAlterCommands();
|
2021-07-30 16:34:18 +00:00
|
|
|
if (1 < command_types_count || mixed_settings_amd_metadata_alter)
|
2021-02-08 19:36:17 +00:00
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "For Replicated databases it's not allowed "
|
2023-09-06 12:17:46 +00:00
|
|
|
"to execute ALTERs of different types (replicated and non replicated) in single query");
|
2021-02-08 19:36:17 +00:00
|
|
|
}
|
|
|
|
|
2023-11-04 14:35:39 +00:00
|
|
|
if (!alter_commands.empty())
|
|
|
|
{
|
|
|
|
auto alter_lock = table->lockForAlter(getContext()->getSettingsRef().lock_acquire_timeout);
|
|
|
|
StorageInMemoryMetadata metadata = table->getInMemoryMetadata();
|
|
|
|
alter_commands.validate(table, getContext());
|
|
|
|
alter_commands.prepare(metadata);
|
|
|
|
table->checkAlterIsPossible(alter_commands, getContext());
|
|
|
|
table->alter(alter_commands, getContext(), alter_lock);
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Get newest metadata_snapshot after execute ALTER command, in order to
|
|
|
|
/// support like materialize index in the same ALTER query that creates it.
|
2023-11-05 02:29:15 +00:00
|
|
|
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
2023-11-04 14:35:39 +00:00
|
|
|
|
2022-12-22 13:31:42 +00:00
|
|
|
if (mutation_commands.hasNonEmptyMutationCommands())
|
2018-02-02 16:02:43 +00:00
|
|
|
{
|
2023-11-04 14:35:39 +00:00
|
|
|
for (const auto & command : mutation_commands)
|
|
|
|
{
|
|
|
|
/// Check it after alter finished, so we can add TTL and materialize TTL in the same ALTER query.
|
|
|
|
if (command.type == MutationCommand::MATERIALIZE_TTL && !metadata_snapshot->hasAnyTTL())
|
|
|
|
throw Exception(ErrorCodes::INCORRECT_QUERY, "Cannot MATERIALIZE TTL as there is no TTL set for table {}",
|
|
|
|
table->getStorageID().getNameForLogs());
|
|
|
|
|
|
|
|
}
|
2021-04-10 23:33:54 +00:00
|
|
|
table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef());
|
2023-05-25 22:54:54 +00:00
|
|
|
MutationsInterpreter::Settings settings(false);
|
|
|
|
MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), settings).validate();
|
2023-01-30 17:38:28 +00:00
|
|
|
table->mutate(mutation_commands, getContext());
|
2018-02-02 16:02:43 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-11-13 13:48:53 +00:00
|
|
|
if (!partition_commands.empty())
|
2014-08-07 09:23:55 +00:00
|
|
|
{
|
2023-10-13 14:22:18 +00:00
|
|
|
table->checkAlterPartitionIsPossible(partition_commands, metadata_snapshot, getContext()->getSettingsRef(), getContext());
|
2021-04-10 23:33:54 +00:00
|
|
|
auto partition_commands_pipe = table->alterPartition(metadata_snapshot, partition_commands, getContext());
|
2020-08-04 15:51:56 +00:00
|
|
|
if (!partition_commands_pipe.empty())
|
2021-09-15 19:35:48 +00:00
|
|
|
res.pipeline = QueryPipeline(std::move(partition_commands_pipe));
|
2014-08-07 09:23:55 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-07-28 15:10:36 +00:00
|
|
|
return res;
|
2013-12-18 11:19:37 +00:00
|
|
|
}
|
|
|
|
|
2020-01-24 16:20:36 +00:00
|
|
|
|
2021-08-27 06:30:21 +00:00
|
|
|
BlockIO InterpreterAlterQuery::executeToDatabase(const ASTAlterQuery & alter)
|
|
|
|
{
|
|
|
|
BlockIO res;
|
|
|
|
getContext()->checkAccess(getRequiredAccess());
|
2021-09-22 13:57:17 +00:00
|
|
|
DatabasePtr database = DatabaseCatalog::instance().getDatabase(alter.getDatabase());
|
2021-08-27 06:30:21 +00:00
|
|
|
AlterCommands alter_commands;
|
|
|
|
|
|
|
|
for (const auto & child : alter.command_list->children)
|
|
|
|
{
|
|
|
|
auto * command_ast = child->as<ASTAlterCommand>();
|
|
|
|
if (auto alter_command = AlterCommand::parse(command_ast))
|
|
|
|
alter_commands.emplace_back(std::move(*alter_command));
|
|
|
|
else
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong parameter type in ALTER DATABASE query");
|
2021-08-27 06:30:21 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if (!alter_commands.empty())
|
|
|
|
{
|
2021-09-08 22:25:08 +00:00
|
|
|
/// Only ALTER SETTING is supported.
|
|
|
|
for (const auto & command : alter_commands)
|
|
|
|
{
|
|
|
|
if (command.type != AlterCommand::MODIFY_DATABASE_SETTING)
|
2021-09-15 15:22:33 +00:00
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported alter type for database engines");
|
2021-09-08 22:25:08 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
for (const auto & command : alter_commands)
|
|
|
|
{
|
|
|
|
if (!command.ignore)
|
|
|
|
{
|
|
|
|
if (command.type == AlterCommand::MODIFY_DATABASE_SETTING)
|
2021-09-15 15:22:33 +00:00
|
|
|
database->applySettingsChanges(command.settings_changes, getContext());
|
2021-09-08 22:25:08 +00:00
|
|
|
else
|
2021-09-09 06:53:01 +00:00
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported alter command");
|
2021-09-08 22:25:08 +00:00
|
|
|
}
|
|
|
|
}
|
2021-08-27 06:30:21 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return res;
|
|
|
|
}
|
2023-05-25 22:54:54 +00:00
|
|
|
|
2020-01-24 16:20:36 +00:00
|
|
|
AccessRightsElements InterpreterAlterQuery::getRequiredAccess() const
|
|
|
|
{
|
|
|
|
AccessRightsElements required_access;
|
|
|
|
const auto & alter = query_ptr->as<ASTAlterQuery &>();
|
2020-12-04 02:15:44 +00:00
|
|
|
for (const auto & child : alter.command_list->children)
|
2021-09-06 22:13:54 +00:00
|
|
|
boost::range::push_back(required_access, getRequiredAccessForCommand(child->as<ASTAlterCommand&>(), alter.getDatabase(), alter.getTable()));
|
2020-03-06 17:13:28 +00:00
|
|
|
return required_access;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const ASTAlterCommand & command, const String & database, const String & table)
|
|
|
|
{
|
|
|
|
AccessRightsElements required_access;
|
|
|
|
|
|
|
|
auto column_name = [&]() -> String { return getIdentifierName(command.column); };
|
|
|
|
auto column_name_from_col_decl = [&]() -> std::string_view { return command.col_decl->as<ASTColumnDeclaration &>().name; };
|
|
|
|
auto column_names_from_update_assignments = [&]() -> std::vector<std::string_view>
|
2020-01-24 16:20:36 +00:00
|
|
|
{
|
2020-03-06 17:13:28 +00:00
|
|
|
std::vector<std::string_view> column_names;
|
|
|
|
for (const ASTPtr & assignment_ast : command.update_assignments->children)
|
|
|
|
column_names.emplace_back(assignment_ast->as<const ASTAssignment &>().column_name);
|
|
|
|
return column_names;
|
|
|
|
};
|
2020-01-24 16:20:36 +00:00
|
|
|
|
2020-03-06 17:13:28 +00:00
|
|
|
switch (command.type)
|
|
|
|
{
|
|
|
|
case ASTAlterCommand::UPDATE:
|
2020-01-24 16:20:36 +00:00
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_UPDATE, database, table, column_names_from_update_assignments());
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::ADD_COLUMN:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_ADD_COLUMN, database, table, column_name_from_col_decl());
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::DROP_COLUMN:
|
|
|
|
{
|
|
|
|
if (command.clear_column)
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_CLEAR_COLUMN, database, table, column_name());
|
2020-03-06 17:13:28 +00:00
|
|
|
else
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_DROP_COLUMN, database, table, column_name());
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::MODIFY_COLUMN:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_MODIFY_COLUMN, database, table, column_name_from_col_decl());
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::COMMENT_COLUMN:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_COMMENT_COLUMN, database, table, column_name());
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
2021-07-31 18:17:06 +00:00
|
|
|
case ASTAlterCommand::MATERIALIZE_COLUMN:
|
|
|
|
{
|
|
|
|
required_access.emplace_back(AccessType::ALTER_MATERIALIZE_COLUMN, database, table);
|
|
|
|
break;
|
|
|
|
}
|
2020-03-06 17:13:28 +00:00
|
|
|
case ASTAlterCommand::MODIFY_ORDER_BY:
|
|
|
|
{
|
|
|
|
required_access.emplace_back(AccessType::ALTER_ORDER_BY, database, table);
|
|
|
|
break;
|
|
|
|
}
|
2021-10-14 13:44:28 +00:00
|
|
|
case ASTAlterCommand::REMOVE_SAMPLE_BY:
|
2020-08-27 13:10:10 +00:00
|
|
|
case ASTAlterCommand::MODIFY_SAMPLE_BY:
|
|
|
|
{
|
|
|
|
required_access.emplace_back(AccessType::ALTER_SAMPLE_BY, database, table);
|
|
|
|
break;
|
|
|
|
}
|
2023-08-09 20:57:49 +00:00
|
|
|
case ASTAlterCommand::ADD_STATISTIC:
|
|
|
|
{
|
|
|
|
required_access.emplace_back(AccessType::ALTER_ADD_STATISTIC, database, table);
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::DROP_STATISTIC:
|
|
|
|
{
|
|
|
|
required_access.emplace_back(AccessType::ALTER_DROP_STATISTIC, database, table);
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::MATERIALIZE_STATISTIC:
|
|
|
|
{
|
|
|
|
required_access.emplace_back(AccessType::ALTER_MATERIALIZE_STATISTIC, database, table);
|
|
|
|
break;
|
|
|
|
}
|
2020-03-06 17:13:28 +00:00
|
|
|
case ASTAlterCommand::ADD_INDEX:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_ADD_INDEX, database, table);
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::DROP_INDEX:
|
|
|
|
{
|
|
|
|
if (command.clear_index)
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_CLEAR_INDEX, database, table);
|
2020-03-06 17:13:28 +00:00
|
|
|
else
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_DROP_INDEX, database, table);
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::MATERIALIZE_INDEX:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_MATERIALIZE_INDEX, database, table);
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::ADD_CONSTRAINT:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_ADD_CONSTRAINT, database, table);
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::DROP_CONSTRAINT:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_DROP_CONSTRAINT, database, table);
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
2021-02-10 14:12:49 +00:00
|
|
|
case ASTAlterCommand::ADD_PROJECTION:
|
2020-03-06 17:13:28 +00:00
|
|
|
{
|
2021-02-10 14:12:49 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_ADD_PROJECTION, database, table);
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::DROP_PROJECTION:
|
|
|
|
{
|
|
|
|
if (command.clear_projection)
|
|
|
|
required_access.emplace_back(AccessType::ALTER_CLEAR_PROJECTION, database, table);
|
|
|
|
else
|
|
|
|
required_access.emplace_back(AccessType::ALTER_DROP_PROJECTION, database, table);
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::MATERIALIZE_PROJECTION:
|
|
|
|
{
|
|
|
|
required_access.emplace_back(AccessType::ALTER_MATERIALIZE_PROJECTION, database, table);
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::MODIFY_TTL:
|
2020-09-20 13:27:33 +00:00
|
|
|
case ASTAlterCommand::REMOVE_TTL:
|
|
|
|
{
|
|
|
|
required_access.emplace_back(AccessType::ALTER_TTL, database, table);
|
|
|
|
break;
|
|
|
|
}
|
2020-03-06 17:13:28 +00:00
|
|
|
case ASTAlterCommand::MATERIALIZE_TTL:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_MATERIALIZE_TTL, database, table);
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
2020-12-02 21:18:25 +00:00
|
|
|
case ASTAlterCommand::RESET_SETTING: [[fallthrough]];
|
2020-03-06 17:13:28 +00:00
|
|
|
case ASTAlterCommand::MODIFY_SETTING:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_SETTINGS, database, table);
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::ATTACH_PARTITION:
|
|
|
|
{
|
|
|
|
required_access.emplace_back(AccessType::INSERT, database, table);
|
|
|
|
break;
|
|
|
|
}
|
2021-04-22 19:50:48 +00:00
|
|
|
case ASTAlterCommand::DELETE:
|
2023-12-01 19:12:05 +00:00
|
|
|
case ASTAlterCommand::APPLY_DELETED_MASK:
|
2021-04-22 19:50:48 +00:00
|
|
|
case ASTAlterCommand::DROP_PARTITION:
|
2020-03-06 17:13:28 +00:00
|
|
|
case ASTAlterCommand::DROP_DETACHED_PARTITION:
|
2024-02-02 10:03:31 +00:00
|
|
|
case ASTAlterCommand::FORGET_PARTITION:
|
2020-03-06 17:13:28 +00:00
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_DELETE, database, table);
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::MOVE_PARTITION:
|
|
|
|
{
|
2021-06-04 16:03:11 +00:00
|
|
|
switch (command.move_destination_type)
|
2020-01-24 16:20:36 +00:00
|
|
|
{
|
2021-06-04 16:03:11 +00:00
|
|
|
case DataDestinationType::DISK: [[fallthrough]];
|
|
|
|
case DataDestinationType::VOLUME:
|
|
|
|
required_access.emplace_back(AccessType::ALTER_MOVE_PARTITION, database, table);
|
|
|
|
break;
|
|
|
|
case DataDestinationType::TABLE:
|
|
|
|
required_access.emplace_back(AccessType::SELECT | AccessType::ALTER_DELETE, database, table);
|
|
|
|
required_access.emplace_back(AccessType::INSERT, command.to_database, command.to_table);
|
|
|
|
break;
|
|
|
|
case DataDestinationType::SHARD:
|
2021-06-04 16:25:40 +00:00
|
|
|
required_access.emplace_back(AccessType::SELECT | AccessType::ALTER_DELETE, database, table);
|
|
|
|
required_access.emplace_back(AccessType::MOVE_PARTITION_BETWEEN_SHARDS);
|
2021-06-04 16:03:11 +00:00
|
|
|
break;
|
|
|
|
case DataDestinationType::DELETE:
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected destination type for command.");
|
2020-01-24 16:20:36 +00:00
|
|
|
}
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::REPLACE_PARTITION:
|
|
|
|
{
|
|
|
|
required_access.emplace_back(AccessType::SELECT, command.from_database, command.from_table);
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_DELETE | AccessType::INSERT, database, table);
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::FETCH_PARTITION:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_FETCH_PARTITION, database, table);
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
2022-04-19 15:51:23 +00:00
|
|
|
case ASTAlterCommand::FREEZE_PARTITION:
|
|
|
|
case ASTAlterCommand::FREEZE_ALL:
|
|
|
|
case ASTAlterCommand::UNFREEZE_PARTITION:
|
2021-03-02 21:10:09 +00:00
|
|
|
case ASTAlterCommand::UNFREEZE_ALL:
|
2020-03-06 17:13:28 +00:00
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_FREEZE_PARTITION, database, table);
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::MODIFY_QUERY:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_VIEW_MODIFY_QUERY, database, table);
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
2023-11-29 02:32:41 +00:00
|
|
|
case ASTAlterCommand::MODIFY_REFRESH:
|
|
|
|
{
|
|
|
|
required_access.emplace_back(AccessType::ALTER_VIEW_MODIFY_REFRESH, database, table);
|
|
|
|
break;
|
|
|
|
}
|
2020-03-24 17:05:38 +00:00
|
|
|
case ASTAlterCommand::RENAME_COLUMN:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_RENAME_COLUMN, database, table, column_name());
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
2020-01-24 16:20:36 +00:00
|
|
|
}
|
2021-08-27 06:30:21 +00:00
|
|
|
case ASTAlterCommand::MODIFY_DATABASE_SETTING:
|
|
|
|
{
|
|
|
|
required_access.emplace_back(AccessType::ALTER_DATABASE_SETTINGS, database, table);
|
|
|
|
break;
|
|
|
|
}
|
2020-03-06 17:13:28 +00:00
|
|
|
case ASTAlterCommand::NO_TYPE: break;
|
2021-09-20 11:06:19 +00:00
|
|
|
case ASTAlterCommand::MODIFY_COMMENT:
|
|
|
|
{
|
|
|
|
required_access.emplace_back(AccessType::ALTER_MODIFY_COMMENT, database, table);
|
|
|
|
break;
|
|
|
|
}
|
2024-02-28 00:00:17 +00:00
|
|
|
case ASTAlterCommand::MODIFY_SQL_SECURITY:
|
|
|
|
{
|
|
|
|
required_access.emplace_back(AccessType::ALTER_VIEW_MODIFY_SQL_SECURITY, database, table);
|
|
|
|
break;
|
|
|
|
}
|
2020-01-24 16:20:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return required_access;
|
|
|
|
}
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
void InterpreterAlterQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const
|
2020-12-14 03:30:39 +00:00
|
|
|
{
|
|
|
|
const auto & alter = ast->as<const ASTAlterQuery &>();
|
|
|
|
|
2021-08-27 06:30:21 +00:00
|
|
|
if (alter.command_list != nullptr && alter.alter_object != ASTAlterQuery::AlterObjectType::DATABASE)
|
2020-12-14 03:30:39 +00:00
|
|
|
{
|
|
|
|
// Alter queries already have their target table inserted into `elem`.
|
|
|
|
if (elem.query_tables.size() != 1)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Alter query should have target table recorded already");
|
2020-12-14 03:30:39 +00:00
|
|
|
|
|
|
|
String prefix = *elem.query_tables.begin() + ".";
|
|
|
|
for (const auto & child : alter.command_list->children)
|
|
|
|
{
|
|
|
|
const auto * command = child->as<ASTAlterCommand>();
|
|
|
|
|
|
|
|
if (command->column)
|
|
|
|
elem.query_columns.insert(prefix + command->column->getColumnName());
|
|
|
|
|
|
|
|
if (command->rename_to)
|
|
|
|
elem.query_columns.insert(prefix + command->rename_to->getColumnName());
|
|
|
|
|
|
|
|
// ADD COLUMN
|
|
|
|
if (command->col_decl)
|
|
|
|
{
|
|
|
|
elem.query_columns.insert(prefix + command->col_decl->as<ASTColumnDeclaration &>().name);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!command->from_table.empty())
|
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
String database = command->from_database.empty() ? getContext()->getCurrentDatabase() : command->from_database;
|
2020-12-14 03:30:39 +00:00
|
|
|
elem.query_databases.insert(database);
|
|
|
|
elem.query_tables.insert(database + "." + command->from_table);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!command->to_table.empty())
|
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
String database = command->to_database.empty() ? getContext()->getCurrentDatabase() : command->to_database;
|
2020-12-14 03:30:39 +00:00
|
|
|
elem.query_databases.insert(database);
|
|
|
|
elem.query_tables.insert(database + "." + command->to_table);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2024-01-09 06:33:48 +00:00
|
|
|
void registerInterpreterAlterQuery(InterpreterFactory & factory)
|
|
|
|
{
|
|
|
|
auto create_fn = [] (const InterpreterFactory::Arguments & args)
|
|
|
|
{
|
|
|
|
return std::make_unique<InterpreterAlterQuery>(args.query, args.context);
|
|
|
|
};
|
|
|
|
factory.registerInterpreter("InterpreterAlterQuery", create_fn);
|
|
|
|
}
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|