2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/InterpreterAlterQuery.h>
|
2017-04-25 15:21:03 +00:00
|
|
|
#include <Interpreters/DDLWorker.h>
|
2018-09-04 13:45:39 +00:00
|
|
|
#include <Interpreters/MutationsInterpreter.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>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/ASTAlterQuery.h>
|
2020-01-24 16:20:36 +00:00
|
|
|
#include <Parsers/ASTAssignment.h>
|
2018-12-25 23:11:36 +00:00
|
|
|
#include <Storages/IStorage.h>
|
|
|
|
#include <Storages/AlterCommands.h>
|
|
|
|
#include <Storages/MutationCommands.h>
|
|
|
|
#include <Storages/PartitionCommands.h>
|
2019-08-22 23:22:57 +00:00
|
|
|
#include <Storages/LiveView/LiveViewCommands.h>
|
|
|
|
#include <Storages/LiveView/StorageLiveView.h>
|
2020-01-26 09:49:53 +00:00
|
|
|
#include <Access/AccessRightsElement.h>
|
2018-06-13 13:49:27 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2020-03-06 17:13:28 +00:00
|
|
|
#include <boost/range/algorithm_ext/push_back.hpp>
|
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
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2019-07-22 11:23:11 +00:00
|
|
|
extern const int SUPPORT_IS_DISABLED;
|
2020-01-22 13:24:20 +00:00
|
|
|
extern const int INCORRECT_QUERY;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
2013-09-23 12:01:19 +00:00
|
|
|
|
2017-05-23 18:01:50 +00:00
|
|
|
InterpreterAlterQuery::InterpreterAlterQuery(const ASTPtr & query_ptr_, const Context & context_)
|
2017-04-01 07:20:54 +00:00
|
|
|
: query_ptr(query_ptr_), context(context_)
|
2013-08-07 13:07:42 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2015-06-18 02:11:05 +00:00
|
|
|
BlockIO InterpreterAlterQuery::execute()
|
2013-12-18 11:19:37 +00:00
|
|
|
{
|
2019-03-15 16:14:13 +00:00
|
|
|
const auto & alter = query_ptr->as<ASTAlterQuery &>();
|
2017-04-25 15:21:03 +00:00
|
|
|
|
2019-03-15 16:14:13 +00:00
|
|
|
if (!alter.cluster.empty())
|
2020-01-24 16:20:36 +00:00
|
|
|
return executeDDLQueryOnCluster(query_ptr, context, getRequiredAccess());
|
|
|
|
|
|
|
|
context.checkAccess(getRequiredAccess());
|
2020-03-13 10:30:55 +00:00
|
|
|
auto table_id = context.resolveStorageID(alter, Context::ResolveOrdinary);
|
2020-05-28 23:01:18 +00:00
|
|
|
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context);
|
2020-06-16 15:51:29 +00:00
|
|
|
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-12-05 16:13:52 +00:00
|
|
|
/// Add default database to table identifiers that we can encounter in e.g. default expressions,
|
|
|
|
/// mutation expression, etc.
|
2020-02-17 19:28:25 +00:00
|
|
|
AddDefaultDatabaseVisitor visitor(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);
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
AlterCommands alter_commands;
|
|
|
|
PartitionCommands partition_commands;
|
2018-02-02 16:02:43 +00:00
|
|
|
MutationCommands mutation_commands;
|
2019-05-28 21:17:48 +00:00
|
|
|
LiveViewCommands live_view_commands;
|
2019-03-15 16:14:13 +00:00
|
|
|
for (ASTAlterCommand * command_ast : alter.command_list->commands)
|
2018-06-13 13:49:27 +00:00
|
|
|
{
|
2020-05-04 00:11:49 +00:00
|
|
|
if (auto alter_command = AlterCommand::parse(command_ast, !context.getSettingsRef().allow_suspicious_codecs))
|
2018-06-13 13:49:27 +00:00
|
|
|
alter_commands.emplace_back(std::move(*alter_command));
|
|
|
|
else if (auto partition_command = PartitionCommand::parse(command_ast))
|
2019-07-22 11:23:11 +00:00
|
|
|
{
|
|
|
|
if (partition_command->type == PartitionCommand::DROP_DETACHED_PARTITION
|
2019-08-08 16:08:43 +00:00
|
|
|
&& !context.getSettingsRef().allow_drop_detached)
|
2019-07-22 11:23:11 +00:00
|
|
|
throw DB::Exception("Cannot execute query: DROP DETACHED PART is disabled "
|
|
|
|
"(see allow_drop_detached setting)", ErrorCodes::SUPPORT_IS_DISABLED);
|
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
|
|
|
{
|
|
|
|
if (mut_command->type == MutationCommand::MATERIALIZE_TTL && !table->hasAnyTTL())
|
|
|
|
throw Exception("Cannot MATERIALIZE TTL as there is no TTL set for table "
|
|
|
|
+ table->getStorageID().getNameForLogs(), ErrorCodes::INCORRECT_QUERY);
|
|
|
|
|
2018-06-13 13:49:27 +00:00
|
|
|
mutation_commands.emplace_back(std::move(*mut_command));
|
2020-01-22 13:24:20 +00:00
|
|
|
}
|
2019-05-28 21:17:48 +00:00
|
|
|
else if (auto live_view_command = LiveViewCommand::parse(command_ast))
|
|
|
|
live_view_commands.emplace_back(std::move(*live_view_command));
|
2018-06-13 13:49:27 +00:00
|
|
|
else
|
|
|
|
throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
2018-02-02 16:02:43 +00:00
|
|
|
|
2018-06-13 13:49:27 +00:00
|
|
|
if (!mutation_commands.empty())
|
2018-02-02 16:02:43 +00:00
|
|
|
{
|
2020-04-09 18:10:27 +00:00
|
|
|
auto table_lock_holder = table->lockStructureForShare(
|
|
|
|
false /* because mutation is executed asyncronously */,
|
|
|
|
context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
2020-06-17 11:52:19 +00:00
|
|
|
MutationsInterpreter(table, metadata_snapshot, mutation_commands, context, false).validate(table_lock_holder);
|
2018-02-02 16:02:43 +00:00
|
|
|
table->mutate(mutation_commands, context);
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-11-13 13:48:53 +00:00
|
|
|
if (!partition_commands.empty())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-06-16 15:51:29 +00:00
|
|
|
table->alterPartition(query_ptr, metadata_snapshot, partition_commands, context);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2019-05-28 21:17:48 +00:00
|
|
|
if (!live_view_commands.empty())
|
|
|
|
{
|
2019-06-10 11:41:33 +00:00
|
|
|
live_view_commands.validate(*table);
|
2019-06-11 02:22:53 +00:00
|
|
|
for (const LiveViewCommand & command : live_view_commands)
|
2019-06-10 11:41:33 +00:00
|
|
|
{
|
|
|
|
auto live_view = std::dynamic_pointer_cast<StorageLiveView>(table);
|
|
|
|
switch (command.type)
|
|
|
|
{
|
|
|
|
case LiveViewCommand::REFRESH:
|
2020-04-01 12:43:09 +00:00
|
|
|
live_view->refresh(context);
|
2019-06-10 11:41:33 +00:00
|
|
|
break;
|
2019-05-28 21:17:48 +00:00
|
|
|
}
|
2019-06-10 11:41:33 +00:00
|
|
|
}
|
2019-05-28 21:17:48 +00:00
|
|
|
}
|
|
|
|
|
2018-02-02 16:02:43 +00:00
|
|
|
if (!alter_commands.empty())
|
|
|
|
{
|
2020-04-09 18:10:27 +00:00
|
|
|
auto table_lock_holder = table->lockAlterIntention(
|
|
|
|
context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
2019-12-26 18:17:05 +00:00
|
|
|
StorageInMemoryMetadata metadata = table->getInMemoryMetadata();
|
|
|
|
alter_commands.validate(metadata, context);
|
2020-02-20 10:04:15 +00:00
|
|
|
alter_commands.prepare(metadata);
|
2019-12-29 11:09:10 +00:00
|
|
|
table->checkAlterIsPossible(alter_commands, context.getSettingsRef());
|
2019-08-26 14:50:34 +00:00
|
|
|
table->alter(alter_commands, context, table_lock_holder);
|
2018-02-02 16:02:43 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
return {};
|
2013-12-18 11:19:37 +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-03-06 17:13:28 +00:00
|
|
|
for (ASTAlterCommand * command : alter.command_list->commands)
|
|
|
|
boost::range::push_back(required_access, getRequiredAccessForCommand(*command, alter.database, alter.table));
|
|
|
|
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::DELETE:
|
|
|
|
{
|
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::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;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::MODIFY_ORDER_BY:
|
|
|
|
{
|
|
|
|
required_access.emplace_back(AccessType::ALTER_ORDER_BY, database, table);
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
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;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::MODIFY_TTL:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_TTL, database, table);
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
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;
|
|
|
|
}
|
|
|
|
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;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::DROP_PARTITION: [[fallthrough]];
|
|
|
|
case ASTAlterCommand::DROP_DETACHED_PARTITION:
|
|
|
|
{
|
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:
|
|
|
|
{
|
2020-05-22 13:29:33 +00:00
|
|
|
if ((command.move_destination_type == DataDestinationType::DISK)
|
|
|
|
|| (command.move_destination_type == DataDestinationType::VOLUME))
|
2020-01-24 16:20:36 +00:00
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_MOVE_PARTITION, database, table);
|
2020-01-24 16:20:36 +00:00
|
|
|
}
|
2020-05-22 13:29:33 +00:00
|
|
|
else if (command.move_destination_type == DataDestinationType::TABLE)
|
2020-01-24 16:20:36 +00:00
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::SELECT | AccessType::ALTER_DELETE, database, table);
|
2020-03-06 17:13:28 +00:00
|
|
|
required_access.emplace_back(AccessType::INSERT, command.to_database, command.to_table);
|
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;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::FREEZE_PARTITION: [[fallthrough]];
|
|
|
|
case ASTAlterCommand::FREEZE_ALL:
|
|
|
|
{
|
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;
|
|
|
|
}
|
|
|
|
case ASTAlterCommand::LIVE_VIEW_REFRESH:
|
|
|
|
{
|
2020-04-03 11:54:50 +00:00
|
|
|
required_access.emplace_back(AccessType::ALTER_VIEW_REFRESH, database, table);
|
2020-03-06 17:13:28 +00:00
|
|
|
break;
|
2020-01-24 16:20:36 +00:00
|
|
|
}
|
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
|
|
|
}
|
2020-03-06 17:13:28 +00:00
|
|
|
case ASTAlterCommand::NO_TYPE: break;
|
2020-01-24 16:20:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return required_access;
|
|
|
|
}
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|