Introduce columns transformers.

This commit is contained in:
Amos Bird 2020-08-29 13:33:46 +08:00
parent adf50fd774
commit be925f8d9c
No known key found for this signature in database
GPG Key ID: 80D430DCBECFEDB4
14 changed files with 515 additions and 6 deletions

View File

@ -18,6 +18,7 @@
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTColumnsMatcher.h>
#include <Parsers/ASTColumnsTransformers.h>
namespace DB
@ -135,8 +136,8 @@ void TranslateQualifiedNamesMatcher::visit(ASTFunction & node, const ASTPtr &, D
void TranslateQualifiedNamesMatcher::visit(const ASTQualifiedAsterisk &, const ASTPtr & ast, Data & data)
{
if (ast->children.size() != 1)
throw Exception("Logical error: qualified asterisk must have exactly one child", ErrorCodes::LOGICAL_ERROR);
if (ast->children.empty())
throw Exception("Logical error: qualified asterisk must have children", ErrorCodes::LOGICAL_ERROR);
auto & ident = ast->children[0];
@ -242,6 +243,10 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
first_table = false;
}
for (const auto & transformer : asterisk->children)
{
IASTColumnsTransformer::transform(transformer, node.children);
}
}
else if (const auto * asterisk_pattern = child->as<ASTColumnsMatcher>())
{
@ -258,6 +263,11 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
first_table = false;
}
// ColumnsMatcher's transformers start to appear at child 1
for (auto it = asterisk_pattern->children.begin() + 1; it != asterisk_pattern->children.end(); ++it)
{
IASTColumnsTransformer::transform(*it, node.children);
}
}
else if (const auto * qualified_asterisk = child->as<ASTQualifiedAsterisk>())
{
@ -274,6 +284,11 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
break;
}
}
// QualifiedAsterisk's transformers start to appear at child 1
for (auto it = qualified_asterisk->children.begin() + 1; it != qualified_asterisk->children.end(); ++it)
{
IASTColumnsTransformer::transform(*it, node.children);
}
}
else
node.children.emplace_back(child);

View File

@ -13,9 +13,14 @@ ASTPtr ASTAsterisk::clone() const
void ASTAsterisk::appendColumnName(WriteBuffer & ostr) const { ostr.write('*'); }
void ASTAsterisk::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
void ASTAsterisk::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << "*";
for (const auto & child : children)
{
settings.ostr << ' ';
child->formatImpl(settings, state, frame);
}
}
}

View File

@ -9,6 +9,9 @@ namespace DB
struct AsteriskSemantic;
struct AsteriskSemanticImpl;
/** SELECT * is expanded to all visible columns of the source table.
* Optional transformers can be attached to further manipulate these expanded columns.
*/
class ASTAsterisk : public IAST
{
public:

View File

@ -28,10 +28,15 @@ void ASTColumnsMatcher::updateTreeHashImpl(SipHash & hash_state) const
IAST::updateTreeHashImpl(hash_state);
}
void ASTColumnsMatcher::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
void ASTColumnsMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "COLUMNS" << (settings.hilite ? hilite_none : "") << "("
<< quoteString(original_pattern) << ")";
for (ASTs::const_iterator it = children.begin() + 1; it != children.end(); ++it)
{
settings.ostr << ' ';
(*it)->formatImpl(settings, state, frame);
}
}
void ASTColumnsMatcher::setPattern(String pattern)

View File

@ -23,6 +23,7 @@ struct AsteriskSemanticImpl;
/** SELECT COLUMNS('regexp') is expanded to multiple columns like * (asterisk).
* Optional transformers can be attached to further manipulate these expanded columns.
*/
class ASTColumnsMatcher : public IAST
{

View File

@ -0,0 +1,158 @@
#include "ASTColumnsTransformers.h"
#include <IO/WriteHelpers.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/SipHash.h>
#include <Common/quoteString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
void IASTColumnsTransformer::transform(const ASTPtr & transformer, ASTs & nodes)
{
if (const auto * apply = transformer->as<ASTColumnsApplyTransformer>())
{
apply->transform(nodes);
}
else if (const auto * except = transformer->as<ASTColumnsExceptTransformer>())
{
except->transform(nodes);
}
else if (const auto * replace = transformer->as<ASTColumnsReplaceTransformer>())
{
replace->transform(nodes);
}
}
void ASTColumnsApplyTransformer::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "APPLY" << (settings.hilite ? hilite_none : "") << "(" << func_name << ")";
}
void ASTColumnsApplyTransformer::transform(ASTs & nodes) const
{
for (auto & column : nodes)
{
column = makeASTFunction(func_name, column);
}
}
void ASTColumnsExceptTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "EXCEPT" << (settings.hilite ? hilite_none : "") << "(";
for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it)
{
if (it != children.begin())
{
settings.ostr << ", ";
}
(*it)->formatImpl(settings, state, frame);
}
settings.ostr << ")";
}
void ASTColumnsExceptTransformer::transform(ASTs & nodes) const
{
nodes.erase(
std::remove_if(
nodes.begin(),
nodes.end(),
[this](const ASTPtr & node_child)
{
if (const auto * id = node_child->as<ASTIdentifier>())
{
for (const auto & except_child : children)
{
if (except_child->as<const ASTIdentifier &>().name == id->shortName())
return true;
}
}
return false;
}),
nodes.end());
}
void ASTColumnsReplaceTransformer::Replacement::formatImpl(
const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
expr->formatImpl(settings, state, frame);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "") << name;
}
void ASTColumnsReplaceTransformer::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "REPLACE" << (settings.hilite ? hilite_none : "") << "(";
for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it)
{
if (it != children.begin())
{
settings.ostr << ", ";
}
(*it)->formatImpl(settings, state, frame);
}
settings.ostr << ")";
}
void ASTColumnsReplaceTransformer::replaceChildren(ASTPtr & node, const ASTPtr & replacement, const String & name)
{
for (auto & child : node->children)
{
if (const auto * id = child->as<ASTIdentifier>())
{
if (id->shortName() == name)
child = replacement;
}
else
replaceChildren(child, replacement, name);
}
}
void ASTColumnsReplaceTransformer::transform(ASTs & nodes) const
{
std::map<String, ASTPtr> replace_map;
for (const auto & replace_child : children)
{
auto & replacement = replace_child->as<Replacement &>();
if (replace_map.find(replacement.name) != replace_map.end())
throw Exception(
"Expressions in columns transformer REPLACE should not contain the same replacement more than once",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
replace_map.emplace(replacement.name, replacement.expr);
}
for (auto & column : nodes)
{
if (const auto * id = column->as<ASTIdentifier>())
{
auto replace_it = replace_map.find(id->shortName());
if (replace_it != replace_map.end())
{
column = replace_it->second;
column->setAlias(replace_it->first);
}
}
else if (auto * ast_with_alias = dynamic_cast<ASTWithAlias *>(column.get()))
{
auto replace_it = replace_map.find(ast_with_alias->alias);
if (replace_it != replace_map.end())
{
auto new_ast = replace_it->second->clone();
ast_with_alias->alias = ""; // remove the old alias as it's useless after replace transformation
replaceChildren(new_ast, column, replace_it->first);
column = new_ast;
column->setAlias(replace_it->first);
}
}
}
}
}

View File

@ -0,0 +1,85 @@
#pragma once
#include <Parsers/IAST.h>
namespace DB
{
class IASTColumnsTransformer : public IAST
{
public:
virtual void transform(ASTs & nodes) const = 0;
static void transform(const ASTPtr & transformer, ASTs & nodes);
};
class ASTColumnsApplyTransformer : public IASTColumnsTransformer
{
public:
String getID(char) const override { return "ColumnsApplyTransformer"; }
ASTPtr clone() const override
{
auto res = std::make_shared<ASTColumnsApplyTransformer>(*this);
return res;
}
void transform(ASTs & nodes) const override;
String func_name;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
class ASTColumnsExceptTransformer : public IASTColumnsTransformer
{
public:
String getID(char) const override { return "ColumnsExceptTransformer"; }
ASTPtr clone() const override
{
auto clone = std::make_shared<ASTColumnsExceptTransformer>(*this);
clone->cloneChildren();
return clone;
}
void transform(ASTs & nodes) const override;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
class ASTColumnsReplaceTransformer : public IASTColumnsTransformer
{
public:
class Replacement : public IAST
{
public:
String getID(char) const override { return "ColumnsReplaceTransformer::Replacement"; }
ASTPtr clone() const override
{
auto replacement = std::make_shared<Replacement>(*this);
replacement->name = name;
replacement->expr = expr->clone();
replacement->children.push_back(replacement->expr);
return replacement;
}
String name;
ASTPtr expr;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
String getID(char) const override { return "ColumnsReplaceTransformer"; }
ASTPtr clone() const override
{
auto clone = std::make_shared<ASTColumnsReplaceTransformer>(*this);
clone->cloneChildren();
return clone;
}
void transform(ASTs & nodes) const override;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
private:
static void replaceChildren(ASTPtr & node, const ASTPtr & replacement, const String & name);
};
}

View File

@ -16,6 +16,11 @@ void ASTQualifiedAsterisk::formatImpl(const FormatSettings & settings, FormatSta
const auto & qualifier = children.at(0);
qualifier->formatImpl(settings, state, frame);
settings.ostr << ".*";
for (ASTs::const_iterator it = children.begin() + 1; it != children.end(); ++it)
{
settings.ostr << ' ';
(*it)->formatImpl(settings, state, frame);
}
}
}

View File

@ -11,6 +11,7 @@ struct AsteriskSemanticImpl;
/** Something like t.*
* It will have qualifier as its child ASTIdentifier.
* Optional transformers can be attached to further manipulate these expanded columns.
*/
class ASTQualifiedAsterisk : public IAST
{

View File

@ -20,7 +20,9 @@
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
#include <Parsers/ASTColumnsTransformers.h>
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Parsers/parseIntervalKind.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserSelectWithUnionQuery.h>
@ -1172,17 +1174,131 @@ bool ParserColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
auto res = std::make_shared<ASTColumnsMatcher>();
res->setPattern(regex_node->as<ASTLiteral &>().value.get<String>());
res->children.push_back(regex_node);
ParserColumnsTransformers transformers_p;
ASTPtr transformer;
while (transformers_p.parse(pos, transformer, expected))
{
res->children.push_back(transformer);
}
node = std::move(res);
return true;
}
bool ParserAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected &)
bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword apply("APPLY");
ParserKeyword except("EXCEPT");
ParserKeyword replace("REPLACE");
ParserKeyword as("AS");
if (apply.ignore(pos, expected))
{
if (pos->type != TokenType::OpeningRoundBracket)
return false;
++pos;
String func_name;
if (!parseIdentifierOrStringLiteral(pos, expected, func_name))
return false;
if (pos->type != TokenType::ClosingRoundBracket)
return false;
++pos;
auto res = std::make_shared<ASTColumnsApplyTransformer>();
res->func_name = func_name;
node = std::move(res);
return true;
}
else if (except.ignore(pos, expected))
{
if (pos->type != TokenType::OpeningRoundBracket)
return false;
++pos;
ASTs identifiers;
auto parse_id = [&identifiers, &pos, &expected]
{
ASTPtr identifier;
if (!ParserIdentifier().parse(pos, identifier, expected))
return false;
identifiers.emplace_back(std::move(identifier));
return true;
};
if (!ParserList::parseUtil(pos, expected, parse_id, false))
return false;
if (pos->type != TokenType::ClosingRoundBracket)
return false;
++pos;
auto res = std::make_shared<ASTColumnsExceptTransformer>();
res->children = std::move(identifiers);
node = std::move(res);
return true;
}
else if (replace.ignore(pos, expected))
{
if (pos->type != TokenType::OpeningRoundBracket)
return false;
++pos;
ASTs replacements;
ParserExpression element_p;
ParserIdentifier ident_p;
auto parse_id = [&]
{
ASTPtr expr;
if (!element_p.parse(pos, expr, expected))
return false;
if (!as.ignore(pos, expected))
return false;
ASTPtr ident;
if (!ident_p.parse(pos, ident, expected))
return false;
auto replacement = std::make_shared<ASTColumnsReplaceTransformer::Replacement>();
replacement->name = getIdentifierName(ident);
replacement->expr = std::move(expr);
replacements.emplace_back(std::move(replacement));
return true;
};
if (!ParserList::parseUtil(pos, expected, parse_id, false))
return false;
if (pos->type != TokenType::ClosingRoundBracket)
return false;
++pos;
auto res = std::make_shared<ASTColumnsReplaceTransformer>();
res->children = std::move(replacements);
node = std::move(res);
return true;
}
return false;
}
bool ParserAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (pos->type == TokenType::Asterisk)
{
++pos;
node = std::make_shared<ASTAsterisk>();
auto asterisk = std::make_shared<ASTAsterisk>();
ParserColumnsTransformers transformers_p;
ASTPtr transformer;
while (transformers_p.parse(pos, transformer, expected))
{
asterisk->children.push_back(transformer);
}
node = asterisk;
return true;
}
return false;
@ -1204,6 +1320,12 @@ bool ParserQualifiedAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
auto res = std::make_shared<ASTQualifiedAsterisk>();
res->children.push_back(node);
ParserColumnsTransformers transformers_p;
ASTPtr transformer;
while (transformers_p.parse(pos, transformer, expected))
{
res->children.push_back(transformer);
}
node = std::move(res);
return true;
}

View File

@ -88,6 +88,15 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** *, t.*, db.table.*, COLUMNS('<regular expression>') APPLY(...) or EXCEPT(...) or REPLACE(...)
*/
class ParserColumnsTransformers : public IParserBase
{
protected:
const char * getName() const override { return "COLUMNS transformers"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** A function, for example, f(x, y + 1, g(z)).
* Or an aggregate function: sum(x + f(y)), corr(x, y). The syntax is the same as the usual function.
* Or a parametric aggregate function: quantile(0.9)(x + y).

View File

@ -10,6 +10,7 @@ SRCS(
ASTAsterisk.cpp
ASTColumnDeclaration.cpp
ASTColumnsMatcher.cpp
ASTColumnsTransformers.cpp
ASTConstraintDeclaration.cpp
ASTCreateQuery.cpp
ASTCreateQuotaQuery.cpp

View File

@ -0,0 +1,63 @@
220 18 347
110 9 173.5
1970-04-11 1970-01-11 1970-11-21
2 3
1 2
18 347
110 173.5
1970-04-11 1970-01-11 1970-11-21
222 18 347
111 11 173.5
1970-04-11 1970-01-11 1970-11-21
SELECT
sum(i),
sum(j),
sum(k)
FROM columns_transformers
SELECT
avg(i),
avg(j),
avg(k)
FROM columns_transformers
SELECT
toDate(any(i)),
toDate(any(j)),
toDate(any(k))
FROM columns_transformers AS a
SELECT
length(toString(j)),
length(toString(k))
FROM columns_transformers
SELECT
sum(j),
sum(k)
FROM columns_transformers
SELECT
avg(i),
avg(k)
FROM columns_transformers
SELECT
toDate(any(i)),
toDate(any(j)),
toDate(any(k))
FROM columns_transformers AS a
SELECT
sum(i + 1 AS i),
sum(j),
sum(k)
FROM columns_transformers
SELECT
avg(i + 1 AS i),
avg(j + 2 AS j),
avg(k)
FROM columns_transformers
SELECT
toDate(any(i)),
toDate(any(j)),
toDate(any(k))
FROM columns_transformers AS a
SELECT
(i + 1) + 1 AS i,
j,
k
FROM columns_transformers

View File

@ -0,0 +1,36 @@
DROP TABLE IF EXISTS columns_transformers;
CREATE TABLE columns_transformers (i Int64, j Int16, k Int64) Engine=TinyLog;
INSERT INTO columns_transformers VALUES (100, 10, 324), (120, 8, 23);
SELECT * APPLY(sum) from columns_transformers;
SELECT columns_transformers.* APPLY(avg) from columns_transformers;
SELECT a.* APPLY(toDate) APPLY(any) from columns_transformers a;
SELECT COLUMNS('[jk]') APPLY(toString) APPLY(length) from columns_transformers;
SELECT * EXCEPT(i) APPLY(sum) from columns_transformers;
SELECT columns_transformers.* EXCEPT(j) APPLY(avg) from columns_transformers;
-- EXCEPT after APPLY will not match anything
SELECT a.* APPLY(toDate) EXCEPT(i, j) APPLY(any) from columns_transformers a;
SELECT * REPLACE(i + 1 AS i) APPLY(sum) from columns_transformers;
SELECT columns_transformers.* REPLACE(j + 2 AS j, i + 1 AS i) APPLY(avg) from columns_transformers;
SELECT columns_transformers.* REPLACE(j + 1 AS j, j + 2 AS j) APPLY(avg) from columns_transformers; -- { serverError 43 }
-- REPLACE after APPLY will not match anything
SELECT a.* APPLY(toDate) REPLACE(i + 1 AS i) APPLY(any) from columns_transformers a;
EXPLAIN SYNTAX SELECT * APPLY(sum) from columns_transformers;
EXPLAIN SYNTAX SELECT columns_transformers.* APPLY(avg) from columns_transformers;
EXPLAIN SYNTAX SELECT a.* APPLY(toDate) APPLY(any) from columns_transformers a;
EXPLAIN SYNTAX SELECT COLUMNS('[jk]') APPLY(toString) APPLY(length) from columns_transformers;
EXPLAIN SYNTAX SELECT * EXCEPT(i) APPLY(sum) from columns_transformers;
EXPLAIN SYNTAX SELECT columns_transformers.* EXCEPT(j) APPLY(avg) from columns_transformers;
EXPLAIN SYNTAX SELECT a.* APPLY(toDate) EXCEPT(i, j) APPLY(any) from columns_transformers a;
EXPLAIN SYNTAX SELECT * REPLACE(i + 1 AS i) APPLY(sum) from columns_transformers;
EXPLAIN SYNTAX SELECT columns_transformers.* REPLACE(j + 2 AS j, i + 1 AS i) APPLY(avg) from columns_transformers;
EXPLAIN SYNTAX SELECT a.* APPLY(toDate) REPLACE(i + 1 AS i) APPLY(any) from columns_transformers a;
-- Multiple REPLACE in a row
EXPLAIN SYNTAX SELECT * REPLACE(i + 1 AS i) REPLACE(i + 1 AS i) from columns_transformers;
DROP TABLE columns_transformers;