mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Restrict select queries in default expressions. Unify default checks in ALTER and CREATE queries.
This commit is contained in:
parent
1aac3a5c9d
commit
a9edc5c3a4
@ -7,6 +7,7 @@
|
||||
#include <Interpreters/SyntaxAnalyzer.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/addTypeConversionToAST.h>
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
#include <Parsers/ASTColumnDeclaration.h>
|
||||
#include <Parsers/ASTConstraintDeclaration.h>
|
||||
@ -541,6 +542,8 @@ void AlterCommands::prepare(const StorageInMemoryMetadata & metadata)
|
||||
void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Context & context) const
|
||||
{
|
||||
auto all_columns = metadata.columns;
|
||||
/// Default expression for all added/modified.
|
||||
ASTPtr default_expr_list = std::make_shared<ASTExpressionList>();
|
||||
for (size_t i = 0; i < size(); ++i)
|
||||
{
|
||||
auto & command = (*this)[i];
|
||||
@ -561,9 +564,6 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con
|
||||
throw Exception{"Data type have to be specified for column " + backQuote(column_name) + " to add",
|
||||
ErrorCodes::BAD_ARGUMENTS};
|
||||
|
||||
if (command.default_expression)
|
||||
validateDefaultExpressionForColumn(command.default_expression, column_name, command.data_type, all_columns, context);
|
||||
|
||||
all_columns.add(ColumnDescription(column_name, command.data_type, false));
|
||||
}
|
||||
else if (command.type == AlterCommand::MODIFY_COLUMN)
|
||||
@ -576,22 +576,6 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con
|
||||
else
|
||||
continue;
|
||||
}
|
||||
|
||||
auto column_in_table = metadata.columns.get(column_name);
|
||||
if (command.default_expression)
|
||||
{
|
||||
if (!command.data_type)
|
||||
validateDefaultExpressionForColumn(
|
||||
command.default_expression, column_name, column_in_table.type, all_columns, context);
|
||||
else
|
||||
validateDefaultExpressionForColumn(
|
||||
command.default_expression, column_name, command.data_type, all_columns, context);
|
||||
}
|
||||
else if (column_in_table.default_desc.expression && command.data_type)
|
||||
{
|
||||
validateDefaultExpressionForColumn(
|
||||
column_in_table.default_desc.expression, column_name, command.data_type, all_columns, context);
|
||||
}
|
||||
}
|
||||
else if (command.type == AlterCommand::DROP_COLUMN)
|
||||
{
|
||||
@ -633,31 +617,52 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con
|
||||
if (metadata.settings_ast == nullptr)
|
||||
throw Exception{"Cannot alter settings, because table engine doesn't support settings changes", ErrorCodes::BAD_ARGUMENTS};
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void AlterCommands::validateDefaultExpressionForColumn(
|
||||
const ASTPtr default_expression,
|
||||
const String & column_name,
|
||||
const DataTypePtr column_type,
|
||||
const ColumnsDescription & all_columns,
|
||||
const Context & context) const
|
||||
{
|
||||
/// Collect default expressions for MODIFY and ADD comands
|
||||
if (command.type == AlterCommand::MODIFY_COLUMN || command.type == AlterCommand::ADD_COLUMN)
|
||||
{
|
||||
if (command.default_expression)
|
||||
{
|
||||
/// If we modify default, but not type
|
||||
if (!command.data_type)
|
||||
{
|
||||
default_expr_list->children.emplace_back(setAlias(command.default_expression->clone(), column_name));
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto & final_column_name = column_name;
|
||||
const auto tmp_column_name = final_column_name + "_tmp";
|
||||
const auto data_type_ptr = command.data_type;
|
||||
|
||||
try
|
||||
{
|
||||
String tmp_column_name = "__tmp" + column_name;
|
||||
auto copy_expression = default_expression->clone();
|
||||
auto default_with_cast = makeASTFunction("CAST", copy_expression, std::make_shared<ASTLiteral>(column_type->getName()));
|
||||
auto query_with_alias = setAlias(default_with_cast, tmp_column_name);
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(query_with_alias, all_columns.getAll());
|
||||
ExpressionAnalyzer(query_with_alias, syntax_result, context).getActions(true);
|
||||
}
|
||||
catch (Exception & ex)
|
||||
{
|
||||
ex.addMessage("default expression and column type are incompatible. Cannot alter column " + backQuote(column_name));
|
||||
throw;
|
||||
|
||||
default_expr_list->children.emplace_back(setAlias(
|
||||
addTypeConversionToAST(std::make_shared<ASTIdentifier>(tmp_column_name), data_type_ptr->getName()),
|
||||
final_column_name));
|
||||
|
||||
default_expr_list->children.emplace_back(setAlias(command.default_expression->clone(), tmp_column_name));
|
||||
}
|
||||
} /// if we change data type for column with default
|
||||
else if (metadata.columns.has(column_name) && command.data_type)
|
||||
{
|
||||
auto column_in_table = metadata.columns.get(column_name);
|
||||
/// Column doesn't have a default, nothing to check
|
||||
if (!column_in_table.default_desc.expression)
|
||||
continue;
|
||||
|
||||
const auto & final_column_name = column_name;
|
||||
const auto tmp_column_name = final_column_name + "_tmp";
|
||||
const auto data_type_ptr = command.data_type;
|
||||
|
||||
|
||||
default_expr_list->children.emplace_back(setAlias(
|
||||
addTypeConversionToAST(std::make_shared<ASTIdentifier>(tmp_column_name), data_type_ptr->getName()), final_column_name));
|
||||
|
||||
default_expr_list->children.emplace_back(setAlias(column_in_table.default_desc.expression->clone(), tmp_column_name));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
validateColumnsDefaultsAndGetSampleBlock(default_expr_list, all_columns.getAll(), context);
|
||||
}
|
||||
|
||||
bool AlterCommands::isModifyingData() const
|
||||
|
@ -117,16 +117,6 @@ class AlterCommands : public std::vector<AlterCommand>
|
||||
{
|
||||
private:
|
||||
bool prepared = false;
|
||||
private:
|
||||
|
||||
/// Validate that default expression and type are compatible, i.e. default
|
||||
/// expression result can be casted to column_type
|
||||
void validateDefaultExpressionForColumn(
|
||||
const ASTPtr default_expression,
|
||||
const String & column_name,
|
||||
const DataTypePtr column_type,
|
||||
const ColumnsDescription & all_columns,
|
||||
const Context & context) const;
|
||||
|
||||
public:
|
||||
/// Validate that commands can be applied to metadata.
|
||||
|
@ -433,14 +433,21 @@ Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const N
|
||||
if (child->as<ASTSelectQuery>() || child->as<ASTSubquery>())
|
||||
throw Exception("Select query is not allowed in columns DEFAULT expression", ErrorCodes::THERE_IS_NO_DEFAULT_VALUE);
|
||||
|
||||
auto syntax_analyzer_result = SyntaxAnalyzer(context).analyze(default_expr_list, all_columns);
|
||||
const auto actions = ExpressionAnalyzer(default_expr_list, syntax_analyzer_result, context).getActions(true);
|
||||
for (auto & action : actions->getActions())
|
||||
if (action.type == ExpressionAction::Type::JOIN || action.type == ExpressionAction::Type::ARRAY_JOIN)
|
||||
throw Exception(
|
||||
"Unsupported default value that requires ARRAY JOIN or JOIN action",
|
||||
ErrorCodes::THERE_IS_NO_DEFAULT_VALUE);
|
||||
try
|
||||
{
|
||||
auto syntax_analyzer_result = SyntaxAnalyzer(context).analyze(default_expr_list, all_columns);
|
||||
const auto actions = ExpressionAnalyzer(default_expr_list, syntax_analyzer_result, context).getActions(true);
|
||||
for (auto & action : actions->getActions())
|
||||
if (action.type == ExpressionAction::Type::JOIN || action.type == ExpressionAction::Type::ARRAY_JOIN)
|
||||
throw Exception("Unsupported default value that requires ARRAY JOIN or JOIN action", ErrorCodes::THERE_IS_NO_DEFAULT_VALUE);
|
||||
|
||||
return actions->getSampleBlock();
|
||||
return actions->getSampleBlock();
|
||||
}
|
||||
catch (Exception & ex)
|
||||
{
|
||||
ex.addMessage("default expression and column type are incompatible.");
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -114,5 +114,9 @@ private:
|
||||
Container columns;
|
||||
};
|
||||
|
||||
/// Validate default expressions and corresponding types compatibility, i.e.
|
||||
/// default expression result can be casted to column_type. Also checks, that we
|
||||
/// don't have strange constructions in default expression like SELECT query or
|
||||
/// arrayJoin function.
|
||||
Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const NamesAndTypesList & all_columns, const Context & context);
|
||||
}
|
||||
|
@ -5,7 +5,6 @@ col3 UInt64 MATERIALIZED col1 + 2
|
||||
col4 UInt64 ALIAS col1 + 3
|
||||
10 11
|
||||
12 13
|
||||
99
|
||||
payload String
|
||||
date Date MATERIALIZED today()
|
||||
key UInt64 MATERIALIZED 0 * rand()
|
||||
|
@ -11,10 +11,7 @@ select * from defaulted;
|
||||
select col3, col4 from defaulted;
|
||||
drop table defaulted;
|
||||
|
||||
create table defaulted (col1 Int8, col2 UInt64 default (SELECT dummy+99 from system.one)) engine=Memory;
|
||||
insert into defaulted (col1) values (0);
|
||||
select col2 from defaulted;
|
||||
drop table defaulted;
|
||||
create table defaulted (col1 Int8, col2 UInt64 default (SELECT dummy+99 from system.one)) engine=Memory; --{serverError 116}
|
||||
|
||||
create table defaulted (payload String, date materialized today(), key materialized 0 * rand()) engine=MergeTree(date, key, 8192);
|
||||
desc table defaulted;
|
||||
|
Loading…
Reference in New Issue
Block a user