mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
support for IF EXISTS/IF NOT EXISTS in ALTER TABLE ADD/DROP/CLEAR/MODIFY/COMMENT COLUMN
This commit is contained in:
parent
d7ac2b3afd
commit
d776d1164a
@ -51,7 +51,7 @@ void ASTAlterCommand::formatImpl(
|
||||
|
||||
if (type == ASTAlterCommand::ADD_COLUMN)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD COLUMN " << (settings.hilite ? hilite_none : "");
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD COLUMN " << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "");
|
||||
col_decl->formatImpl(settings, state, frame);
|
||||
|
||||
/// AFTER
|
||||
@ -64,7 +64,7 @@ void ASTAlterCommand::formatImpl(
|
||||
else if (type == ASTAlterCommand::DROP_COLUMN)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str
|
||||
<< (clear_column ? "CLEAR " : "DROP ") << "COLUMN " << (settings.hilite ? hilite_none : "");
|
||||
<< (clear_column ? "CLEAR " : "DROP ") << "COLUMN " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
|
||||
column->formatImpl(settings, state, frame);
|
||||
if (partition)
|
||||
{
|
||||
@ -74,7 +74,7 @@ void ASTAlterCommand::formatImpl(
|
||||
}
|
||||
else if (type == ASTAlterCommand::MODIFY_COLUMN)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY COLUMN " << (settings.hilite ? hilite_none : "");
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY COLUMN " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
|
||||
col_decl->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (type == ASTAlterCommand::MODIFY_ORDER_BY)
|
||||
|
@ -78,6 +78,10 @@ public:
|
||||
|
||||
bool clear_column = false; /// for CLEAR COLUMN (do not drop column from metadata)
|
||||
|
||||
bool if_not_exists = false; /// option for ADD_COLUMN
|
||||
|
||||
bool if_exists = false; /// option for DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN
|
||||
|
||||
/** For FETCH PARTITION - the path in ZK to the shard, from which to download the partition.
|
||||
*/
|
||||
String from;
|
||||
|
@ -36,6 +36,8 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserKeyword s_partition("PARTITION");
|
||||
|
||||
ParserKeyword s_after("AFTER");
|
||||
ParserKeyword s_if_not_exists("IF NOT EXISTS");
|
||||
ParserKeyword s_if_exists("IF EXISTS");
|
||||
ParserKeyword s_from("FROM");
|
||||
ParserKeyword s_in_partition("IN PARTITION");
|
||||
ParserKeyword s_with("WITH");
|
||||
@ -57,6 +59,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
if (s_add_column.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_not_exists.ignore(pos, expected))
|
||||
command->if_not_exists = true;
|
||||
|
||||
if (!parser_col_decl.parse(pos, command->col_decl, expected))
|
||||
return false;
|
||||
|
||||
@ -77,6 +82,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_drop_column.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->column, expected))
|
||||
return false;
|
||||
|
||||
@ -85,6 +93,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_clear_column.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->column, expected))
|
||||
return false;
|
||||
|
||||
@ -190,6 +201,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_modify_column.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_modify_col_decl.parse(pos, command->col_decl, expected))
|
||||
return false;
|
||||
|
||||
@ -224,6 +238,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
}
|
||||
else if (s_comment_column.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->column, expected))
|
||||
return false;
|
||||
|
||||
|
@ -8,12 +8,12 @@ namespace DB
|
||||
|
||||
/** Query like this:
|
||||
* ALTER TABLE [db.]name [ON CLUSTER cluster]
|
||||
* [ADD COLUMN col_name type [AFTER col_after],]
|
||||
* [DROP COLUMN col_to_drop, ...]
|
||||
* [CLEAR COLUMN col_to_clear [IN PARTITION partition],]
|
||||
* [MODIFY COLUMN col_to_modify type, ...]
|
||||
* [ADD COLUMN [IF NOT EXISTS] col_name type [AFTER col_after],]
|
||||
* [DROP COLUMN [IF EXISTS] col_to_drop, ...]
|
||||
* [CLEAR COLUMN [IF EXISTS] col_to_clear [IN PARTITION partition],]
|
||||
* [MODIFY COLUMN [IF EXISTS] col_to_modify type, ...]
|
||||
* [MODIFY PRIMARY KEY (a, b, c...)]
|
||||
* [COMMENT COLUMN col_name string]
|
||||
* [COMMENT COLUMN [IF EXISTS] col_name string]
|
||||
* [DROP|DETACH|ATTACH PARTITION|PART partition, ...]
|
||||
* [FETCH PARTITION partition FROM ...]
|
||||
* [FREEZE [PARTITION] [WITH NAME name]]
|
||||
|
@ -52,6 +52,8 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
if (command_ast->column)
|
||||
command.after_column = typeid_cast<const ASTIdentifier &>(*command_ast->column).name;
|
||||
|
||||
command.if_not_exists = command_ast->if_not_exists;
|
||||
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::DROP_COLUMN && !command_ast->partition)
|
||||
@ -62,6 +64,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
AlterCommand command;
|
||||
command.type = AlterCommand::DROP_COLUMN;
|
||||
command.column_name = typeid_cast<const ASTIdentifier &>(*(command_ast->column)).name;
|
||||
command.if_exists = command_ast->if_exists;
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::MODIFY_COLUMN)
|
||||
@ -88,6 +91,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
const auto & ast_comment = typeid_cast<ASTLiteral &>(*ast_col_decl.comment);
|
||||
command.comment = ast_comment.value.get<String>();
|
||||
}
|
||||
command.if_exists = command_ast->if_exists;
|
||||
|
||||
return command;
|
||||
}
|
||||
@ -99,6 +103,7 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
command.column_name = ast_identifier.name;
|
||||
const auto & ast_comment = typeid_cast<ASTLiteral &>(*command_ast->comment);
|
||||
command.comment = ast_comment.value.get<String>();
|
||||
command.if_exists = command_ast->if_exists;
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::MODIFY_ORDER_BY)
|
||||
@ -300,7 +305,8 @@ void AlterCommands::apply(ColumnsDescription & columns_description, ASTPtr & ord
|
||||
auto new_primary_key_ast = primary_key_ast;
|
||||
|
||||
for (const AlterCommand & command : *this)
|
||||
command.apply(new_columns_description, new_order_by_ast, new_primary_key_ast);
|
||||
if (!command.ignore)
|
||||
command.apply(new_columns_description, new_order_by_ast, new_primary_key_ast);
|
||||
|
||||
columns_description = std::move(new_columns_description);
|
||||
order_by_ast = std::move(new_order_by_ast);
|
||||
@ -328,45 +334,61 @@ void AlterCommands::validate(const IStorage & table, const Context & context)
|
||||
if (command.type == AlterCommand::ADD_COLUMN)
|
||||
{
|
||||
if (std::end(all_columns) != column_it)
|
||||
throw Exception{"Cannot add column " + column_name + ": column with this name already exists", ErrorCodes::ILLEGAL_COLUMN};
|
||||
{
|
||||
if (command.if_not_exists)
|
||||
command.ignore = true;
|
||||
else
|
||||
throw Exception{"Cannot add column " + column_name + ": column with this name already exists", ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
}
|
||||
else if (command.type == AlterCommand::MODIFY_COLUMN)
|
||||
{
|
||||
|
||||
if (std::end(all_columns) == column_it)
|
||||
throw Exception{"Wrong column name. Cannot find column " + column_name + " to modify", ErrorCodes::ILLEGAL_COLUMN};
|
||||
{
|
||||
if (command.if_exists)
|
||||
command.ignore = true;
|
||||
else
|
||||
throw Exception{"Wrong column name. Cannot find column " + column_name + " to modify", ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
|
||||
all_columns.erase(column_it);
|
||||
defaults.erase(column_name);
|
||||
if (!command.ignore)
|
||||
{
|
||||
all_columns.erase(column_it);
|
||||
defaults.erase(column_name);
|
||||
}
|
||||
}
|
||||
|
||||
/// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions
|
||||
all_columns.emplace_back(column_name, command.data_type ? command.data_type : std::make_shared<DataTypeUInt8>());
|
||||
|
||||
if (command.default_expression)
|
||||
if (!command.ignore)
|
||||
{
|
||||
if (command.data_type)
|
||||
/// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions
|
||||
all_columns.emplace_back(column_name, command.data_type ? command.data_type : std::make_shared<DataTypeUInt8>());
|
||||
|
||||
if (command.default_expression)
|
||||
{
|
||||
const auto & final_column_name = column_name;
|
||||
const auto tmp_column_name = final_column_name + "_tmp";
|
||||
const auto column_type_raw_ptr = command.data_type.get();
|
||||
if (command.data_type)
|
||||
{
|
||||
const auto &final_column_name = column_name;
|
||||
const auto tmp_column_name = final_column_name + "_tmp";
|
||||
const auto column_type_raw_ptr = command.data_type.get();
|
||||
|
||||
default_expr_list->children.emplace_back(setAlias(
|
||||
makeASTFunction("CAST", std::make_shared<ASTIdentifier>(tmp_column_name),
|
||||
std::make_shared<ASTLiteral>(column_type_raw_ptr->getName())),
|
||||
final_column_name));
|
||||
default_expr_list->children.emplace_back(setAlias(
|
||||
makeASTFunction("CAST", std::make_shared<ASTIdentifier>(tmp_column_name),
|
||||
std::make_shared<ASTLiteral>(column_type_raw_ptr->getName())),
|
||||
final_column_name));
|
||||
|
||||
default_expr_list->children.emplace_back(setAlias(command.default_expression->clone(), tmp_column_name));
|
||||
default_expr_list->children.emplace_back(setAlias(command.default_expression->clone(), tmp_column_name));
|
||||
|
||||
defaulted_columns.emplace_back(NameAndTypePair{column_name, command.data_type}, &command);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// no type explicitly specified, will deduce later
|
||||
default_expr_list->children.emplace_back(
|
||||
setAlias(command.default_expression->clone(), column_name));
|
||||
defaulted_columns.emplace_back(NameAndTypePair{column_name, command.data_type}, &command);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// no type explicitly specified, will deduce later
|
||||
default_expr_list->children.emplace_back(
|
||||
setAlias(command.default_expression->clone(), column_name));
|
||||
|
||||
defaulted_columns.emplace_back(NameAndTypePair{column_name, nullptr}, &command);
|
||||
defaulted_columns.emplace_back(NameAndTypePair{column_name, nullptr}, &command);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -407,8 +429,13 @@ void AlterCommands::validate(const IStorage & table, const Context & context)
|
||||
}
|
||||
|
||||
if (!found)
|
||||
throw Exception("Wrong column name. Cannot find column " + command.column_name + " to drop",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
{
|
||||
if (command.if_exists)
|
||||
command.ignore = true;
|
||||
else
|
||||
throw Exception("Wrong column name. Cannot find column " + command.column_name + " to drop",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
else if (command.type == AlterCommand::COMMENT_COLUMN)
|
||||
{
|
||||
@ -416,7 +443,10 @@ void AlterCommands::validate(const IStorage & table, const Context & context)
|
||||
std::bind(namesEqual, std::cref(command.column_name), std::placeholders::_1));
|
||||
if (column_it == std::end(all_columns))
|
||||
{
|
||||
throw Exception{"Wrong column name. Cannot find column " + command.column_name + " to comment", ErrorCodes::ILLEGAL_COLUMN};
|
||||
if (command.if_exists)
|
||||
command.ignore = true;
|
||||
else
|
||||
throw Exception{"Wrong column name. Cannot find column " + command.column_name + " to comment", ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -43,15 +43,26 @@ struct AlterCommand
|
||||
/// For ADD - after which column to add a new one. If an empty string, add to the end. To add to the beginning now it is impossible.
|
||||
String after_column;
|
||||
|
||||
/// For DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN
|
||||
bool if_exists;
|
||||
|
||||
/// For ADD_COLUMN
|
||||
bool if_not_exists;
|
||||
|
||||
/// For MODIFY_ORDER_BY
|
||||
ASTPtr order_by;
|
||||
|
||||
/// indicates that this command should not be applied, for example in case of if_exists=true and column doesn't exist.
|
||||
bool ignore = false;
|
||||
|
||||
AlterCommand() = default;
|
||||
AlterCommand(const Type type, const String & column_name, const DataTypePtr & data_type,
|
||||
const ColumnDefaultKind default_kind, const ASTPtr & default_expression,
|
||||
const String & after_column = String{}, const String & comment = "") // TODO: разобраться здесь с параметром по умолчанию
|
||||
const String & after_column = String{}, const String & comment = "",
|
||||
const bool if_exists = false, const bool if_not_exists = false) // TODO: разобраться здесь с параметром по умолчанию
|
||||
: type{type}, column_name{column_name}, data_type{data_type}, default_kind{default_kind},
|
||||
default_expression{default_expression}, comment(comment), after_column{after_column}
|
||||
default_expression{default_expression}, comment(comment), after_column{after_column},
|
||||
if_exists(if_exists), if_not_exists(if_not_exists)
|
||||
{}
|
||||
|
||||
static std::optional<AlterCommand> parse(const ASTAlterCommand * command);
|
||||
|
@ -23,6 +23,13 @@ ALTER TABLE test.alter_test DROP COLUMN NestedColumn.S;
|
||||
|
||||
ALTER TABLE test.alter_test DROP COLUMN AddedNested1.B;
|
||||
|
||||
ALTER TABLE test.alter_test ADD COLUMN IF NOT EXISTS Added0 UInt32;
|
||||
ALTER TABLE test.alter_test ADD COLUMN IF NOT EXISTS AddedNested1 Nested(A UInt32, B UInt64);
|
||||
ALTER TABLE test.alter_test ADD COLUMN IF NOT EXISTS AddedNested1.C Array(String);
|
||||
ALTER TABLE test.alter_test MODIFY COLUMN IF EXISTS ToDrop UInt64;
|
||||
ALTER TABLE test.alter_test DROP COLUMN IF EXISTS ToDrop;
|
||||
ALTER TABLE test.alter_test COMMENT COLUMN IF EXISTS ToDrop 'new comment';
|
||||
|
||||
DESC TABLE test.alter_test;
|
||||
|
||||
SELECT * FROM test.alter_test;
|
||||
|
Loading…
Reference in New Issue
Block a user