mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
dbms: add partial validation of ALTER queries. [#METR-12739]
This commit is contained in:
parent
220ce78326
commit
f675bed75b
@ -53,10 +53,12 @@ private:
|
||||
typedef std::vector<PartitionCommand> PartitionCommands;
|
||||
|
||||
ASTPtr query_ptr;
|
||||
|
||||
|
||||
Context context;
|
||||
|
||||
static void parseAlter(const ASTAlterQuery::ParameterContainer & params, const DataTypeFactory & data_type_factory,
|
||||
AlterCommands & out_alter_commands, PartitionCommands & out_partition_commands);
|
||||
|
||||
static void validateColumnChanges(ASTAlterQuery::ParameterContainer & params, const StoragePtr & table, const Context & context);
|
||||
};
|
||||
}
|
||||
|
@ -36,7 +36,7 @@ public:
|
||||
* В запросе DROP это поле не используется
|
||||
* В запросе MODIFY здесь хранится имя столбца и новый тип
|
||||
*/
|
||||
ASTPtr name_type;
|
||||
ASTPtr col_decl;
|
||||
|
||||
/** В запросе ADD COLUMN здесь опционально хранится имя столбца, следующее после AFTER
|
||||
* В запросе DROP здесь хранится имя столбца для удаления
|
||||
@ -55,7 +55,7 @@ public:
|
||||
void clone(Parameters & p) const
|
||||
{
|
||||
p = *this;
|
||||
p.name_type = name_type->clone();
|
||||
p.col_decl = col_decl->clone();
|
||||
p.column = column->clone();
|
||||
p.partition = partition->clone();
|
||||
}
|
||||
@ -69,8 +69,8 @@ public:
|
||||
void addParameters(const Parameters & params)
|
||||
{
|
||||
parameters.push_back(params);
|
||||
if (params.name_type)
|
||||
children.push_back(params.name_type);
|
||||
if (params.col_decl)
|
||||
children.push_back(params.col_decl);
|
||||
if (params.column)
|
||||
children.push_back(params.column);
|
||||
if (params.partition)
|
||||
|
@ -41,4 +41,3 @@ public:
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <DB/Parsers/ASTColumnDeclaration.h>
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
#include <DB/Parsers/CommonParsers.h>
|
||||
#include <Poco/String.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -112,9 +113,9 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr
|
||||
NameParser name_parser;
|
||||
ParserIdentifierWithOptionalParameters type_parser;
|
||||
ParserWhiteSpaceOrComments ws;
|
||||
ParserString s_default{"DEFAULT"};
|
||||
ParserString s_materialized{"MATERIALIZED"};
|
||||
ParserString s_alias{"ALIAS"};
|
||||
ParserString s_default{"DEFAULT", true, true};
|
||||
ParserString s_materialized{"MATERIALIZED", true, true};
|
||||
ParserString s_alias{"ALIAS", true, true};
|
||||
ParserTernaryOperatorExpression expr_parser;
|
||||
|
||||
const auto begin = pos;
|
||||
@ -152,7 +153,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr
|
||||
s_materialized.ignore(pos, end, expected) ||
|
||||
s_alias.ignore(pos, end, expected))
|
||||
{
|
||||
default_specifier.assign(pos_before_specifier, pos);
|
||||
default_specifier = Poco::toUpper(std::string{pos_before_specifier, pos});
|
||||
|
||||
/// should be followed by an expression
|
||||
ws.ignore(pos, end, expected);
|
||||
|
@ -38,7 +38,7 @@ struct AlterCommand
|
||||
{
|
||||
if (type == ADD)
|
||||
{
|
||||
if (std::count_if(columns.begin(), columns.end(), std::bind(namesEqual, column_name, std::placeholders::_1)))
|
||||
if (std::count_if(columns.begin(), columns.end(), std::bind(namesEqual, std::cref(column_name), std::placeholders::_1)))
|
||||
throw Exception("Cannot add column " + column_name + ": column with this name already exisits.",
|
||||
DB::ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
@ -54,7 +54,7 @@ struct AlterCommand
|
||||
/// Например "fruits.bananas"
|
||||
/// одинаковыми считаются имена, если они совпадают целиком или name_without_dot совпадает с частью имени до точки
|
||||
NamesAndTypesList::reverse_iterator reverse_insert_it = std::find_if(columns.rbegin(), columns.rend(),
|
||||
std::bind(namesEqual, after_column, std::placeholders::_1));
|
||||
std::bind(namesEqual, std::cref(after_column), std::placeholders::_1));
|
||||
|
||||
if (reverse_insert_it == columns.rend())
|
||||
throw Exception("Wrong column name. Cannot find column " + column_name + " to insert after",
|
||||
@ -77,7 +77,8 @@ struct AlterCommand
|
||||
NamesAndTypesList::iterator column_it;
|
||||
do
|
||||
{
|
||||
column_it = std::find_if(columns.begin(), columns.end(), std::bind(namesEqual, column_name, std::placeholders::_1));
|
||||
column_it = std::find_if(columns.begin(), columns.end(),
|
||||
std::bind(namesEqual, std::cref(column_name), std::placeholders::_1));
|
||||
|
||||
if (column_it == columns.end())
|
||||
{
|
||||
@ -94,7 +95,7 @@ struct AlterCommand
|
||||
else if (type == MODIFY)
|
||||
{
|
||||
NamesAndTypesList::iterator column_it = std::find_if(columns.begin(), columns.end(),
|
||||
std::bind(namesEqual, column_name, std::placeholders::_1) );
|
||||
std::bind(namesEqual, std::cref(column_name), std::placeholders::_1) );
|
||||
if (column_it == columns.end())
|
||||
throw Exception("Wrong column name. Cannot find column " + column_name + " to modify.",
|
||||
DB::ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <DB/Interpreters/InterpreterAlterQuery.h>
|
||||
#include <DB/Interpreters/InterpreterCreateQuery.h>
|
||||
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
||||
#include <DB/Parsers/ASTAlterQuery.h>
|
||||
#include <DB/Parsers/ASTCreateQuery.h>
|
||||
#include <DB/Parsers/ASTExpressionList.h>
|
||||
@ -30,15 +31,16 @@ InterpreterAlterQuery::InterpreterAlterQuery(ASTPtr query_ptr_, Context & contex
|
||||
|
||||
void InterpreterAlterQuery::execute()
|
||||
{
|
||||
ASTAlterQuery & alter = typeid_cast<ASTAlterQuery &>(*query_ptr);
|
||||
String & table_name = alter.table;
|
||||
auto & alter = typeid_cast<ASTAlterQuery &>(*query_ptr);
|
||||
const String & table_name = alter.table;
|
||||
String database_name = alter.database.empty() ? context.getCurrentDatabase() : alter.database;
|
||||
StoragePtr table = context.getTable(database_name, table_name);
|
||||
validateColumnChanges(alter.parameters, table, context);
|
||||
|
||||
AlterCommands alter_commands;
|
||||
PartitionCommands partition_commands;
|
||||
parseAlter(alter.parameters, context.getDataTypeFactory(), alter_commands, partition_commands);
|
||||
|
||||
StoragePtr table = context.getTable(database_name, table_name);
|
||||
|
||||
for (const PartitionCommand & command : partition_commands)
|
||||
{
|
||||
if (command.type == PartitionCommand::DROP_PARTITION)
|
||||
@ -64,11 +66,11 @@ void InterpreterAlterQuery::parseAlter(
|
||||
AlterCommand command;
|
||||
command.type = AlterCommand::ADD;
|
||||
|
||||
const ASTNameTypePair & ast_name_type = typeid_cast<const ASTNameTypePair &>(*params.name_type);
|
||||
StringRange type_range = ast_name_type.type->range;
|
||||
const auto & ast_col_decl = typeid_cast<const ASTColumnDeclaration &>(*params.col_decl);
|
||||
StringRange type_range = ast_col_decl.type->range;
|
||||
String type_string = String(type_range.first, type_range.second - type_range.first);
|
||||
|
||||
command.column_name = ast_name_type.name;
|
||||
command.column_name = ast_col_decl.name;
|
||||
command.data_type = data_type_factory.get(type_string);
|
||||
|
||||
if (params.column)
|
||||
@ -89,11 +91,11 @@ void InterpreterAlterQuery::parseAlter(
|
||||
AlterCommand command;
|
||||
command.type = AlterCommand::MODIFY;
|
||||
|
||||
const ASTNameTypePair & ast_name_type = typeid_cast<const ASTNameTypePair &>(*params.name_type);
|
||||
StringRange type_range = ast_name_type.type->range;
|
||||
const auto & ast_col_decl = typeid_cast<const ASTColumnDeclaration &>(*params.col_decl);
|
||||
StringRange type_range = ast_col_decl.type->range;
|
||||
String type_string = String(type_range.first, type_range.second - type_range.first);
|
||||
|
||||
command.column_name = ast_name_type.name;
|
||||
command.column_name = ast_col_decl.name;
|
||||
command.data_type = data_type_factory.get(type_string);
|
||||
|
||||
out_alter_commands.push_back(command);
|
||||
@ -113,6 +115,141 @@ void InterpreterAlterQuery::parseAlter(
|
||||
}
|
||||
}
|
||||
|
||||
void InterpreterAlterQuery::validateColumnChanges(ASTAlterQuery::ParameterContainer & params_container, const StoragePtr & table, const Context & context)
|
||||
{
|
||||
auto columns = table->getColumnsList();
|
||||
columns.insert(std::end(columns), std::begin(table->materialized_columns), std::end(table->materialized_columns));
|
||||
columns.insert(std::end(columns), std::begin(table->alias_columns), std::end(table->alias_columns));
|
||||
auto defaults = table->column_defaults;
|
||||
|
||||
std::vector<std::pair<IDataType *, ASTColumnDeclaration *>> defaulted_columns{};
|
||||
|
||||
ASTPtr default_expr_list{new ASTExpressionList};
|
||||
default_expr_list->children.reserve(table->column_defaults.size());
|
||||
|
||||
for (auto & params : params_container)
|
||||
{
|
||||
if (params.type == ASTAlterQuery::ADD_COLUMN || params.type == ASTAlterQuery::MODIFY_COLUMN)
|
||||
{
|
||||
auto & col_decl = typeid_cast<ASTColumnDeclaration &>(*params.col_decl);
|
||||
const auto & column_name = col_decl.name;
|
||||
|
||||
if (params.type == ASTAlterQuery::MODIFY_COLUMN)
|
||||
{
|
||||
const auto it = std::find_if(std::begin(columns), std::end(columns),
|
||||
std::bind(AlterCommand::namesEqual, std::cref(column_name), std::placeholders::_1));
|
||||
|
||||
if (it == std::end(columns))
|
||||
throw Exception("Wrong column name. Cannot find column " + column_name + " to modify.",
|
||||
DB::ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
columns.erase(it);
|
||||
defaults.erase(column_name);
|
||||
}
|
||||
|
||||
if (col_decl.type)
|
||||
{
|
||||
const StringRange & type_range = col_decl.type->range;
|
||||
columns.emplace_back(col_decl.name,
|
||||
context.getDataTypeFactory().get({type_range.first, type_range.second}));
|
||||
}
|
||||
|
||||
if (col_decl.default_expression)
|
||||
{
|
||||
|
||||
if (col_decl.type)
|
||||
{
|
||||
const auto tmp_column_name = col_decl.name + "_tmp";
|
||||
const auto & final_column_name = col_decl.name;
|
||||
const auto conversion_function_name = "to" + columns.back().type->getName();
|
||||
|
||||
default_expr_list->children.emplace_back(setAlias(
|
||||
makeASTFunction(conversion_function_name, ASTPtr{new ASTIdentifier{{}, tmp_column_name}}),
|
||||
final_column_name));
|
||||
|
||||
default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), tmp_column_name));
|
||||
|
||||
defaulted_columns.emplace_back(columns.back().type.get(), &col_decl);
|
||||
}
|
||||
else
|
||||
{
|
||||
default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), col_decl.name));
|
||||
|
||||
defaulted_columns.emplace_back(nullptr, &col_decl);
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (params.type == ASTAlterQuery::DROP_COLUMN)
|
||||
{
|
||||
const auto & column_name = typeid_cast<const ASTIdentifier &>(*(params.column)).name;
|
||||
|
||||
auto found = false;
|
||||
for (auto it = std::begin(columns); it != std::end(columns);)
|
||||
if (AlterCommand::namesEqual(column_name, *it))
|
||||
{
|
||||
found = true;
|
||||
it = columns.erase(it);
|
||||
}
|
||||
else
|
||||
++it;
|
||||
|
||||
for (auto it = std::begin(defaults); it != std::end(defaults);)
|
||||
if (AlterCommand::namesEqual(column_name, { it->first, nullptr }))
|
||||
it = defaults.erase(it);
|
||||
else
|
||||
++it;
|
||||
|
||||
if (!found)
|
||||
throw Exception("Wrong column name. Cannot find column " + column_name + " to drop.",
|
||||
DB::ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
|
||||
for (const auto & col_def : defaults)
|
||||
default_expr_list->children.emplace_back(setAlias(col_def.second.expression->clone(), col_def.first));
|
||||
|
||||
const auto actions = ExpressionAnalyzer{default_expr_list, context, columns}.getActions(true);
|
||||
const auto block = actions->getSampleBlock();
|
||||
|
||||
for (auto & column : defaulted_columns)
|
||||
{
|
||||
const auto type_ptr = column.first;
|
||||
const auto col_decl_ptr = column.second;
|
||||
|
||||
if (type_ptr)
|
||||
{
|
||||
const auto & tmp_column = block.getByName(col_decl_ptr->name + "_tmp");
|
||||
|
||||
/// type mismatch between explicitly specified and deduced type, add conversion
|
||||
if (typeid(*type_ptr) != typeid(*tmp_column.type))
|
||||
{
|
||||
col_decl_ptr->default_expression = makeASTFunction(
|
||||
"to" + type_ptr->getName(),
|
||||
col_decl_ptr->default_expression);
|
||||
|
||||
col_decl_ptr->children.clear();
|
||||
col_decl_ptr->children.push_back(col_decl_ptr->type);
|
||||
col_decl_ptr->children.push_back(col_decl_ptr->default_expression);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
col_decl_ptr->type = new ASTIdentifier{};
|
||||
col_decl_ptr->query_string = new String{block.getByName(col_decl_ptr->name).type->getName()};
|
||||
col_decl_ptr->range = {
|
||||
col_decl_ptr->query_string->data(),
|
||||
col_decl_ptr->query_string->data() + col_decl_ptr->query_string->size()
|
||||
};
|
||||
static_cast<ASTIdentifier &>(*col_decl_ptr->type).name = *col_decl_ptr->query_string;
|
||||
}
|
||||
|
||||
defaults.emplace(col_decl_ptr->name, ColumnDefault{
|
||||
columnDefaultTypeFromString(col_decl_ptr->default_specifier),
|
||||
setAlias(col_decl_ptr->default_expression, col_decl_ptr->name)
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
void InterpreterAlterQuery::updateMetadata(
|
||||
const String & database_name, const String & table_name, const NamesAndTypesList & columns, Context & context)
|
||||
{
|
||||
|
@ -296,10 +296,6 @@ InterpreterCreateQuery::ColumnsAndDefaults InterpreterCreateQuery::parseColumns(
|
||||
else
|
||||
{
|
||||
default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), col_decl.name));
|
||||
defaults.emplace(col_decl.name, ColumnDefault{
|
||||
columnDefaultTypeFromString(col_decl.default_specifier),
|
||||
setAlias(col_decl.default_expression, col_decl.name)
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -32,7 +32,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & e
|
||||
|
||||
ParserIdentifier table_parser;
|
||||
ParserCompoundIdentifier parser_name;
|
||||
ParserCompoundNameTypePair parser_name_type;
|
||||
ParserCompoundColumnDeclaration parser_col_decl;
|
||||
ParserLiteral parser_literal;
|
||||
|
||||
ASTPtr table;
|
||||
@ -85,7 +85,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & e
|
||||
return false;
|
||||
ws.ignore(pos, end);
|
||||
|
||||
parser_name_type.parse(pos, end, params.name_type, expected);
|
||||
parser_col_decl.parse(pos, end, params.col_decl, expected);
|
||||
|
||||
ws.ignore(pos, end);
|
||||
if (s_after.ignore(pos, end, expected))
|
||||
@ -168,7 +168,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & e
|
||||
return false;
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (!parser_name_type.parse(pos, end, params.name_type, expected))
|
||||
if (!parser_col_decl.parse(pos, end, params.col_decl, expected))
|
||||
return false;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
@ -750,7 +750,7 @@ void formatAST(const ASTAlterQuery & ast, std::ostream & s, size_t indent, bo
|
||||
if (p.type == ASTAlterQuery::ADD_COLUMN)
|
||||
{
|
||||
s << (hilite ? hilite_keyword : "") << indent_str << "ADD COLUMN " << (hilite ? hilite_none : "");
|
||||
formatAST(*p.name_type, s, indent, hilite, true);
|
||||
formatAST(*p.col_decl, s, indent, hilite, true);
|
||||
|
||||
/// AFTER
|
||||
if (p.column)
|
||||
@ -767,7 +767,7 @@ void formatAST(const ASTAlterQuery & ast, std::ostream & s, size_t indent, bo
|
||||
else if (p.type == ASTAlterQuery::MODIFY_COLUMN)
|
||||
{
|
||||
s << (hilite ? hilite_keyword : "") << indent_str << "MODIFY COLUMN " << (hilite ? hilite_none : "");
|
||||
formatAST(*p.name_type, s, indent, hilite, true);
|
||||
formatAST(*p.col_decl, s, indent, hilite, true);
|
||||
}
|
||||
else if (p.type == ASTAlterQuery::DROP_PARTITION)
|
||||
{
|
||||
|
Loading…
Reference in New Issue
Block a user