mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
Merge pull request #17563 from ClickHouse/parser-alter-missing-code
Allow query parameters in UPDATE statement.
This commit is contained in:
commit
df90cbd7d3
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -0,0 +1,2 @@
|
||||
2 1
|
||||
1 1
|
32
tests/queries/0_stateless/01599_mutation_query_params.sh
Executable file
32
tests/queries/0_stateless/01599_mutation_query_params.sh
Executable 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"
|
Loading…
Reference in New Issue
Block a user