Merge pull request #17563 from ClickHouse/parser-alter-missing-code

Allow query parameters in UPDATE statement.
This commit is contained in:
alexey-milovidov 2020-11-30 09:38:14 +03:00 committed by GitHub
commit df90cbd7d3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 58 additions and 15 deletions

View File

@ -10,21 +10,18 @@ class ASTAssignment : public IAST
{
public:
String column_name;
ASTPtr expression;
ASTPtr expression() const
{
return children.at(0);
}
String getID(char delim) const override { return "Assignment" + (delim + column_name); }
ASTPtr clone() const override
{
auto res = std::make_shared<ASTAssignment>(*this);
res->children.clear();
if (expression)
{
res->expression = expression->clone();
res->children.push_back(res->expression);
}
res->children = { expression()->clone() };
return res;
}
@ -37,7 +34,7 @@ protected:
settings.ostr << (settings.hilite ? hilite_operator : "") << " = " << (settings.hilite ? hilite_none : "");
expression->formatImpl(settings, state, frame);
expression()->formatImpl(settings, state, frame);
}
};

View File

@ -588,7 +588,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
}
else
return false;
}
if (command->col_decl)
@ -601,6 +600,14 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
command->children.push_back(command->order_by);
if (command->sample_by)
command->children.push_back(command->sample_by);
if (command->index_decl)
command->children.push_back(command->index_decl);
if (command->index)
command->children.push_back(command->index);
if (command->constraint_decl)
command->children.push_back(command->constraint_decl);
if (command->constraint)
command->children.push_back(command->constraint);
if (command->predicate)
command->children.push_back(command->predicate);
if (command->update_assignments)
@ -613,6 +620,10 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
command->children.push_back(command->ttl);
if (command->settings_changes)
command->children.push_back(command->settings_changes);
if (command->select)
command->children.push_back(command->select);
if (command->rename_to)
command->children.push_back(command->rename_to);
return true;
}
@ -656,12 +667,13 @@ bool ParserAssignment::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!s_equals.ignore(pos, expected))
return false;
if (!p_expression.parse(pos, assignment->expression, expected))
ASTPtr expression;
if (!p_expression.parse(pos, expression, expected))
return false;
tryGetIdentifierNameInto(column, assignment->column_name);
if (assignment->expression)
assignment->children.push_back(assignment->expression);
if (expression)
assignment->children.push_back(expression);
return true;
}

View File

@ -47,7 +47,7 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command,
for (const ASTPtr & assignment_ast : command->update_assignments->children)
{
const auto & assignment = assignment_ast->as<ASTAssignment &>();
auto insertion = res.column_to_update_expression.emplace(assignment.column_name, assignment.expression);
auto insertion = res.column_to_update_expression.emplace(assignment.column_name, assignment.expression());
if (!insertion.second)
throw Exception("Multiple assignments in the single statement to column " + backQuote(assignment.column_name),
ErrorCodes::MULTIPLE_ASSIGNMENTS_TO_COLUMN);

View File

@ -0,0 +1,2 @@
2 1
1 1

View File

@ -0,0 +1,32 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -n --query "
DROP TABLE IF EXISTS test;
CREATE TABLE test
(
id UUID,
date_time DateTime,
x UInt32,
y UInt32
) ENGINE = MergeTree()
PARTITION BY toYYYYMMDD(date_time)
ORDER BY (date_time);
INSERT INTO test (x, y) VALUES (2, 1);
"
$CLICKHOUSE_CLIENT --query "SELECT x, y FROM test"
$CLICKHOUSE_CLIENT --mutations_sync 1 --param_x 1 --param_y 1 --query "
ALTER TABLE test
UPDATE x = {x:UInt32}
WHERE y = {y:UInt32};
"
$CLICKHOUSE_CLIENT --query "SELECT x, y FROM test"
$CLICKHOUSE_CLIENT --query "DROP TABLE test"