提交 8e7c58dd 编写于 作者: A Alexey Zatelepin

parse ALTER UPDATE command [#CLICKHOUSE-13]

上级 8125bd3d
......@@ -393,6 +393,7 @@ namespace ErrorCodes
extern const int REPLICA_STATUS_CHANGED = 416;
extern const int EXPECTED_ALL_OR_ANY = 417;
extern const int UNKNOWN_JOIN_STRICTNESS = 418;
extern const int MULTIPLE_ASSIGNMENTS_TO_COLUMN = 419;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;
......
......@@ -133,6 +133,14 @@ void ASTAlterCommand::formatImpl(
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "DELETE WHERE " << (settings.hilite ? hilite_none : "");
predicate->formatImpl(settings, state, frame);
}
else if (type == ASTAlterCommand::UPDATE)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "UPDATE " << (settings.hilite ? hilite_none : "");
update_assignments->formatImpl(settings, state, frame);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : "");
predicate->formatImpl(settings, state, frame);
}
else
throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
}
......
......@@ -33,6 +33,7 @@ public:
FREEZE_PARTITION,
DELETE,
UPDATE,
NO_TYPE,
};
......@@ -59,9 +60,12 @@ public:
*/
ASTPtr partition;
/// For DELETE WHERE: the predicate that filters the rows to delete.
/// For DELETE/UPDATE WHERE: the predicate that filters the rows to delete/update.
ASTPtr predicate;
/// A list of expressions of the form `column = expr` for the UPDATE command.
ASTPtr update_assignments;
bool detach = false; /// true for DETACH PARTITION
bool part = false; /// true for ATTACH PART
......
#pragma once
#include <Parsers/IAST.h>
namespace DB
{
/// Part of the ALTER UPDATE statement of the form: column = expr
class ASTAssignment : public IAST
{
public:
String column_name;
ASTPtr expression;
String getID() const override { return "Assignment_" + 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);
}
return res;
}
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
settings.ostr << (settings.hilite ? hilite_identifier : "");
settings.writeIdentifier(column_name);
settings.ostr << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_operator : "") << " = " << (settings.hilite ? hilite_none : "");
expression->formatImpl(settings, state, frame);
}
};
}
......@@ -21,8 +21,6 @@ public:
ASTPtr clone() const override
{
const auto res = std::make_shared<ASTColumnDeclaration>(*this);
ASTPtr ptr{res};
res->children.clear();
if (type)
......@@ -37,7 +35,7 @@ public:
res->children.push_back(res->default_expression);
}
return ptr;
return res;
}
protected:
......
......@@ -11,11 +11,7 @@ void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, Form
auto format_element = [&](const String & name)
{
settings.ostr << (settings.hilite ? hilite_identifier : "");
WriteBufferFromOStream wb(settings.ostr, 32);
settings.writeIdentifier(name, wb);
wb.next();
settings.writeIdentifier(name);
settings.ostr << (settings.hilite ? hilite_none : "");
};
......
......@@ -9,11 +9,7 @@ namespace DB
void ASTWithAlias::writeAlias(const String & name, const FormatSettings & settings) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_alias : "");
WriteBufferFromOStream wb(settings.ostr, 32);
settings.writeIdentifier(name, wb);
wb.next();
settings.writeIdentifier(name);
settings.ostr << (settings.hilite ? hilite_none : "");
}
......@@ -25,8 +21,7 @@ void ASTWithAlias::formatImpl(const FormatSettings & settings, FormatState & sta
/// If we have previously output this node elsewhere in the query, now it is enough to output only the alias.
if (!state.printed_asts_with_alias.emplace(frame.current_select, alias).second)
{
WriteBufferFromOStream wb(settings.ostr, 32);
settings.writeIdentifier(alias, wb);
settings.writeIdentifier(alias);
return;
}
}
......
......@@ -99,8 +99,10 @@ String IAST::getColumnName() const
}
void IAST::FormatSettings::writeIdentifier(const String & name, WriteBuffer & out) const
void IAST::FormatSettings::writeIdentifier(const String & name) const
{
WriteBufferFromOStream out(ostr, 32);
switch (identifier_quoting_style)
{
case IdentifierQuotingStyle::None:
......@@ -128,6 +130,8 @@ void IAST::FormatSettings::writeIdentifier(const String & name, WriteBuffer & ou
break;
}
}
out.next();
}
}
......@@ -164,7 +164,7 @@ public:
nl_or_ws = one_line ? ' ' : '\n';
}
void writeIdentifier(const String & name, WriteBuffer & out) const;
void writeIdentifier(const String & name) const;
};
/// State. For example, a set of nodes can be remembered, which we already walk through.
......
#include <Parsers/ParserAlterQuery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ParserPartition.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTAssignment.h>
#include <Parsers/parseDatabaseAndTableName.h>
......@@ -38,12 +40,17 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
ParserKeyword s_name("NAME");
ParserKeyword s_delete_where("DELETE WHERE");
ParserKeyword s_update("UPDATE");
ParserKeyword s_where("WHERE");
ParserCompoundIdentifier parser_name;
ParserStringLiteral parser_string_literal;
ParserCompoundColumnDeclaration parser_col_decl;
ParserPartition parser_partition;
ParserExpression exp_elem;
ParserExpression parser_exp_elem;
ParserList parser_assignment_list(
std::make_unique<ParserAssignment>(), std::make_unique<ParserToken>(TokenType::Comma),
/* allow_empty = */ false);
if (s_add_column.ignore(pos, expected))
{
......@@ -195,11 +202,24 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
}
else if (s_delete_where.ignore(pos, expected))
{
if (!exp_elem.parse(pos, command->predicate, expected))
if (!parser_exp_elem.parse(pos, command->predicate, expected))
return false;
command->type = ASTAlterCommand::DELETE;
}
else if (s_update.ignore(pos, expected))
{
if (!parser_assignment_list.parse(pos, command->update_assignments, expected))
return false;
if (!s_where.ignore(pos, expected))
return false;
if (!parser_exp_elem.parse(pos, command->predicate, expected))
return false;
command->type = ASTAlterCommand::UPDATE;
}
else
return false;
......@@ -213,6 +233,8 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
command->children.push_back(command->partition);
if (command->predicate)
command->children.push_back(command->predicate);
if (command->update_assignments)
command->children.push_back(command->update_assignments);
return true;
}
......@@ -240,6 +262,33 @@ bool ParserAlterCommandList::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
}
bool ParserAssignment::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
auto assignment = std::make_shared<ASTAssignment>();
node = assignment;
ParserIdentifier p_identifier;
ParserToken s_equals(TokenType::Equals);
ParserExpression p_expression;
ASTPtr column;
if (!p_identifier.parse(pos, column, expected))
return false;
if (!s_equals.ignore(pos, expected))
return false;
if (!p_expression.parse(pos, assignment->expression, expected))
return false;
assignment->column_name = typeid_cast<const ASTIdentifier &>(*column).name;
if (assignment->expression)
assignment->children.push_back(assignment->expression);
return true;
}
bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
auto query = std::make_shared<ASTAlterQuery>();
......@@ -269,4 +318,5 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return true;
}
}
......@@ -17,12 +17,13 @@ namespace DB
* [FETCH PARTITION partition FROM ...]
* [FREEZE PARTITION]
* [DELETE WHERE ...]
* [UPDATE col_name = expr, ... WHERE ...]
*/
class ParserAlterCommand : public IParserBase
class ParserAlterQuery : public IParserBase
{
protected:
const char * getName() const { return "ALTER command"; }
const char * getName() const { return "ALTER query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
};
......@@ -35,10 +36,19 @@ protected:
};
class ParserAlterQuery : public IParserBase
class ParserAlterCommand : public IParserBase
{
protected:
const char * getName() const { return "ALTER query"; }
const char * getName() const { return "ALTER command"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
};
/// Part of the UPDATE command of the form: col_name = expr
class ParserAssignment : public IParserBase
{
protected:
const char * getName() const { return "column assignment"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
};
......
......@@ -7,6 +7,8 @@
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserAlterQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTAssignment.h>
#include <Common/typeid_cast.h>
......@@ -16,6 +18,7 @@ namespace DB
namespace ErrorCodes
{
extern const int UNKNOWN_MUTATION_COMMAND;
extern const int MULTIPLE_ASSIGNMENTS_TO_COLUMN;
}
std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command)
......@@ -28,6 +31,22 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command)
res.predicate = command->predicate;
return res;
}
else if (command->type == ASTAlterCommand::UPDATE)
{
MutationCommand res;
res.ast = command->ptr();
res.type = UPDATE;
res.predicate = command->predicate;
for (const ASTPtr & assignment_ast : command->update_assignments->children)
{
const auto & assignment = typeid_cast<const ASTAssignment &>(*assignment_ast);
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 `" + assignment.column_name + "`",
ErrorCodes::MULTIPLE_ASSIGNMENTS_TO_COLUMN);
}
return res;
}
else
return {};
}
......
......@@ -2,6 +2,7 @@
#include <Parsers/ASTAlterQuery.h>
#include <optional>
#include <unordered_map>
namespace DB
......@@ -20,12 +21,15 @@ struct MutationCommand
{
EMPTY, /// Not used.
DELETE,
UPDATE,
};
Type type = EMPTY;
ASTPtr predicate;
std::unordered_map<String, ASTPtr> column_to_update_expression;
static std::optional<MutationCommand> parse(ASTAlterCommand * command);
};
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册