dbms: add partial validation of ALTER queries. [#METR-12739]

This commit is contained in:
Andrey Mironov 2014-10-07 13:09:59 +04:00
parent 220ce78326
commit f675bed75b
9 changed files with 169 additions and 33 deletions

View File

@ -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);
};
}

View File

@ -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)

View File

@ -41,4 +41,3 @@ public:
};
}

View File

@ -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);

View File

@ -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);

View File

@ -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)
{

View File

@ -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)
});
}
}
}

View File

@ -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);

View File

@ -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)
{