mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #42884 from evillique/better_asterisk_parser
Improve Asterisk and ColumnMatcher parsers
This commit is contained in:
commit
88ba1b0b85
@ -11,6 +11,7 @@
|
||||
#include <Parsers/ASTQualifiedAsterisk.h>
|
||||
#include <Parsers/ASTColumnsMatcher.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTColumnsTransformers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -206,19 +207,43 @@ QueryTreeNodePtr MatcherNode::cloneImpl() const
|
||||
ASTPtr MatcherNode::toASTImpl() const
|
||||
{
|
||||
ASTPtr result;
|
||||
ASTPtr transformers;
|
||||
|
||||
if (!children.empty())
|
||||
{
|
||||
transformers = std::make_shared<ASTColumnsTransformerList>();
|
||||
|
||||
for (const auto & child : children)
|
||||
transformers->children.push_back(child->toAST());
|
||||
}
|
||||
|
||||
if (matcher_type == MatcherNodeType::ASTERISK)
|
||||
{
|
||||
if (qualified_identifier.empty())
|
||||
{
|
||||
result = std::make_shared<ASTAsterisk>();
|
||||
auto asterisk = std::make_shared<ASTAsterisk>();
|
||||
|
||||
if (transformers)
|
||||
{
|
||||
asterisk->transformers = std::move(transformers);
|
||||
asterisk->children.push_back(asterisk->transformers);
|
||||
}
|
||||
|
||||
result = asterisk;
|
||||
}
|
||||
else
|
||||
{
|
||||
auto qualified_asterisk = std::make_shared<ASTQualifiedAsterisk>();
|
||||
|
||||
auto identifier_parts = qualified_identifier.getParts();
|
||||
qualified_asterisk->children.push_back(std::make_shared<ASTIdentifier>(std::move(identifier_parts)));
|
||||
qualified_asterisk->qualifier = std::make_shared<ASTIdentifier>(std::move(identifier_parts));
|
||||
qualified_asterisk->children.push_back(qualified_asterisk->qualifier);
|
||||
|
||||
if (transformers)
|
||||
{
|
||||
qualified_asterisk->transformers = std::move(transformers);
|
||||
qualified_asterisk->children.push_back(qualified_asterisk->transformers);
|
||||
}
|
||||
|
||||
result = qualified_asterisk;
|
||||
}
|
||||
@ -229,6 +254,13 @@ ASTPtr MatcherNode::toASTImpl() const
|
||||
{
|
||||
auto regexp_matcher = std::make_shared<ASTColumnsRegexpMatcher>();
|
||||
regexp_matcher->setPattern(columns_matcher->pattern());
|
||||
|
||||
if (transformers)
|
||||
{
|
||||
regexp_matcher->transformers = std::move(transformers);
|
||||
regexp_matcher->children.push_back(regexp_matcher->transformers);
|
||||
}
|
||||
|
||||
result = regexp_matcher;
|
||||
}
|
||||
else
|
||||
@ -237,7 +269,14 @@ ASTPtr MatcherNode::toASTImpl() const
|
||||
regexp_matcher->setPattern(columns_matcher->pattern());
|
||||
|
||||
auto identifier_parts = qualified_identifier.getParts();
|
||||
regexp_matcher->children.push_back(std::make_shared<ASTIdentifier>(std::move(identifier_parts)));
|
||||
regexp_matcher->qualifier = std::make_shared<ASTIdentifier>(std::move(identifier_parts));
|
||||
regexp_matcher->children.push_back(regexp_matcher->qualifier);
|
||||
|
||||
if (transformers)
|
||||
{
|
||||
regexp_matcher->transformers = std::move(transformers);
|
||||
regexp_matcher->children.push_back(regexp_matcher->transformers);
|
||||
}
|
||||
|
||||
result = regexp_matcher;
|
||||
}
|
||||
@ -257,23 +296,36 @@ ASTPtr MatcherNode::toASTImpl() const
|
||||
{
|
||||
auto columns_list_matcher = std::make_shared<ASTColumnsListMatcher>();
|
||||
columns_list_matcher->column_list = std::move(column_list);
|
||||
columns_list_matcher->children.push_back(columns_list_matcher->column_list);
|
||||
|
||||
if (transformers)
|
||||
{
|
||||
columns_list_matcher->transformers = std::move(transformers);
|
||||
columns_list_matcher->children.push_back(columns_list_matcher->transformers);
|
||||
}
|
||||
|
||||
result = columns_list_matcher;
|
||||
}
|
||||
else
|
||||
{
|
||||
auto columns_list_matcher = std::make_shared<ASTQualifiedColumnsListMatcher>();
|
||||
columns_list_matcher->column_list = std::move(column_list);
|
||||
|
||||
auto identifier_parts = qualified_identifier.getParts();
|
||||
columns_list_matcher->children.push_back(std::make_shared<ASTIdentifier>(std::move(identifier_parts)));
|
||||
columns_list_matcher->qualifier = std::make_shared<ASTIdentifier>(std::move(identifier_parts));
|
||||
columns_list_matcher->column_list = std::move(column_list);
|
||||
columns_list_matcher->children.push_back(columns_list_matcher->qualifier);
|
||||
columns_list_matcher->children.push_back(columns_list_matcher->column_list);
|
||||
|
||||
if (transformers)
|
||||
{
|
||||
columns_list_matcher->transformers = std::move(transformers);
|
||||
columns_list_matcher->children.push_back(columns_list_matcher->transformers);
|
||||
}
|
||||
|
||||
result = columns_list_matcher;
|
||||
}
|
||||
}
|
||||
|
||||
for (const auto & child : children)
|
||||
result->children.push_back(child->toAST());
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
|
@ -111,7 +111,7 @@ private:
|
||||
|
||||
QueryTreeNodePtr buildJoinTree(const ASTPtr & tables_in_select_query, const ContextPtr & context) const;
|
||||
|
||||
ColumnTransformersNodes buildColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index, const ContextPtr & context) const;
|
||||
ColumnTransformersNodes buildColumnTransformers(const ASTPtr & matcher_expression, const ContextPtr & context) const;
|
||||
|
||||
ASTPtr query;
|
||||
QueryTreeNodePtr query_tree_node;
|
||||
@ -439,13 +439,13 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, co
|
||||
}
|
||||
else if (const auto * asterisk = expression->as<ASTAsterisk>())
|
||||
{
|
||||
auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/, context);
|
||||
auto column_transformers = buildColumnTransformers(asterisk->transformers, context);
|
||||
result = std::make_shared<MatcherNode>(std::move(column_transformers));
|
||||
}
|
||||
else if (const auto * qualified_asterisk = expression->as<ASTQualifiedAsterisk>())
|
||||
{
|
||||
auto & qualified_identifier = qualified_asterisk->children.at(0)->as<ASTTableIdentifier &>();
|
||||
auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/, context);
|
||||
auto & qualified_identifier = qualified_asterisk->qualifier->as<ASTIdentifier &>();
|
||||
auto column_transformers = buildColumnTransformers(qualified_asterisk->transformers, context);
|
||||
result = std::make_shared<MatcherNode>(Identifier(qualified_identifier.name_parts), std::move(column_transformers));
|
||||
}
|
||||
else if (const auto * ast_literal = expression->as<ASTLiteral>())
|
||||
@ -543,7 +543,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, co
|
||||
}
|
||||
else if (const auto * columns_regexp_matcher = expression->as<ASTColumnsRegexpMatcher>())
|
||||
{
|
||||
auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/, context);
|
||||
auto column_transformers = buildColumnTransformers(columns_regexp_matcher->transformers, context);
|
||||
result = std::make_shared<MatcherNode>(columns_regexp_matcher->getMatcher(), std::move(column_transformers));
|
||||
}
|
||||
else if (const auto * columns_list_matcher = expression->as<ASTColumnsListMatcher>())
|
||||
@ -557,18 +557,18 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, co
|
||||
column_list_identifiers.emplace_back(Identifier{column_list_identifier.name_parts});
|
||||
}
|
||||
|
||||
auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/, context);
|
||||
auto column_transformers = buildColumnTransformers(columns_list_matcher->transformers, context);
|
||||
result = std::make_shared<MatcherNode>(std::move(column_list_identifiers), std::move(column_transformers));
|
||||
}
|
||||
else if (const auto * qualified_columns_regexp_matcher = expression->as<ASTQualifiedColumnsRegexpMatcher>())
|
||||
{
|
||||
auto & qualified_identifier = qualified_columns_regexp_matcher->children.at(0)->as<ASTTableIdentifier &>();
|
||||
auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/, context);
|
||||
auto & qualified_identifier = qualified_columns_regexp_matcher->qualifier->as<ASTIdentifier &>();
|
||||
auto column_transformers = buildColumnTransformers(qualified_columns_regexp_matcher->transformers, context);
|
||||
result = std::make_shared<MatcherNode>(Identifier(qualified_identifier.name_parts), qualified_columns_regexp_matcher->getMatcher(), std::move(column_transformers));
|
||||
}
|
||||
else if (const auto * qualified_columns_list_matcher = expression->as<ASTQualifiedColumnsListMatcher>())
|
||||
{
|
||||
auto & qualified_identifier = qualified_columns_list_matcher->children.at(0)->as<ASTTableIdentifier &>();
|
||||
auto & qualified_identifier = qualified_columns_list_matcher->qualifier->as<ASTIdentifier &>();
|
||||
|
||||
Identifiers column_list_identifiers;
|
||||
column_list_identifiers.reserve(qualified_columns_list_matcher->column_list->children.size());
|
||||
@ -579,7 +579,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, co
|
||||
column_list_identifiers.emplace_back(Identifier{column_list_identifier.name_parts});
|
||||
}
|
||||
|
||||
auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/, context);
|
||||
auto column_transformers = buildColumnTransformers(qualified_columns_list_matcher->transformers, context);
|
||||
result = std::make_shared<MatcherNode>(Identifier(qualified_identifier.name_parts), std::move(column_list_identifiers), std::move(column_transformers));
|
||||
}
|
||||
else
|
||||
@ -833,15 +833,15 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select
|
||||
}
|
||||
|
||||
|
||||
ColumnTransformersNodes QueryTreeBuilder::buildColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index, const ContextPtr & context) const
|
||||
ColumnTransformersNodes QueryTreeBuilder::buildColumnTransformers(const ASTPtr & matcher_expression, const ContextPtr & context) const
|
||||
{
|
||||
ColumnTransformersNodes column_transformers;
|
||||
size_t children_size = matcher_expression->children.size();
|
||||
|
||||
for (; start_child_index < children_size; ++start_child_index)
|
||||
if (!matcher_expression)
|
||||
return column_transformers;
|
||||
|
||||
for (const auto & child : matcher_expression->children)
|
||||
{
|
||||
const auto & child = matcher_expression->children[start_child_index];
|
||||
|
||||
if (auto * apply_transformer = child->as<ASTColumnsApplyTransformer>())
|
||||
{
|
||||
if (apply_transformer->lambda)
|
||||
|
@ -28,13 +28,29 @@ DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTTableIdentifier &
|
||||
database = current_database;
|
||||
}
|
||||
|
||||
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database)
|
||||
{
|
||||
alias = identifier.tryGetAlias();
|
||||
|
||||
if (identifier.name_parts.size() == 2)
|
||||
std::tie(database, table) = std::tie(identifier.name_parts[0], identifier.name_parts[1]);
|
||||
else if (identifier.name_parts.size() == 1)
|
||||
table = identifier.name_parts[0];
|
||||
else
|
||||
throw Exception("Logical error: invalid identifier", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (database.empty())
|
||||
database = current_database;
|
||||
}
|
||||
|
||||
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTPtr & node, const String & current_database)
|
||||
{
|
||||
const auto * identifier = node->as<ASTTableIdentifier>();
|
||||
if (!identifier)
|
||||
throw Exception("Logical error: table identifier expected", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (const auto * table_identifier = node->as<ASTTableIdentifier>())
|
||||
*this = DatabaseAndTableWithAlias(*table_identifier, current_database);
|
||||
else if (const auto * identifier = node->as<ASTIdentifier>())
|
||||
*this = DatabaseAndTableWithAlias(*identifier, current_database);
|
||||
else
|
||||
throw Exception("Logical error: identifier or table identifier expected", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database)
|
||||
|
@ -14,6 +14,7 @@ namespace DB
|
||||
{
|
||||
|
||||
class ASTSelectQuery;
|
||||
class ASTIdentifier;
|
||||
class ASTTableIdentifier;
|
||||
struct ASTTableExpression;
|
||||
|
||||
@ -28,6 +29,7 @@ struct DatabaseAndTableWithAlias
|
||||
|
||||
DatabaseAndTableWithAlias() = default;
|
||||
explicit DatabaseAndTableWithAlias(const ASTPtr & identifier_node, const String & current_database = "");
|
||||
explicit DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database = "");
|
||||
explicit DatabaseAndTableWithAlias(const ASTTableIdentifier & identifier, const String & current_database = "");
|
||||
explicit DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database = "");
|
||||
|
||||
|
@ -49,7 +49,8 @@ ASTPtr makeSubqueryTemplate()
|
||||
ASTPtr makeSubqueryQualifiedAsterisk()
|
||||
{
|
||||
auto asterisk = std::make_shared<ASTQualifiedAsterisk>();
|
||||
asterisk->children.emplace_back(std::make_shared<ASTTableIdentifier>("--.s"));
|
||||
asterisk->qualifier = std::make_shared<ASTIdentifier>("--.s");
|
||||
asterisk->children.push_back(asterisk->qualifier);
|
||||
return asterisk;
|
||||
}
|
||||
|
||||
@ -153,26 +154,36 @@ private:
|
||||
for (auto & table_name : data.tables_order)
|
||||
data.addTableColumns(table_name, columns);
|
||||
|
||||
for (const auto & transformer : asterisk->children)
|
||||
if (asterisk->transformers)
|
||||
{
|
||||
for (const auto & transformer : asterisk->transformers->children)
|
||||
IASTColumnsTransformer::transform(transformer, columns);
|
||||
}
|
||||
}
|
||||
else if (const auto * qualified_asterisk = child->as<ASTQualifiedAsterisk>())
|
||||
{
|
||||
has_asterisks = true;
|
||||
|
||||
auto & identifier = child->children[0]->as<ASTTableIdentifier &>();
|
||||
if (!qualified_asterisk->qualifier)
|
||||
throw Exception("Logical error: qualified asterisk must have a qualifier", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto & identifier = qualified_asterisk->qualifier->as<ASTIdentifier &>();
|
||||
|
||||
data.addTableColumns(identifier.name(), columns);
|
||||
|
||||
// QualifiedAsterisk's transformers start to appear at child 1
|
||||
for (const auto * it = qualified_asterisk->children.begin() + 1; it != qualified_asterisk->children.end(); ++it)
|
||||
if (qualified_asterisk->transformers)
|
||||
{
|
||||
if (it->get()->as<ASTColumnsApplyTransformer>() || it->get()->as<ASTColumnsExceptTransformer>() || it->get()->as<ASTColumnsReplaceTransformer>())
|
||||
IASTColumnsTransformer::transform(*it, columns);
|
||||
for (const auto & transformer : qualified_asterisk->transformers->children)
|
||||
{
|
||||
if (transformer->as<ASTColumnsApplyTransformer>() ||
|
||||
transformer->as<ASTColumnsExceptTransformer>() ||
|
||||
transformer->as<ASTColumnsReplaceTransformer>())
|
||||
IASTColumnsTransformer::transform(transformer, columns);
|
||||
else
|
||||
throw Exception("Logical error: qualified asterisk must only have children of IASTColumnsTransformer type", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (const auto * columns_list_matcher = child->as<ASTColumnsListMatcher>())
|
||||
{
|
||||
has_asterisks = true;
|
||||
@ -180,9 +191,12 @@ private:
|
||||
for (const auto & ident : columns_list_matcher->column_list->children)
|
||||
columns.emplace_back(ident->clone());
|
||||
|
||||
for (const auto & transformer : columns_list_matcher->children)
|
||||
if (columns_list_matcher->transformers)
|
||||
{
|
||||
for (const auto & transformer : columns_list_matcher->transformers->children)
|
||||
IASTColumnsTransformer::transform(transformer, columns);
|
||||
}
|
||||
}
|
||||
else if (const auto * columns_regexp_matcher = child->as<ASTColumnsRegexpMatcher>())
|
||||
{
|
||||
has_asterisks = true;
|
||||
@ -193,9 +207,12 @@ private:
|
||||
columns,
|
||||
[&](const String & column_name) { return columns_regexp_matcher->isColumnMatching(column_name); });
|
||||
|
||||
for (const auto & transformer : columns_regexp_matcher->children)
|
||||
if (columns_regexp_matcher->transformers)
|
||||
{
|
||||
for (const auto & transformer : columns_regexp_matcher->transformers->children)
|
||||
IASTColumnsTransformer::transform(transformer, columns);
|
||||
}
|
||||
}
|
||||
else
|
||||
data.new_select_expression_list->children.push_back(child);
|
||||
|
||||
@ -425,6 +442,7 @@ private:
|
||||
{
|
||||
if (data.expression_list->children.empty())
|
||||
data.expression_list->children.emplace_back(std::make_shared<ASTAsterisk>());
|
||||
|
||||
select.setExpression(ASTSelectQuery::Expression::SELECT, std::move(data.expression_list));
|
||||
}
|
||||
data.done = true;
|
||||
|
@ -154,7 +154,7 @@ private:
|
||||
|
||||
static void visit(const ASTQualifiedAsterisk & node, const ASTPtr &, Data & data)
|
||||
{
|
||||
auto & identifier = node.children[0]->as<ASTTableIdentifier &>();
|
||||
auto & identifier = node.qualifier->as<ASTIdentifier &>();
|
||||
bool rewritten = false;
|
||||
for (const auto & table : data)
|
||||
{
|
||||
|
@ -156,21 +156,19 @@ void TranslateQualifiedNamesMatcher::visit(ASTFunction & node, const ASTPtr &, D
|
||||
func_arguments->children.clear();
|
||||
}
|
||||
|
||||
void TranslateQualifiedNamesMatcher::visit(const ASTQualifiedAsterisk &, const ASTPtr & ast, Data & data)
|
||||
void TranslateQualifiedNamesMatcher::visit(const ASTQualifiedAsterisk & node, const ASTPtr &, Data & data)
|
||||
{
|
||||
if (ast->children.empty())
|
||||
throw Exception("Logical error: qualified asterisk must have children", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto & ident = ast->children[0];
|
||||
if (!node.qualifier)
|
||||
throw Exception("Logical error: qualified asterisk must have a qualifier", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
/// @note it could contain table alias as table name.
|
||||
DatabaseAndTableWithAlias db_and_table(ident);
|
||||
DatabaseAndTableWithAlias db_and_table(node.qualifier);
|
||||
|
||||
for (const auto & known_table : data.tables)
|
||||
if (db_and_table.satisfies(known_table.table, true))
|
||||
return;
|
||||
|
||||
throw Exception("Unknown qualified identifier: " + ident->getAliasOrColumnName(), ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
throw Exception("Unknown qualified identifier: " + node.qualifier->getAliasOrColumnName(), ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
}
|
||||
|
||||
void TranslateQualifiedNamesMatcher::visit(ASTTableJoin & join, const ASTPtr & , Data & data)
|
||||
@ -266,17 +264,23 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
||||
first_table = false;
|
||||
}
|
||||
|
||||
for (const auto & transformer : asterisk->children)
|
||||
if (asterisk->transformers)
|
||||
{
|
||||
for (const auto & transformer : asterisk->transformers->children)
|
||||
IASTColumnsTransformer::transform(transformer, columns);
|
||||
}
|
||||
}
|
||||
else if (auto * asterisk_column_list = child->as<ASTColumnsListMatcher>())
|
||||
{
|
||||
for (const auto & ident : asterisk_column_list->column_list->children)
|
||||
columns.emplace_back(ident->clone());
|
||||
|
||||
for (const auto & transformer : asterisk_column_list->children)
|
||||
if (asterisk_column_list->transformers)
|
||||
{
|
||||
for (const auto & transformer : asterisk_column_list->transformers->children)
|
||||
IASTColumnsTransformer::transform(transformer, columns);
|
||||
}
|
||||
}
|
||||
else if (const auto * asterisk_regexp_pattern = child->as<ASTColumnsRegexpMatcher>())
|
||||
{
|
||||
bool first_table = true;
|
||||
@ -292,12 +296,15 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
||||
first_table = false;
|
||||
}
|
||||
|
||||
for (const auto & transformer : asterisk_regexp_pattern->children)
|
||||
if (asterisk_regexp_pattern->transformers)
|
||||
{
|
||||
for (const auto & transformer : asterisk_regexp_pattern->transformers->children)
|
||||
IASTColumnsTransformer::transform(transformer, columns);
|
||||
}
|
||||
}
|
||||
else if (const auto * qualified_asterisk = child->as<ASTQualifiedAsterisk>())
|
||||
{
|
||||
DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->children[0]);
|
||||
DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->qualifier);
|
||||
|
||||
for (const auto & table : tables_with_columns)
|
||||
{
|
||||
@ -309,10 +316,10 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
||||
}
|
||||
}
|
||||
|
||||
// QualifiedAsterisk's transformers start to appear at child 1
|
||||
for (const auto * it = qualified_asterisk->children.begin() + 1; it != qualified_asterisk->children.end(); ++it)
|
||||
if (qualified_asterisk->transformers)
|
||||
{
|
||||
IASTColumnsTransformer::transform(*it, columns);
|
||||
for (const auto & transformer : qualified_asterisk->transformers->children)
|
||||
IASTColumnsTransformer::transform(transformer, columns);
|
||||
}
|
||||
}
|
||||
else
|
||||
|
@ -8,21 +8,37 @@ namespace DB
|
||||
ASTPtr ASTAsterisk::clone() const
|
||||
{
|
||||
auto clone = std::make_shared<ASTAsterisk>(*this);
|
||||
clone->cloneChildren();
|
||||
|
||||
if (expression) { clone->expression = expression->clone(); clone->children.push_back(clone->expression); }
|
||||
if (transformers) { clone->transformers = transformers->clone(); clone->children.push_back(clone->transformers); }
|
||||
|
||||
return clone;
|
||||
}
|
||||
|
||||
void ASTAsterisk::appendColumnName(WriteBuffer & ostr) const { ostr.write('*'); }
|
||||
void ASTAsterisk::appendColumnName(WriteBuffer & ostr) const
|
||||
{
|
||||
if (expression)
|
||||
{
|
||||
expression->appendColumnName(ostr);
|
||||
writeCString(".", ostr);
|
||||
}
|
||||
|
||||
ostr.write('*');
|
||||
}
|
||||
|
||||
void ASTAsterisk::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
||||
{
|
||||
if (expression)
|
||||
{
|
||||
expression->formatImpl(settings, state, frame);
|
||||
settings.ostr << ".";
|
||||
}
|
||||
|
||||
settings.ostr << "*";
|
||||
|
||||
/// Format column transformers
|
||||
for (const auto & child : children)
|
||||
if (transformers)
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
child->formatImpl(settings, state, frame);
|
||||
transformers->formatImpl(settings, state, frame);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -16,6 +16,8 @@ public:
|
||||
ASTPtr clone() const override;
|
||||
void appendColumnName(WriteBuffer & ostr) const override;
|
||||
|
||||
ASTPtr expression;
|
||||
ASTPtr transformers;
|
||||
protected:
|
||||
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
|
||||
};
|
||||
|
@ -18,12 +18,20 @@ namespace ErrorCodes
|
||||
ASTPtr ASTColumnsRegexpMatcher::clone() const
|
||||
{
|
||||
auto clone = std::make_shared<ASTColumnsRegexpMatcher>(*this);
|
||||
clone->cloneChildren();
|
||||
|
||||
if (expression) { clone->expression = expression->clone(); clone->children.push_back(clone->expression); }
|
||||
if (transformers) { clone->transformers = transformers->clone(); clone->children.push_back(clone->transformers); }
|
||||
|
||||
return clone;
|
||||
}
|
||||
|
||||
void ASTColumnsRegexpMatcher::appendColumnName(WriteBuffer & ostr) const
|
||||
{
|
||||
if (expression)
|
||||
{
|
||||
expression->appendColumnName(ostr);
|
||||
writeCString(".", ostr);
|
||||
}
|
||||
writeCString("COLUMNS(", ostr);
|
||||
writeQuotedString(original_pattern, ostr);
|
||||
writeChar(')', ostr);
|
||||
@ -38,15 +46,21 @@ void ASTColumnsRegexpMatcher::updateTreeHashImpl(SipHash & hash_state) const
|
||||
|
||||
void ASTColumnsRegexpMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "");
|
||||
|
||||
if (expression)
|
||||
{
|
||||
expression->formatImpl(settings, state, frame);
|
||||
settings.ostr << ".";
|
||||
}
|
||||
|
||||
settings.ostr << "COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
|
||||
settings.ostr << quoteString(original_pattern);
|
||||
settings.ostr << ")";
|
||||
|
||||
/// Format column transformers
|
||||
for (const auto & child : children)
|
||||
if (transformers)
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
child->formatImpl(settings, state, frame);
|
||||
transformers->formatImpl(settings, state, frame);
|
||||
}
|
||||
}
|
||||
|
||||
@ -60,6 +74,11 @@ void ASTColumnsRegexpMatcher::setPattern(String pattern)
|
||||
DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
|
||||
}
|
||||
|
||||
const String & ASTColumnsRegexpMatcher::getPattern() const
|
||||
{
|
||||
return original_pattern;
|
||||
}
|
||||
|
||||
const std::shared_ptr<re2::RE2> & ASTColumnsRegexpMatcher::getMatcher() const
|
||||
{
|
||||
return column_matcher;
|
||||
@ -73,19 +92,23 @@ bool ASTColumnsRegexpMatcher::isColumnMatching(const String & column_name) const
|
||||
ASTPtr ASTColumnsListMatcher::clone() const
|
||||
{
|
||||
auto clone = std::make_shared<ASTColumnsListMatcher>(*this);
|
||||
clone->column_list = column_list->clone();
|
||||
clone->cloneChildren();
|
||||
return clone;
|
||||
}
|
||||
|
||||
void ASTColumnsListMatcher::updateTreeHashImpl(SipHash & hash_state) const
|
||||
{
|
||||
column_list->updateTreeHash(hash_state);
|
||||
IAST::updateTreeHashImpl(hash_state);
|
||||
if (expression) { clone->expression = expression->clone(); clone->children.push_back(clone->expression); }
|
||||
if (transformers) { clone->transformers = transformers->clone(); clone->children.push_back(clone->transformers); }
|
||||
|
||||
clone->column_list = column_list->clone();
|
||||
clone->children.push_back(clone->column_list);
|
||||
|
||||
return clone;
|
||||
}
|
||||
|
||||
void ASTColumnsListMatcher::appendColumnName(WriteBuffer & ostr) const
|
||||
{
|
||||
if (expression)
|
||||
{
|
||||
expression->appendColumnName(ostr);
|
||||
writeCString(".", ostr);
|
||||
}
|
||||
writeCString("COLUMNS(", ostr);
|
||||
for (auto * it = column_list->children.begin(); it != column_list->children.end(); ++it)
|
||||
{
|
||||
@ -99,7 +122,15 @@ void ASTColumnsListMatcher::appendColumnName(WriteBuffer & ostr) const
|
||||
|
||||
void ASTColumnsListMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "");
|
||||
|
||||
if (expression)
|
||||
{
|
||||
expression->formatImpl(settings, state, frame);
|
||||
settings.ostr << ".";
|
||||
}
|
||||
|
||||
settings.ostr << "COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
|
||||
|
||||
for (ASTs::const_iterator it = column_list->children.begin(); it != column_list->children.end(); ++it)
|
||||
{
|
||||
@ -111,33 +142,39 @@ void ASTColumnsListMatcher::formatImpl(const FormatSettings & settings, FormatSt
|
||||
}
|
||||
settings.ostr << ")";
|
||||
|
||||
/// Format column transformers
|
||||
for (const auto & child : children)
|
||||
if (transformers)
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
child->formatImpl(settings, state, frame);
|
||||
transformers->formatImpl(settings, state, frame);
|
||||
}
|
||||
}
|
||||
|
||||
ASTPtr ASTQualifiedColumnsRegexpMatcher::clone() const
|
||||
{
|
||||
auto clone = std::make_shared<ASTQualifiedColumnsRegexpMatcher>(*this);
|
||||
clone->cloneChildren();
|
||||
|
||||
if (transformers) { clone->transformers = transformers->clone(); clone->children.push_back(clone->transformers); }
|
||||
|
||||
clone->qualifier = qualifier->clone();
|
||||
clone->children.push_back(clone->qualifier);
|
||||
|
||||
return clone;
|
||||
}
|
||||
|
||||
void ASTQualifiedColumnsRegexpMatcher::appendColumnName(WriteBuffer & ostr) const
|
||||
{
|
||||
const auto & qualifier = children.at(0);
|
||||
qualifier->appendColumnName(ostr);
|
||||
writeCString(".COLUMNS(", ostr);
|
||||
writeQuotedString(original_pattern, ostr);
|
||||
writeChar(')', ostr);
|
||||
}
|
||||
|
||||
void ASTQualifiedColumnsRegexpMatcher::setPattern(String pattern)
|
||||
void ASTQualifiedColumnsRegexpMatcher::setPattern(String pattern, bool set_matcher)
|
||||
{
|
||||
original_pattern = std::move(pattern);
|
||||
|
||||
if (!set_matcher)
|
||||
return;
|
||||
|
||||
column_matcher = std::make_shared<RE2>(original_pattern, RE2::Quiet);
|
||||
if (!column_matcher->ok())
|
||||
throw DB::Exception(
|
||||
@ -166,35 +203,35 @@ void ASTQualifiedColumnsRegexpMatcher::formatImpl(const FormatSettings & setting
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "");
|
||||
|
||||
const auto & qualifier = children.at(0);
|
||||
qualifier->formatImpl(settings, state, frame);
|
||||
|
||||
settings.ostr << ".COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
|
||||
settings.ostr << quoteString(original_pattern);
|
||||
settings.ostr << ")";
|
||||
|
||||
/// Format column transformers
|
||||
size_t children_size = children.size();
|
||||
|
||||
for (size_t i = 1; i < children_size; ++i)
|
||||
if (transformers)
|
||||
{
|
||||
const auto & child = children[i];
|
||||
settings.ostr << ' ';
|
||||
child->formatImpl(settings, state, frame);
|
||||
transformers->formatImpl(settings, state, frame);
|
||||
}
|
||||
}
|
||||
|
||||
ASTPtr ASTQualifiedColumnsListMatcher::clone() const
|
||||
{
|
||||
auto clone = std::make_shared<ASTQualifiedColumnsListMatcher>(*this);
|
||||
|
||||
if (transformers) { clone->transformers = transformers->clone(); clone->children.push_back(clone->transformers); }
|
||||
|
||||
clone->qualifier = qualifier->clone();
|
||||
clone->column_list = column_list->clone();
|
||||
clone->cloneChildren();
|
||||
|
||||
clone->children.push_back(clone->qualifier);
|
||||
clone->children.push_back(clone->column_list);
|
||||
|
||||
return clone;
|
||||
}
|
||||
|
||||
void ASTQualifiedColumnsListMatcher::appendColumnName(WriteBuffer & ostr) const
|
||||
{
|
||||
const auto & qualifier = children.at(0);
|
||||
qualifier->appendColumnName(ostr);
|
||||
writeCString(".COLUMNS(", ostr);
|
||||
|
||||
@ -208,19 +245,10 @@ void ASTQualifiedColumnsListMatcher::appendColumnName(WriteBuffer & ostr) const
|
||||
writeChar(')', ostr);
|
||||
}
|
||||
|
||||
void ASTQualifiedColumnsListMatcher::updateTreeHashImpl(SipHash & hash_state) const
|
||||
{
|
||||
column_list->updateTreeHash(hash_state);
|
||||
IAST::updateTreeHashImpl(hash_state);
|
||||
}
|
||||
|
||||
void ASTQualifiedColumnsListMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "");
|
||||
|
||||
const auto & qualifier = children.at(0);
|
||||
qualifier->formatImpl(settings, state, frame);
|
||||
|
||||
settings.ostr << ".COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
|
||||
|
||||
for (ASTs::const_iterator it = column_list->children.begin(); it != column_list->children.end(); ++it)
|
||||
@ -232,14 +260,9 @@ void ASTQualifiedColumnsListMatcher::formatImpl(const FormatSettings & settings,
|
||||
}
|
||||
settings.ostr << ")";
|
||||
|
||||
/// Format column transformers
|
||||
size_t children_size = children.size();
|
||||
|
||||
for (size_t i = 1; i < children_size; ++i)
|
||||
if (transformers)
|
||||
{
|
||||
const auto & child = children[i];
|
||||
settings.ostr << ' ';
|
||||
child->formatImpl(settings, state, frame);
|
||||
transformers->formatImpl(settings, state, frame);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -24,10 +24,13 @@ public:
|
||||
|
||||
void appendColumnName(WriteBuffer & ostr) const override;
|
||||
void setPattern(String pattern);
|
||||
const String & getPattern() const;
|
||||
const std::shared_ptr<re2::RE2> & getMatcher() const;
|
||||
bool isColumnMatching(const String & column_name) const;
|
||||
void updateTreeHashImpl(SipHash & hash_state) const override;
|
||||
|
||||
ASTPtr expression;
|
||||
ASTPtr transformers;
|
||||
protected:
|
||||
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
|
||||
|
||||
@ -43,9 +46,10 @@ public:
|
||||
String getID(char) const override { return "ColumnsListMatcher"; }
|
||||
ASTPtr clone() const override;
|
||||
void appendColumnName(WriteBuffer & ostr) const override;
|
||||
void updateTreeHashImpl(SipHash & hash_state) const override;
|
||||
|
||||
ASTPtr expression;
|
||||
ASTPtr column_list;
|
||||
ASTPtr transformers;
|
||||
protected:
|
||||
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
|
||||
};
|
||||
@ -59,10 +63,12 @@ public:
|
||||
|
||||
void appendColumnName(WriteBuffer & ostr) const override;
|
||||
const std::shared_ptr<re2::RE2> & getMatcher() const;
|
||||
void setPattern(String pattern);
|
||||
void setPattern(String pattern, bool set_matcher = true);
|
||||
void setMatcher(std::shared_ptr<re2::RE2> matcher);
|
||||
void updateTreeHashImpl(SipHash & hash_state) const override;
|
||||
|
||||
ASTPtr qualifier;
|
||||
ASTPtr transformers;
|
||||
protected:
|
||||
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
|
||||
|
||||
@ -78,9 +84,10 @@ public:
|
||||
String getID(char) const override { return "QualifiedColumnsListMatcher"; }
|
||||
ASTPtr clone() const override;
|
||||
void appendColumnName(WriteBuffer & ostr) const override;
|
||||
void updateTreeHashImpl(SipHash & hash_state) const override;
|
||||
|
||||
ASTPtr qualifier;
|
||||
ASTPtr column_list;
|
||||
ASTPtr transformers;
|
||||
protected:
|
||||
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
|
||||
};
|
||||
|
@ -19,6 +19,15 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_COMPILE_REGEXP;
|
||||
}
|
||||
|
||||
void ASTColumnsTransformerList::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
||||
{
|
||||
for (const auto & child : children)
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
child->formatImpl(settings, state, frame);
|
||||
}
|
||||
}
|
||||
|
||||
void IASTColumnsTransformer::transform(const ASTPtr & transformer, ASTs & nodes)
|
||||
{
|
||||
if (const auto * apply = transformer->as<ASTColumnsApplyTransformer>())
|
||||
|
@ -9,6 +9,23 @@ namespace re2
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// A list of column transformers
|
||||
class ASTColumnsTransformerList : public IAST
|
||||
{
|
||||
public:
|
||||
String getID(char) const override { return "ColumnsTransformerList"; }
|
||||
ASTPtr clone() const override
|
||||
{
|
||||
auto clone = std::make_shared<ASTColumnsTransformerList>(*this);
|
||||
clone->cloneChildren();
|
||||
return clone;
|
||||
}
|
||||
|
||||
protected:
|
||||
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
};
|
||||
|
||||
class IASTColumnsTransformer : public IAST
|
||||
{
|
||||
public:
|
||||
|
@ -7,22 +7,18 @@ namespace DB
|
||||
|
||||
void ASTQualifiedAsterisk::appendColumnName(WriteBuffer & ostr) const
|
||||
{
|
||||
const auto & qualifier = children.at(0);
|
||||
qualifier->appendColumnName(ostr);
|
||||
writeCString(".*", ostr);
|
||||
}
|
||||
|
||||
void ASTQualifiedAsterisk::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
||||
{
|
||||
const auto & qualifier = children.at(0);
|
||||
qualifier->formatImpl(settings, state, frame);
|
||||
settings.ostr << ".*";
|
||||
|
||||
/// Format column transformers
|
||||
for (ASTs::const_iterator it = children.begin() + 1; it != children.end(); ++it)
|
||||
if (transformers)
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
(*it)->formatImpl(settings, state, frame);
|
||||
transformers->formatImpl(settings, state, frame);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -17,11 +17,18 @@ public:
|
||||
ASTPtr clone() const override
|
||||
{
|
||||
auto clone = std::make_shared<ASTQualifiedAsterisk>(*this);
|
||||
clone->cloneChildren();
|
||||
|
||||
if (transformers) { clone->transformers = transformers->clone(); clone->children.push_back(clone->transformers); }
|
||||
|
||||
clone->qualifier = qualifier->clone();
|
||||
clone->children.push_back(clone->qualifier);
|
||||
|
||||
return clone;
|
||||
}
|
||||
void appendColumnName(WriteBuffer & ostr) const override;
|
||||
|
||||
ASTPtr qualifier;
|
||||
ASTPtr transformers;
|
||||
protected:
|
||||
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
};
|
||||
|
@ -1657,13 +1657,21 @@ bool ParserAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
++pos;
|
||||
auto asterisk = std::make_shared<ASTAsterisk>();
|
||||
auto transformers = std::make_shared<ASTColumnsTransformerList>();
|
||||
ParserColumnsTransformers transformers_p(allowed_transformers);
|
||||
ASTPtr transformer;
|
||||
while (transformers_p.parse(pos, transformer, expected))
|
||||
{
|
||||
asterisk->children.push_back(transformer);
|
||||
transformers->children.push_back(transformer);
|
||||
}
|
||||
node = asterisk;
|
||||
|
||||
if (!transformers->children.empty())
|
||||
{
|
||||
asterisk->transformers = std::move(transformers);
|
||||
asterisk->children.push_back(asterisk->transformers);
|
||||
}
|
||||
|
||||
node = std::move(asterisk);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
@ -1672,7 +1680,7 @@ bool ParserAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
|
||||
bool ParserQualifiedAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
if (!ParserCompoundIdentifier(true, true).parse(pos, node, expected))
|
||||
if (!ParserCompoundIdentifier(false, true).parse(pos, node, expected))
|
||||
return false;
|
||||
|
||||
if (pos->type != TokenType::Dot)
|
||||
@ -1684,13 +1692,23 @@ bool ParserQualifiedAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|
||||
++pos;
|
||||
|
||||
auto res = std::make_shared<ASTQualifiedAsterisk>();
|
||||
res->children.push_back(node);
|
||||
auto transformers = std::make_shared<ASTColumnsTransformerList>();
|
||||
ParserColumnsTransformers transformers_p;
|
||||
ASTPtr transformer;
|
||||
while (transformers_p.parse(pos, transformer, expected))
|
||||
{
|
||||
res->children.push_back(transformer);
|
||||
transformers->children.push_back(transformer);
|
||||
}
|
||||
|
||||
res->qualifier = std::move(node);
|
||||
res->children.push_back(res->qualifier);
|
||||
|
||||
if (!transformers->children.empty())
|
||||
{
|
||||
res->transformers = std::move(transformers);
|
||||
res->children.push_back(res->transformers);
|
||||
}
|
||||
|
||||
node = std::move(res);
|
||||
return true;
|
||||
}
|
||||
@ -1714,28 +1732,44 @@ static bool parseColumnsMatcherBody(IParser::Pos & pos, ASTPtr & node, Expected
|
||||
return false;
|
||||
++pos;
|
||||
|
||||
auto transformers = std::make_shared<ASTColumnsTransformerList>();
|
||||
ParserColumnsTransformers transformers_p(allowed_transformers);
|
||||
ASTPtr transformer;
|
||||
while (transformers_p.parse(pos, transformer, expected))
|
||||
{
|
||||
transformers->children.push_back(transformer);
|
||||
}
|
||||
|
||||
ASTPtr res;
|
||||
if (column_list)
|
||||
{
|
||||
auto list_matcher = std::make_shared<ASTColumnsListMatcher>();
|
||||
list_matcher->column_list = column_list;
|
||||
res = list_matcher;
|
||||
|
||||
list_matcher->column_list = std::move(column_list);
|
||||
list_matcher->children.push_back(list_matcher->column_list);
|
||||
|
||||
if (!transformers->children.empty())
|
||||
{
|
||||
list_matcher->transformers = std::move(transformers);
|
||||
list_matcher->children.push_back(list_matcher->transformers);
|
||||
}
|
||||
|
||||
node = std::move(list_matcher);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto regexp_matcher = std::make_shared<ASTColumnsRegexpMatcher>();
|
||||
regexp_matcher->setPattern(regex_node->as<ASTLiteral &>().value.get<String>());
|
||||
res = regexp_matcher;
|
||||
}
|
||||
|
||||
ParserColumnsTransformers transformers_p(allowed_transformers);
|
||||
ASTPtr transformer;
|
||||
while (transformers_p.parse(pos, transformer, expected))
|
||||
if (!transformers->children.empty())
|
||||
{
|
||||
res->children.push_back(transformer);
|
||||
regexp_matcher->transformers = std::move(transformers);
|
||||
regexp_matcher->children.push_back(regexp_matcher->transformers);
|
||||
}
|
||||
|
||||
node = std::move(regexp_matcher);
|
||||
}
|
||||
|
||||
node = std::move(res);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -1751,29 +1785,19 @@ bool ParserColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
|
||||
|
||||
bool ParserQualifiedColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
if (!ParserCompoundIdentifier(true, true).parse(pos, node, expected))
|
||||
if (!ParserCompoundIdentifier(false, true).parse(pos, node, expected))
|
||||
return false;
|
||||
|
||||
auto identifier_node = node;
|
||||
const auto & identifier_node_typed = identifier_node->as<ASTTableIdentifier &>();
|
||||
auto & identifier_node_typed = identifier_node->as<ASTIdentifier &>();
|
||||
auto & name_parts = identifier_node_typed.name_parts;
|
||||
|
||||
/// ParserCompoundIdentifier parse identifier.COLUMNS
|
||||
if (identifier_node_typed.name_parts.size() == 1 || identifier_node_typed.name_parts.back() != "COLUMNS")
|
||||
if (name_parts.size() == 1 || name_parts.back() != "COLUMNS")
|
||||
return false;
|
||||
|
||||
/// TODO: ASTTableIdentifier can contain only 2 parts
|
||||
|
||||
if (identifier_node_typed.name_parts.size() == 2)
|
||||
{
|
||||
auto table_name = identifier_node_typed.name_parts[0];
|
||||
identifier_node = std::make_shared<ASTTableIdentifier>(table_name);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Expected identifier to contain no more than 2 parts. Actual {}",
|
||||
identifier_node_typed.full_name);
|
||||
}
|
||||
name_parts.pop_back();
|
||||
identifier_node = std::make_shared<ASTIdentifier>(std::move(name_parts), false, std::move(node->children));
|
||||
|
||||
if (!parseColumnsMatcherBody(pos, node, expected, allowed_transformers))
|
||||
return false;
|
||||
@ -1781,28 +1805,36 @@ bool ParserQualifiedColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected
|
||||
if (auto * columns_list_matcher = node->as<ASTColumnsListMatcher>())
|
||||
{
|
||||
auto result = std::make_shared<ASTQualifiedColumnsListMatcher>();
|
||||
result->qualifier = std::move(identifier_node);
|
||||
result->column_list = std::move(columns_list_matcher->column_list);
|
||||
|
||||
result->children.reserve(columns_list_matcher->children.size() + 1);
|
||||
result->children.push_back(std::move(identifier_node));
|
||||
result->children.push_back(result->qualifier);
|
||||
result->children.push_back(result->column_list);
|
||||
|
||||
for (auto && child : columns_list_matcher->children)
|
||||
result->children.push_back(std::move(child));
|
||||
if (columns_list_matcher->transformers)
|
||||
{
|
||||
result->transformers = std::move(columns_list_matcher->transformers);
|
||||
result->children.push_back(result->transformers);
|
||||
}
|
||||
|
||||
node = result;
|
||||
node = std::move(result);
|
||||
}
|
||||
else if (auto * column_regexp_matcher = node->as<ASTColumnsRegexpMatcher>())
|
||||
{
|
||||
auto result = std::make_shared<ASTQualifiedColumnsRegexpMatcher>();
|
||||
result->setPattern(column_regexp_matcher->getPattern(), false);
|
||||
result->setMatcher(column_regexp_matcher->getMatcher());
|
||||
|
||||
result->children.reserve(column_regexp_matcher->children.size() + 1);
|
||||
result->children.push_back(std::move(identifier_node));
|
||||
result->qualifier = std::move(identifier_node);
|
||||
result->children.push_back(result->qualifier);
|
||||
|
||||
for (auto && child : column_regexp_matcher->children)
|
||||
result->children.push_back(std::move(child));
|
||||
if (column_regexp_matcher->transformers)
|
||||
{
|
||||
result->transformers = std::move(column_regexp_matcher->transformers);
|
||||
result->children.push_back(result->transformers);
|
||||
}
|
||||
|
||||
node = result;
|
||||
node = std::move(result);
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Parsers/ParserSetQuery.h>
|
||||
|
||||
#include <Parsers/ASTAsterisk.h>
|
||||
#include <Parsers/ASTColumnsMatcher.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
|
||||
@ -2194,7 +2195,7 @@ struct ParserExpressionImpl
|
||||
using Layers = std::vector<std::unique_ptr<Layer>>;
|
||||
|
||||
Action tryParseOperand(Layers & layers, IParser::Pos & pos, Expected & expected);
|
||||
static Action tryParseOperator(Layers & layers, IParser::Pos & pos, Expected & expected);
|
||||
Action tryParseOperator(Layers & layers, IParser::Pos & pos, Expected & expected);
|
||||
};
|
||||
|
||||
|
||||
@ -2523,8 +2524,6 @@ Action ParserExpressionImpl::tryParseOperand(Layers & layers, IParser::Pos & pos
|
||||
|
||||
Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & pos, Expected & expected)
|
||||
{
|
||||
ASTPtr tmp;
|
||||
|
||||
/// ParserExpression can be called in this part of the query:
|
||||
/// ALTER TABLE partition_all2 CLEAR INDEX [ p ] IN PARTITION ALL
|
||||
///
|
||||
@ -2544,17 +2543,17 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po
|
||||
|
||||
if (cur_op == operators_table.end())
|
||||
{
|
||||
ASTPtr alias;
|
||||
ParserAlias alias_parser(layers.back()->allow_alias_without_as_keyword);
|
||||
auto old_pos = pos;
|
||||
|
||||
if (layers.back()->allow_alias &&
|
||||
!layers.back()->parsed_alias &&
|
||||
alias_parser.parse(pos, tmp, expected) &&
|
||||
layers.back()->insertAlias(tmp))
|
||||
alias_parser.parse(pos, alias, expected) &&
|
||||
layers.back()->insertAlias(alias))
|
||||
{
|
||||
layers.back()->parsed_alias = true;
|
||||
return Action::OPERATOR;
|
||||
}
|
||||
pos = old_pos;
|
||||
return Action::NONE;
|
||||
}
|
||||
|
||||
@ -2618,33 +2617,57 @@ Action ParserExpressionImpl::tryParseOperator(Layers & layers, IParser::Pos & po
|
||||
layers.back()->pushOperand(function);
|
||||
}
|
||||
|
||||
/// Dot (TupleElement operator) can be a beginning of a .* or .COLUMNS expressions
|
||||
if (op.type == OperatorType::TupleElement)
|
||||
{
|
||||
ASTPtr tmp;
|
||||
if (asterisk_parser.parse(pos, tmp, expected) ||
|
||||
columns_matcher_parser.parse(pos, tmp, expected))
|
||||
{
|
||||
if (auto * asterisk = tmp->as<ASTAsterisk>())
|
||||
{
|
||||
if (!layers.back()->popOperand(asterisk->expression))
|
||||
return Action::NONE;
|
||||
}
|
||||
else if (auto * columns_list_matcher = tmp->as<ASTColumnsListMatcher>())
|
||||
{
|
||||
if (!layers.back()->popOperand(columns_list_matcher->expression))
|
||||
return Action::NONE;
|
||||
}
|
||||
else if (auto * columns_regexp_matcher = tmp->as<ASTColumnsRegexpMatcher>())
|
||||
{
|
||||
if (!layers.back()->popOperand(columns_regexp_matcher->expression))
|
||||
return Action::NONE;
|
||||
}
|
||||
|
||||
layers.back()->pushOperand(std::move(tmp));
|
||||
return Action::OPERATOR;
|
||||
}
|
||||
}
|
||||
|
||||
layers.back()->pushOperator(op);
|
||||
|
||||
if (op.type == OperatorType::ArrayElement)
|
||||
layers.push_back(std::make_unique<ArrayElementLayer>());
|
||||
|
||||
|
||||
Action next = Action::OPERAND;
|
||||
|
||||
/// isNull & isNotNull are postfix unary operators
|
||||
if (op.type == OperatorType::IsNull)
|
||||
next = Action::OPERATOR;
|
||||
|
||||
if (op.type == OperatorType::StartBetween || op.type == OperatorType::StartNotBetween)
|
||||
layers.back()->between_counter++;
|
||||
return Action::OPERATOR;
|
||||
|
||||
if (op.type == OperatorType::Cast)
|
||||
{
|
||||
next = Action::OPERATOR;
|
||||
|
||||
ASTPtr type_ast;
|
||||
if (!ParserDataType().parse(pos, type_ast, expected))
|
||||
return Action::NONE;
|
||||
|
||||
layers.back()->pushOperand(std::make_shared<ASTLiteral>(queryToString(type_ast)));
|
||||
return Action::OPERATOR;
|
||||
}
|
||||
|
||||
return next;
|
||||
if (op.type == OperatorType::ArrayElement)
|
||||
layers.push_back(std::make_unique<ArrayElementLayer>());
|
||||
|
||||
if (op.type == OperatorType::StartBetween || op.type == OperatorType::StartNotBetween)
|
||||
layers.back()->between_counter++;
|
||||
|
||||
return Action::OPERAND;
|
||||
}
|
||||
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user