Added Analyzer

This commit is contained in:
Maksim Kita 2022-07-14 13:20:16 +02:00
parent 729692f862
commit 75885ce2e1
60 changed files with 7533 additions and 73 deletions

View File

@ -0,0 +1,7 @@
if (ENABLE_TESTS)
add_subdirectory(tests)
endif()
if (ENABLE_EXAMPLES)
add_subdirectory(examples)
endif()

View File

@ -0,0 +1,70 @@
#include <Analyzer/ColumnNode.h>
#include <Common/SipHash.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <Parsers/ASTIdentifier.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
QueryTreeNodePtr ColumnNode::getColumnSource() const
{
auto lock = column_source.lock();
if (!lock)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Column {} {} query tree node does not have valid source node",
column.name,
column.type->getName());
return lock;
}
void ColumnNode::dumpTree(WriteBuffer & buffer, size_t indent) const
{
buffer << std::string(indent, ' ') << "COLUMN ";
writePointerHex(this, buffer);
buffer << ' ' << column.name << " : " << column.type->getName() << " source ";
auto column_source_ptr = column_source.lock();
writePointerHex(column_source_ptr.get(), buffer);
}
void ColumnNode::updateTreeHashImpl(HashState & hash_state) const
{
hash_state.update(column.name.size());
hash_state.update(column.name);
auto column_source_ptr = column_source.lock();
if (column_source_ptr)
column_source_ptr->updateTreeHashImpl(hash_state);
}
QueryTreeNodePtr ColumnNode::cloneImpl() const
{
return std::make_shared<ColumnNode>(column, column_source);
}
void ColumnNode::getPointersToUpdateAfterClone(QueryTreePointersToUpdate & pointers_to_update)
{
/** This method is called on node returned from `cloneImpl`. Check IQueryTreeNode.h interface.
* old pointer is current column source pointer.
* update place is address of column source.
*/
const auto * old_pointer = getColumnSource().get();
pointers_to_update.emplace_back(old_pointer, &column_source);
}
ASTPtr ColumnNode::toASTImpl() const
{
return std::make_shared<ASTIdentifier>(column.name);
}
}

96
src/Analyzer/ColumnNode.h Normal file
View File

@ -0,0 +1,96 @@
#pragma once
#include <Core/NamesAndTypes.h>
#include <Analyzer/IQueryTreeNode.h>
namespace DB
{
/** Column node represents column in query tree.
* Column must have some column source.
* Column can be table expression, lambda, subquery.
* Column source must be valid during column node lifetime.
*
* During query analysis pass identifier node is resolved into column. See IdentifierNode.h.
*
* Examples:
* SELECT id FROM test_table. id is identifier that must be resolved to column node during query analysis pass.
* SELECT lambda(x -> x + 1, [1,2,3]). x is identifier inside lambda that must be resolved to column node during query analysis pass.
*
* Column node is initialized with column name, type and column source weak pointer.
* Additional care must be taken during clone to repoint column source to another node if its necessary see IQueryTreeNode.h `clone` method.
*/
class ColumnNode;
using ColumnNodePtr = std::shared_ptr<ColumnNode>;
class ColumnNode final : public IQueryTreeNode
{
public:
/// Construct column node with column name, type and column source weak pointer.
explicit ColumnNode(NameAndTypePair column_, QueryTreeNodeWeakPtr column_source_)
: column(std::move(column_))
, column_source(std::move(column_source_))
{}
/// Get column
const NameAndTypePair & getColumn() const
{
return column;
}
/// Get column name
const String & getColumnName() const
{
return column.name;
}
/// Get column type
const DataTypePtr & getColumnType() const
{
return column.type;
}
/** Get column source.
* If column source is not valid logical exception is thrown.
*/
QueryTreeNodePtr getColumnSource() const;
/// Get column source weak pointer
QueryTreeNodeWeakPtr getColumnSourceWeak() const
{
return column_source;
}
QueryTreeNodeType getNodeType() const override
{
return QueryTreeNodeType::COLUMN;
}
void dumpTree(WriteBuffer & buffer, size_t indent) const override;
String getName() const override
{
return column.name;
}
DataTypePtr getResultType() const override
{
return column.type;
}
protected:
void updateTreeHashImpl(HashState & hash_state) const override;
ASTPtr toASTImpl() const override;
QueryTreeNodePtr cloneImpl() const override;
void getPointersToUpdateAfterClone(QueryTreePointersToUpdate & pointers_to_update) override;
private:
NameAndTypePair column;
QueryTreeNodeWeakPtr column_source;
};
}

View File

@ -0,0 +1,302 @@
#include <Analyzer/ColumnTransformers.h>
#include <Common/SipHash.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTColumnsTransformers.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/LambdaNode.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
const char * toString(ColumnTransfomerType type)
{
switch (type)
{
case ColumnTransfomerType::APPLY: return "APPLY";
case ColumnTransfomerType::EXCEPT: return "EXCEPT";
case ColumnTransfomerType::REPLACE: return "REPLACE";
}
}
/// ApplyColumnTransformerNode implementation
const char * toString(ApplyColumnTransformerType type)
{
switch (type)
{
case ApplyColumnTransformerType::LAMBDA: return "LAMBDA";
case ApplyColumnTransformerType::FUNCTION: return "FUNCTION";
}
}
ApplyColumnTransformerNode::ApplyColumnTransformerNode(QueryTreeNodePtr expression_node_)
{
if (expression_node_->getNodeType() == QueryTreeNodeType::LAMBDA)
apply_transformer_type = ApplyColumnTransformerType::LAMBDA;
else if (expression_node_->getNodeType() == QueryTreeNodeType::FUNCTION)
apply_transformer_type = ApplyColumnTransformerType::FUNCTION;
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Apply column transformer expression must be lambda or function. Actual {}",
expression_node_->getNodeTypeName());
children.resize(1);
children[expression_child_index] = std::move(expression_node_);
}
void ApplyColumnTransformerNode::dumpTree(WriteBuffer & buffer, size_t indent) const
{
buffer << std::string(indent, ' ') << "APPLY COLUMN TRANSFORMER ";
writePointerHex(this, buffer);
buffer << ' ' << toString(apply_transformer_type) << '\n';
buffer << std::string(indent + 2, ' ') << "EXPRESSION" << '\n';
const auto & expression_node = getExpressionNode();
expression_node->dumpTree(buffer, indent + 4);
}
void ApplyColumnTransformerNode::updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const
{
hash_state.update(static_cast<size_t>(getTransformerType()));
getExpressionNode()->updateTreeHash(hash_state);
}
ASTPtr ApplyColumnTransformerNode::toASTImpl() const
{
auto ast_apply_transformer = std::make_shared<ASTColumnsApplyTransformer>();
const auto & expression_node = getExpressionNode();
if (apply_transformer_type == ApplyColumnTransformerType::FUNCTION)
{
auto & function_expression = expression_node->as<FunctionNode &>();
ast_apply_transformer->func_name = function_expression.getFunctionName();
ast_apply_transformer->parameters = function_expression.getParametersNode()->toAST();
}
else
{
auto & lambda_expression = expression_node->as<LambdaNode &>();
if (!lambda_expression.getArgumentNames().empty())
ast_apply_transformer->lambda_arg = lambda_expression.getArgumentNames()[0];
ast_apply_transformer->lambda = lambda_expression.toAST();
}
return ast_apply_transformer;
}
QueryTreeNodePtr ApplyColumnTransformerNode::cloneImpl() const
{
ApplyColumnTransformerNodePtr result_apply_transformer(new ApplyColumnTransformerNode());
return result_apply_transformer;
}
/// ExceptColumnTransformerNode implementation
bool ExceptColumnTransformerNode::isColumnMatching(const std::string & column_name) const
{
if (column_matcher)
return RE2::PartialMatch(column_name, *column_matcher);
for (const auto & name : except_column_names)
if (column_name == name)
return true;
return false;
}
const char * toString(ExceptColumnTransformerType type)
{
switch (type)
{
case ExceptColumnTransformerType::REGEXP:
return "REGEXP";
case ExceptColumnTransformerType::COLUMN_LIST:
return "COLUMN_LIST";
}
}
void ExceptColumnTransformerNode::dumpTree(WriteBuffer & buffer, size_t indent) const
{
buffer << std::string(indent, ' ') << "EXCEPT COLUMN TRANSFORMER ";
writePointerHex(this, buffer);
buffer << ' ' << toString(except_transformer_type) << ' ';
if (column_matcher)
{
buffer << column_matcher->pattern();
return;
}
size_t except_column_names_size = except_column_names.size();
for (size_t i = 0; i < except_column_names_size; ++i)
{
buffer << except_column_names[i];
if (i + 1 != except_column_names_size)
buffer << ", ";
}
}
void ExceptColumnTransformerNode::updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const
{
hash_state.update(static_cast<size_t>(getTransformerType()));
hash_state.update(static_cast<size_t>(getExceptTransformerType()));
for (const auto & column_name : except_column_names)
{
hash_state.update(column_name.size());
hash_state.update(column_name);
}
if (column_matcher)
{
const auto & pattern = column_matcher->pattern();
hash_state.update(pattern.size());
hash_state.update(pattern);
}
}
ASTPtr ExceptColumnTransformerNode::toASTImpl() const
{
auto ast_except_transformer = std::make_shared<ASTColumnsExceptTransformer>();
if (column_matcher)
{
ast_except_transformer->setPattern(column_matcher->pattern());
return ast_except_transformer;
}
ast_except_transformer->children.reserve(except_column_names.size());
for (const auto & name : except_column_names)
ast_except_transformer->children.push_back(std::make_shared<ASTIdentifier>(name));
return ast_except_transformer;
}
QueryTreeNodePtr ExceptColumnTransformerNode::cloneImpl() const
{
if (except_transformer_type == ExceptColumnTransformerType::REGEXP)
return std::make_shared<ExceptColumnTransformerNode>(column_matcher);
return std::make_shared<ExceptColumnTransformerNode>(except_column_names);
}
/// ReplaceColumnTransformerNode implementation
ReplaceColumnTransformerNode::ReplaceColumnTransformerNode(const std::vector<Replacement> & replacements_)
{
children.resize(1);
children[replacements_child_index] = std::make_shared<ListNode>();
auto & replacement_expressions_nodes = getReplacements().getNodes();
std::unordered_set<std::string> replacement_names_set;
for (const auto & replacement : replacements_)
{
auto [_, inserted] = replacement_names_set.emplace(replacement.column_name);
if (!inserted)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Expressions in column transformer replace should not contain same replacement {} more than once",
replacement.column_name);
replacements_names.push_back(replacement.column_name);
replacement_expressions_nodes.push_back(replacement.expression_node);
}
}
QueryTreeNodePtr ReplaceColumnTransformerNode::findReplacementExpression(const std::string & expression_name)
{
auto it = std::find(replacements_names.begin(), replacements_names.end(), expression_name);
if (it == replacements_names.end())
return {};
size_t replacement_index = it - replacements_names.begin();
auto & replacement_expressions_nodes = getReplacements().getNodes();
return replacement_expressions_nodes[replacement_index];
}
void ReplaceColumnTransformerNode::dumpTree(WriteBuffer & buffer, size_t indent) const
{
buffer << std::string(indent, ' ') << "REPLACE TRANSFORMER ";
writePointerHex(this, buffer);
buffer << '\n';
auto & replacements_nodes = getReplacements().getNodes();
size_t replacements_size = replacements_nodes.size();
buffer << std::string(indent + 2, ' ') << "REPLACEMENTS " << replacements_size << '\n';
for (size_t i = 0; i < replacements_size; ++i)
{
const auto & replacement_name = replacements_names[i];
buffer << std::string(indent + 4, ' ') << "REPLACEMENT NAME " << replacement_name;
buffer << " EXPRESSION" << '\n';
const auto & expression_node = replacements_nodes[i];
expression_node->dumpTree(buffer, indent + 6);
if (i + 1 != replacements_size)
buffer << '\n';
}
}
void ReplaceColumnTransformerNode::updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const
{
hash_state.update(static_cast<size_t>(getTransformerType()));
auto & replacement_expressions_nodes = getReplacements().getNodes();
size_t replacements_size = replacement_expressions_nodes.size();
hash_state.update(replacements_size);
for (size_t i = 0; i < replacements_size; ++i)
{
const auto & replacement_name = replacements_names[i];
hash_state.update(replacement_name.size());
hash_state.update(replacement_name);
replacement_expressions_nodes[i]->updateTreeHash(hash_state);
}
}
ASTPtr ReplaceColumnTransformerNode::toASTImpl() const
{
auto ast_replace_transformer = std::make_shared<ASTColumnsReplaceTransformer>();
auto & replacement_expressions_nodes = getReplacements().getNodes();
size_t replacements_size = replacement_expressions_nodes.size();
ast_replace_transformer->children.reserve(replacements_size);
for (size_t i = 0; i < replacements_size; ++i)
{
auto replacement_ast = std::make_shared<ASTColumnsReplaceTransformer::Replacement>();
replacement_ast->name = replacements_names[i];
replacement_ast->expr = replacement_expressions_nodes[i]->toAST();
ast_replace_transformer->children.push_back(replacement_ast);
}
return ast_replace_transformer;
}
QueryTreeNodePtr ReplaceColumnTransformerNode::cloneImpl() const
{
ReplaceColumnTransformerNodePtr result_replace_transformers(new ReplaceColumnTransformerNode());
result_replace_transformers->replacements_names = replacements_names;
return result_replace_transformers;
}
}

View File

@ -0,0 +1,267 @@
#pragma once
#include <re2/re2.h>
#include <Analyzer/Identifier.h>
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/ListNode.h>
namespace DB
{
/** Transformers are query tree nodes that handle additional logic that you can apply after MatcherQueryTreeNode is resolved.
* Check MatcherQueryTreeNode.h before reading this documentation.
*
* They main purpose it to apply some logic for expressions after matcher is resolved.
* There are 3 types of transformers:
*
* 1. APPLY transformer:
* APPLY transformer transform expression using lambda or function into another expression.
* It has 2 syntax variants:
* 1. lambda variant: SELECT matcher APPLY (x -> expr(x)).
* 2. function variant: SELECT matcher APPLY function_name(optional_parameters).
*
* 2. EXCEPT transformer:
* EXCEPT transformer discard some columns.
* It has 2 syntax variants:
* 1. regexp variant: SELECT matcher EXCEPT ('regexp').
* 2. column names list variant: SELECT matcher EXCEPT (column_name_1, ...).
*
* 3. REPLACE transfomer:
* REPLACE transformer applies similar transformation as APPLY transformer, but only for expressions
* that match replacement expression name.
*
* Example:
* CREATE TABLE test_table (id UInt64) ENGINE=TinyLog;
* SELECT * REPLACE (id + 1 AS id) FROM test_table.
* This query is transformed into SELECT id + 1 FROM test_table.
* It is important that AS id is not alias, it is replacement name. id + 1 is replacement expression.
*
* REPLACE transformer cannot contain multiple replacements with same name.
*
* REPLACE transformer expression does not necessary include replacement column name.
* Example:
* SELECT * REPLACE (1 AS id) FROM test_table.
*
* REPLACE transformer expression does not throw exception if there are no columns to apply replacement.
* Example:
* SELECT * REPLACE (1 AS unknown_column) FROM test_table;
*
* REPLACE transform can contain multiple replacements.
* Example:
* SELECT * REPLACE (1 AS id, 2 AS value).
*
* Matchers can be combined together and chained.
* Example:
* SELECT * EXCEPT (id) APPLY (x -> toString(x)) APPLY (x -> length(x)) FROM test_table.
*/
/// Column transformer type
enum class ColumnTransfomerType
{
APPLY,
EXCEPT,
REPLACE
};
/// Get column transformer type name
const char * toString(ColumnTransfomerType type);
class IColumnTransformerNode;
using ColumnTransformerNodePtr = std::shared_ptr<IColumnTransformerNode>;
using ColumnTransformersNodes = std::vector<ColumnTransformerNodePtr>;
/// IColumnTransformer base interface.
class IColumnTransformerNode : public IQueryTreeNode
{
public:
/// Get transformer type
virtual ColumnTransfomerType getTransformerType() const = 0;
/// Get transformer type name
const char * getTransformerTypeName() const
{
return toString(getTransformerType());
}
QueryTreeNodeType getNodeType() const final
{
return QueryTreeNodeType::TRANSFORMER;
}
};
enum class ApplyColumnTransformerType
{
LAMBDA,
FUNCTION
};
/// Get apply column transformer type name
const char * toString(ApplyColumnTransformerType type);
class ApplyColumnTransformerNode;
using ApplyColumnTransformerNodePtr = std::shared_ptr<ApplyColumnTransformerNode>;
/// Apply column transformer
class ApplyColumnTransformerNode final : public IColumnTransformerNode
{
public:
/** Initialize apply column transformer with expression node.
* Expression node must be lambda or function otherwise exception is throwned.
*/
explicit ApplyColumnTransformerNode(QueryTreeNodePtr expression_node_);
/// Get apply transformer type
ApplyColumnTransformerType getApplyTransformerType() const
{
return apply_transformer_type;
}
/// Get apply transformer expression node
const QueryTreeNodePtr & getExpressionNode() const
{
return children[expression_child_index];
}
ColumnTransfomerType getTransformerType() const override
{
return ColumnTransfomerType::APPLY;
}
void dumpTree(WriteBuffer & buffer, size_t indent) const override;
protected:
void updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const override;
ASTPtr toASTImpl() const override;
QueryTreeNodePtr cloneImpl() const override;
private:
ApplyColumnTransformerNode() = default;
ApplyColumnTransformerType apply_transformer_type = ApplyColumnTransformerType::LAMBDA;
static constexpr size_t expression_child_index = 0;
};
/// Except column transformer type
enum class ExceptColumnTransformerType
{
REGEXP,
COLUMN_LIST,
};
const char * toString(ExceptColumnTransformerType type);
class ExceptColumnTransformerNode;
using ExceptColumnTransformerNodePtr = std::shared_ptr<ExceptColumnTransformerNode>;
/// Except column transformer
class ExceptColumnTransformerNode final : public IColumnTransformerNode
{
public:
/// Initialize except column transformer with column names
explicit ExceptColumnTransformerNode(Names except_column_names_)
: except_transformer_type(ExceptColumnTransformerType::COLUMN_LIST)
, except_column_names(std::move(except_column_names_))
{
}
/// Initialize except column transformer with regexp column matcher
explicit ExceptColumnTransformerNode(std::shared_ptr<re2::RE2> column_matcher_)
: except_transformer_type(ExceptColumnTransformerType::REGEXP)
, column_matcher(std::move(column_matcher_))
{
}
/// Get except transformer type
ExceptColumnTransformerType getExceptTransformerType() const
{
return except_transformer_type;
}
/// Returns true if except transformer match column name, false otherwise.
bool isColumnMatching(const std::string & column_name) const;
ColumnTransfomerType getTransformerType() const override
{
return ColumnTransfomerType::EXCEPT;
}
void dumpTree(WriteBuffer & buffer, size_t indent) const override;
protected:
void updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const override;
ASTPtr toASTImpl() const override;
QueryTreeNodePtr cloneImpl() const override;
private:
ExceptColumnTransformerType except_transformer_type;
Names except_column_names;
std::shared_ptr<re2::RE2> column_matcher;
};
class ReplaceColumnTransformerNode;
using ReplaceColumnTransformerNodePtr = std::shared_ptr<ReplaceColumnTransformerNode>;
/// Replace column transformer
class ReplaceColumnTransformerNode final : public IColumnTransformerNode
{
public:
/// Replacement is column name and replace expression
struct Replacement
{
std::string column_name;
QueryTreeNodePtr expression_node;
};
/// Initialize replace column transformer with replacements
explicit ReplaceColumnTransformerNode(const std::vector<Replacement> & replacements_);
ColumnTransfomerType getTransformerType() const override
{
return ColumnTransfomerType::REPLACE;
}
/// Get replacements
ListNode & getReplacements() const
{
return children[replacements_child_index]->as<ListNode &>();
}
/// Get replacements node
const QueryTreeNodePtr & getReplacementsNode() const
{
return children[replacements_child_index];
}
/// Get replacements names
const Names & getReplacementsNames() const
{
return replacements_names;
}
/** Returns replacement expression if for expression name replacements exists, nullptr otherwise.
* Returned replacement expression must be cloned by caller.
*/
QueryTreeNodePtr findReplacementExpression(const std::string & expression_name);
void dumpTree(WriteBuffer & buffer, size_t indent) const override;
protected:
void updateTreeHashImpl(IQueryTreeNode::HashState & hash_state) const override;
ASTPtr toASTImpl() const override;
QueryTreeNodePtr cloneImpl() const override;
private:
ReplaceColumnTransformerNode() = default;
Names replacements_names;
static constexpr size_t replacements_child_index = 0;
};
}

View File

@ -0,0 +1,56 @@
#include <Analyzer/ConstantNode.h>
#include <Common/FieldVisitorToString.h>
#include <Common/SipHash.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <DataTypes/FieldToDataType.h>
#include <Parsers/ASTLiteral.h>
namespace DB
{
ConstantNode::ConstantNode(Field value_, DataTypePtr value_data_type_)
: value(std::move(value_))
, value_string_dump(applyVisitor(FieldVisitorToString(), value))
, type(std::move(value_data_type_))
{}
ConstantNode::ConstantNode(Field value_)
: value(std::move(value_))
, value_string_dump(applyVisitor(FieldVisitorToString(), value))
, type(applyVisitor(FieldToDataType(), value))
{}
void ConstantNode::dumpTree(WriteBuffer & buffer, size_t indent) const
{
buffer << std::string(indent, ' ') << "CONSTANT ";
writePointerHex(this, buffer);
buffer << ' ' << value.dump() << " : " << type->getName();
}
void ConstantNode::updateTreeHashImpl(HashState & hash_state) const
{
auto type_name = type->getName();
hash_state.update(type_name.size());
hash_state.update(type_name);
hash_state.update(value_string_dump.size());
hash_state.update(value_string_dump);
}
ASTPtr ConstantNode::toASTImpl() const
{
return std::make_shared<ASTLiteral>(value);
}
QueryTreeNodePtr ConstantNode::cloneImpl() const
{
return std::make_shared<ConstantNode>(value, type);
}
}

View File

@ -0,0 +1,62 @@
#pragma once
#include <Core/Field.h>
#include <Analyzer/IQueryTreeNode.h>
namespace DB
{
/** Constant node represents constant value in query tree.
* Constant value must be representable by Field.
* Examples: 1, 'constant_string', [1,2,3].
*/
class ConstantNode;
using ConstantNodePtr = std::shared_ptr<ConstantNode>;
class ConstantNode final : public IQueryTreeNode
{
public:
/// Construct constant query tree node from field and data type
explicit ConstantNode(Field value_, DataTypePtr value_data_type_);
/// Construct constant query tree node from field, data type will be derived from field value
explicit ConstantNode(Field value_);
/// Get constant value
const Field & getConstantValue() const
{
return value;
}
QueryTreeNodeType getNodeType() const override
{
return QueryTreeNodeType::CONSTANT;
}
void dumpTree(WriteBuffer & buffer, size_t indent) const override;
String getName() const override
{
return value_string_dump;
}
DataTypePtr getResultType() const override
{
return type;
}
protected:
void updateTreeHashImpl(HashState & hash_state) const override;
ASTPtr toASTImpl() const override;
QueryTreeNodePtr cloneImpl() const override;
private:
Field value;
String value_string_dump;
DataTypePtr type;
};
}

View File

@ -0,0 +1,123 @@
#include <Analyzer/FunctionNode.h>
#include <Common/FieldVisitorToString.h>
#include <Common/SipHash.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <Functions/IFunction.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <Parsers/ASTFunction.h>
namespace DB
{
void FunctionNode::resolveAsFunction(FunctionOverloadResolverPtr function_value, DataTypePtr result_type_value)
{
aggregate_function = nullptr;
function = std::move(function_value);
result_type = std::move(result_type_value);
function_name = function->getName();
}
void FunctionNode::resolveAsAggregateFunction(AggregateFunctionPtr aggregate_function_value, DataTypePtr result_type_value)
{
function = nullptr;
aggregate_function = std::move(aggregate_function_value);
result_type = std::move(result_type_value);
function_name = aggregate_function->getName();
}
void FunctionNode::dumpTree(WriteBuffer & buffer, size_t indent) const
{
buffer << std::string(indent, ' ') << "FUNCTION ";
writePointerHex(this, buffer);
buffer << ' ' << function_name << (result_type ? (" : " + result_type->getName()) : "");
const auto & parameters = getParameters();
if (!parameters.getNodes().empty())
{
buffer << '\n' << std::string(indent + 2, ' ') << "PARAMETERS\n";
parameters.dumpTree(buffer, indent + 4);
}
const auto & arguments = getArguments();
if (!arguments.getNodes().empty())
{
buffer << '\n' << std::string(indent + 2, ' ') << "ARGUMENTS\n";
arguments.dumpTree(buffer, indent + 4);
}
}
String FunctionNode::getName() const
{
String name = function_name;
const auto & parameters = getParameters();
const auto & parameters_nodes = parameters.getNodes();
if (!parameters_nodes.empty())
{
name += '(';
name += parameters.getName();
name += ')';
}
const auto & arguments = getArguments();
name += '(';
name += arguments.getName();
name += ')';
return name;
}
void FunctionNode::updateTreeHashImpl(HashState & hash_state) const
{
hash_state.update(function_name.size());
hash_state.update(function_name);
hash_state.update(isAggregateFunction());
if (result_type)
{
auto result_type_name = result_type->getName();
hash_state.update(result_type_name.size());
hash_state.update(result_type_name);
}
}
ASTPtr FunctionNode::toASTImpl() const
{
auto function_ast = std::make_shared<ASTFunction>();
function_ast->name = function_name;
const auto & parameters = getParameters();
if (!parameters.getNodes().empty())
{
function_ast->children.push_back(parameters.toAST());
function_ast->parameters = function_ast->children.back();
}
const auto & arguments = getArguments();
if (!arguments.getNodes().empty())
{
function_ast->children.push_back(arguments.toAST());
function_ast->arguments = function_ast->children.back();
}
return function_ast;
}
QueryTreeNodePtr FunctionNode::cloneImpl() const
{
auto result_function = std::make_shared<FunctionNode>(function_name);
/// This is valid for clone method function or aggregate function must be stateless
result_function->function = function;
result_function->aggregate_function = aggregate_function;
result_function->result_type = result_type;
return result_function;
}
}

181
src/Analyzer/FunctionNode.h Normal file
View File

@ -0,0 +1,181 @@
#pragma once
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/ListNode.h>
namespace DB
{
class IFunctionOverloadResolver;
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
class IAggregateFunction;
using AggregateFunctionPtr = std::shared_ptr<const IAggregateFunction>;
/** Function node represents function in query tree.
* Function syntax: function_name(parameter_1, ...)(argument_1, ...).
* If funciton does not have parameters its syntax is function_name(argument_1, ...).
* If function does not have arguments its syntax is function_name().
*
* In query tree function parameters and arguments are represented by ListNode.
*
* Function can be:
* 1. Aggregate function. Example: quantile(0.5)(x), sum(x).
* 2. Non aggregate function. Example: plus(x, x).
*
* Initially function node is initialize with function name.
* During query analysis pass function must be resolved using `resolveAsFunction` or `resolveAsAggregateFunction` methods.
* Resolved function is function that has result type and is initialized with concrete aggregate or non aggregate function.
*/
class FunctionNode;
using FunctionNodePtr = std::shared_ptr<FunctionNode>;
class FunctionNode final : public IQueryTreeNode
{
public:
/** Initialize function node with function name.
* Later during query analysis path function must be resolved.
*/
explicit FunctionNode(String function_name_)
: function_name(function_name_)
{
children.resize(2);
children[parameters_child_index] = std::make_shared<ListNode>();
children[arguments_child_index] = std::make_shared<ListNode>();
}
/// Get name
const String & getFunctionName() const
{
return function_name;
}
/// Get parameters
const ListNode & getParameters() const
{
return children[parameters_child_index]->as<const ListNode &>();
}
/// Get parameters
ListNode & getParameters()
{
return children[parameters_child_index]->as<ListNode &>();
}
/// Get parameters node
const QueryTreeNodePtr & getParametersNode() const
{
return children[parameters_child_index];
}
/// Get parameters node
QueryTreeNodePtr & getParametersNode()
{
return children[parameters_child_index];
}
/// Get arguments
const ListNode & getArguments() const
{
return children[arguments_child_index]->as<const ListNode &>();
}
/// Get arguments
ListNode & getArguments()
{
return children[arguments_child_index]->as<ListNode &>();
}
/// Get arguments node
const QueryTreeNodePtr & getArgumentsNode() const
{
return children[arguments_child_index];
}
/// Get arguments node
QueryTreeNodePtr & getArgumentsNode()
{
return children[arguments_child_index];
}
/** Get non aggregate function.
* If function is not resolved nullptr returned.
*/
const FunctionOverloadResolverPtr & getFunction() const
{
return function;
}
/** Get aggregate function.
* If function is not resolved nullptr returned.
* If function is resolved as non aggregate function nullptr returned.
*/
const AggregateFunctionPtr & getAggregateFunction() const
{
return aggregate_function;
}
/// Is function node resolved
bool isResolved() const
{
return function != nullptr || aggregate_function != nullptr;
}
/// Is function node resolved as aggregate function
bool isAggregateFunction() const
{
return aggregate_function != nullptr;
}
/// Is function node resolved as non aggregate function
bool isNonAggregateFunction() const
{
return function != nullptr;
}
/** Resolve function node as non aggregate function.
* It is important that function name is update with resolved function name.
* Main motiviation for this is query tree optimizations.
* Assume we have `multiIf` function with single argument, it can be converted to `if` function.
* Function name must be updated accordingly.
*/
void resolveAsFunction(FunctionOverloadResolverPtr function_value, DataTypePtr result_type_value);
/** Resolve function node as aggregate function.
* It is important that function name is update with resolved function name.
* Main motiviation for this is query tree optimizations.
*/
void resolveAsAggregateFunction(AggregateFunctionPtr aggregate_function_value, DataTypePtr result_type_value);
QueryTreeNodeType getNodeType() const override
{
return QueryTreeNodeType::FUNCTION;
}
void dumpTree(WriteBuffer & buffer, size_t indent) const override;
DataTypePtr getResultType() const override
{
return result_type;
}
String getName() const override;
protected:
void updateTreeHashImpl(HashState & hash_state) const override;
ASTPtr toASTImpl() const override;
QueryTreeNodePtr cloneImpl() const override;
private:
static constexpr size_t parameters_child_index = 0;
static constexpr size_t arguments_child_index = 1;
String function_name;
FunctionOverloadResolverPtr function;
AggregateFunctionPtr aggregate_function;
DataTypePtr result_type;
};
}

View File

@ -0,0 +1,158 @@
#include <Analyzer/IQueryTreeNode.h>
#include <unordered_map>
#include <Common/SipHash.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <Parsers/ASTWithAlias.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
}
const char * toString(QueryTreeNodeType type)
{
switch (type)
{
case QueryTreeNodeType::IDENTIFIER: return "IDENTIFIER";
case QueryTreeNodeType::ASTERISK: return "ASTERISK";
case QueryTreeNodeType::TRANSFORMER: return "TRANSFORMER";
case QueryTreeNodeType::LIST: return "LIST";
case QueryTreeNodeType::CONSTANT: return "CONSTANT";
case QueryTreeNodeType::FUNCTION: return "FUNCTION";
case QueryTreeNodeType::COLUMN: return "COLUMN";
case QueryTreeNodeType::LAMBDA: return "LAMBDA";
case QueryTreeNodeType::TABLE: return "TABLE";
case QueryTreeNodeType::QUERY: return "QUERY";
}
}
String IQueryTreeNode::dumpTree() const
{
WriteBufferFromOwnString buff;
dumpTree(buff, 0);
return buff.str();
}
IQueryTreeNode::Hash IQueryTreeNode::getTreeHash() const
{
HashState hash_state;
updateTreeHash(hash_state);
Hash result;
hash_state.get128(result);
return result;
}
void IQueryTreeNode::updateTreeHash(HashState & state) const
{
updateTreeHashImpl(state);
state.update(children.size());
for (const auto & child : children)
{
if (!child)
continue;
child->updateTreeHash(state);
}
}
QueryTreeNodePtr IQueryTreeNode::clone() const
{
/** Main motivation for this method is to allow nodes in query tree have weak pointers to other nodes.
* Main use cases is for column node to have weak pointer to its source.
* Source can be lambda, table, subquery and such information is useful for later analysis stages.
*
* Algorithm
* For each node we clone state and also create mapping old_pointer to new pointer.
* For each cloned node we also update node_pointer_to_update_node_pointers array.
*
* After that we can update pointer in node_pointer_to_update_node_pointers using old_pointer to new pointer mapping.
*/
std::unordered_map<const IQueryTreeNode *, QueryTreeNodePtr> old_pointer_to_new_pointer;
QueryTreePointersToUpdate pointers_to_update_after_clone;
QueryTreeNodePtr result_cloned_node_place;
std::deque<std::pair<const IQueryTreeNode *, QueryTreeNodePtr *>> nodes_to_clone;
nodes_to_clone.emplace_back(this, &result_cloned_node_place);
while (!nodes_to_clone.empty())
{
const auto [node_to_clone, cloned_node_place] = nodes_to_clone.front();
nodes_to_clone.pop_front();
auto node_clone = node_to_clone->cloneImpl();
*cloned_node_place = node_clone;
node_clone->setAlias(node_to_clone->alias);
node_clone->setOriginalAST(node_to_clone->original_ast);
node_clone->children = node_to_clone->children;
node_clone->getPointersToUpdateAfterClone(pointers_to_update_after_clone);
old_pointer_to_new_pointer.emplace(node_to_clone, node_clone);
for (auto & child : node_clone->children)
{
if (!child)
continue;
nodes_to_clone.emplace_back(child.get(), &child);
}
}
for (auto & [old_pointer, new_pointer] : pointers_to_update_after_clone)
{
auto it = old_pointer_to_new_pointer.find(old_pointer);
/** If node had weak pointer to some other node and this node is not valid in cloned subtree part do not clone it.
* It will continue to point to previous location and it is expected.
*
* For example: SELECT id as a, a FROM test_table.
* id is resolved as column and test_table is source.
* a is resolved as id and after resolve must be cloned.
* weak pointer to source from a will point to old id location.
*/
if (it == old_pointer_to_new_pointer.end())
continue;
*new_pointer = it->second;
}
return result_cloned_node_place;
}
ASTPtr IQueryTreeNode::toAST() const
{
auto converted_node = toASTImpl();
if (auto * ast_with_alias = typeid_cast<ASTWithAlias *>(converted_node.get()))
converted_node->setAlias(alias);
return converted_node;
}
String IQueryTreeNode::formatOriginalASTForErrorMessage() const
{
if (!original_ast)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Original AST was not set");
return original_ast->formatForErrorMessage();
}
String IQueryTreeNode::formatConvertedASTForErrorMessage() const
{
return toAST()->formatForErrorMessage();
}
}

View File

@ -0,0 +1,234 @@
#pragma once
#include <memory>
#include <string>
#include <vector>
#include <Common/TypePromotion.h>
#include <DataTypes/IDataType.h>
#include <Parsers/IAST_fwd.h>
#include <Analyzer/Identifier.h>
class SipHash;
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
}
class WriteBuffer;
/// Query tree node type
enum class QueryTreeNodeType
{
IDENTIFIER,
ASTERISK,
TRANSFORMER,
LIST,
CONSTANT,
FUNCTION,
COLUMN,
LAMBDA,
TABLE,
QUERY,
};
/// Convert query tree node type to string
const char * toString(QueryTreeNodeType type);
/** Query tree node represent node in query tree.
* This is base class for all query tree nodes.
*/
class IQueryTreeNode;
using QueryTreeNodePtr = std::shared_ptr<IQueryTreeNode>;
using QueryTreeNodeWeakPtr = std::weak_ptr<IQueryTreeNode>;
using QueryTreeNodes = std::vector<QueryTreeNodePtr>;
class IQueryTreeNode : public TypePromotion<IQueryTreeNode>
{
public:
virtual ~IQueryTreeNode() = default;
/// Get query tree node type
virtual QueryTreeNodeType getNodeType() const = 0;
const char * getNodeTypeName() const
{
return toString(getNodeType());
}
/** Get name of query tree node that can be used as part of expression.
* TODO: Projection name, expression name must be refactored in better interface.
*/
virtual String getName() const
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Method is not supported for {} query node", getNodeTypeName());
}
/** Get result type of query tree node that can be used as part of expression.
* If node does not support this method exception is throwed.
* TODO: Maybe this can be a part of ExpressionQueryTreeNode.
*/
virtual DataTypePtr getResultType() const
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Method is not supported for {} query node", getNodeTypeName());
}
/// Dump query tree to string
String dumpTree() const;
/// Dump query tree to buffer
void dumpTree(WriteBuffer & buffer) const
{
dumpTree(buffer, 0);
}
/// Dump query tree to buffer starting with indent
virtual void dumpTree(WriteBuffer & buffer, size_t indent) const = 0;
using Hash = std::pair<UInt64, UInt64>;
using HashState = SipHash;
/// Get tree hash identifying current tree
Hash getTreeHash() const;
/// Update tree hash
void updateTreeHash(HashState & state) const;
/// Get a deep copy of the query tree
QueryTreeNodePtr clone() const;
/// Check if node has alias
bool hasAlias() const
{
return !alias.empty();
}
/// Get node alias value if specified
const String & getAlias() const
{
return alias;
}
/// Set node alias value
void setAlias(String alias_value)
{
alias = std::move(alias_value);
}
/// Remove node alias value
void removeAlias()
{
alias = {};
}
/// Check if query tree node has original AST
bool hasOriginalAST() const
{
return original_ast != nullptr;
}
/// Get query tree node original AST
const ASTPtr & getOriginalAST() const
{
return original_ast;
}
/** Set query tree node original AST.
* This AST will not be modified later.
*/
void setOriginalAST(ASTPtr original_ast_value)
{
original_ast = std::move(original_ast_value);
}
/** If query tree has original AST format it for error message.
* Otherwise throw an exception.
*/
String formatOriginalASTForErrorMessage() const;
/// Convert query tree to AST
ASTPtr toAST() const;
/// Convert query tree to AST and then format it for error message.
String formatConvertedASTForErrorMessage() const;
/** Format AST for error message.
* If original AST exists use `formatOriginalASTForErrorMessage`.
* Otherwise use `formatConvertedASTForErrorMessage`.
*/
String formatASTForErrorMessage() const
{
if (original_ast)
return formatOriginalASTForErrorMessage();
return formatConvertedASTForErrorMessage();
}
/// Get query tree node children
QueryTreeNodes & getChildren()
{
return children;
}
/// Get query tree node children
const QueryTreeNodes & getChildren() const
{
return children;
}
/** Subclass must update tree hash of its internal state and do not update tree hash for children.
* Caller must update tree hash for node children.
*
* This method is not protected because if subclass node has weak pointers to other query tree nodes it must use it
* as part of its updateTreeHashImpl method. In child classes this method should be protected.
*/
virtual void updateTreeHashImpl(HashState & hash_state) const = 0;
protected:
/** Subclass node must convert itself to AST.
* Subclass must convert children to AST.
*/
virtual ASTPtr toASTImpl() const = 0;
/** Subclass must clone only it internal state.
* Subclass children will be cloned separately by caller.
*/
virtual QueryTreeNodePtr cloneImpl() const = 0;
/** If node has weak pointers to other tree nodes during clone they will point to other tree nodes.
* Keeping weak pointer to other tree nodes can be useful for example for column to keep weak pointer to column source.
* Source can be table, lambda, subquery and such information is necessary to preserve.
*
* Example:
* SELECT id FROM table;
* id during query analysis will be resolved as ColumnNode and source will be TableNode.
* During clone we must update id ColumnNode source pointer.
*
* Subclass must save old pointer and place of pointer update into pointers_to_update.
* This method will be called on query tree node after clone.
*
* Root of clone process will update pointers as necessary.
*/
using QueryTreePointerToUpdate = std::pair<const IQueryTreeNode *, QueryTreeNodeWeakPtr *>;
using QueryTreePointersToUpdate = std::vector<QueryTreePointerToUpdate>;
virtual void getPointersToUpdateAfterClone(QueryTreePointersToUpdate & pointers_to_update)
{
(void)(pointers_to_update);
}
QueryTreeNodes children;
private:
String alias;
ASTPtr original_ast;
};
}

View File

@ -0,0 +1,38 @@
#pragma once
#include <Interpreters/Context_fwd.h>
#include <Analyzer/IQueryTreeNode.h>
namespace DB
{
/** After query tree is build it can be later processed by QueryTreePassManager.
* This is abstract base class for all query tree passes.
*
* Query tree pass can make query tree modifications, after each pass query tree must be valid.
* Query tree pass must perform be isolated and perform only necessary query tree modifications for doing its job.
* Dependencies between passes must be avoided.
*/
class IQueryTreePass;
using QueryTreePassPtr = std::shared_ptr<IQueryTreePass>;
using QueryTreePasses = std::vector<QueryTreePassPtr>;
class IQueryTreePass
{
public:
virtual ~IQueryTreePass() = default;
/// Get query tree pass name
virtual String getName() = 0;
/// Get query tree pass description
virtual String getDescription() = 0;
/// Run pass over query tree
virtual void run(QueryTreeNodePtr query_tree_node, ContextPtr context) = 0;
};
}

320
src/Analyzer/Identifier.h Normal file
View File

@ -0,0 +1,320 @@
#pragma once
#include <vector>
#include <string>
#include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/join.hpp>
namespace DB
{
/** Identifier constists from identifier parts.
* Each identifier part is arbitrary long sequence of digits, underscores, lowercase and uppercase letters.
* Example: a, a.b, a.b.c.
*/
class Identifier
{
public:
Identifier() = default;
/// Create Identifier from parts
explicit Identifier(const std::vector<std::string> & parts_)
: full_name(boost::algorithm::join(parts_, "."))
, parts(parts_)
{
}
/// Create Identifier from full_name. Full_name is splitted with '.' as separator.
explicit Identifier(const std::string & full_name_)
: full_name(full_name_)
{
boost::split(parts, full_name, [](char c) { return c == '.'; });
}
const std::string & getFullName() const
{
return full_name;
}
const std::vector<std::string> & getParts() const
{
return parts;
}
size_t getPartsSize() const
{
return parts.size();
}
bool empty() const
{
return parts.empty();
}
bool isEmpty() const
{
return parts.empty();
}
bool isShort() const
{
return parts.size() == 1;
}
bool isCompound() const
{
return parts.size() > 1;
}
const std::string & at(size_t index) const
{
if (index >= parts.size())
throw std::out_of_range("identifier access part is out of range");
return parts[index];
}
const std::string & operator[](size_t index) const
{
return parts[index];
}
const std::string & front() const
{
return parts.front();
}
const std::string & back() const
{
return parts.back();
}
/// Returns true, if identifier starts with part, false otherwise
bool startsWith(const std::string_view & part)
{
return !parts.empty() && parts[0] == part;
}
/// Returns true, if identifier ends with part, false otherwise
bool endsWith(const std::string_view & part)
{
return !parts.empty() && parts.back() == part;
}
using const_iterator = std::vector<std::string>::const_iterator;
const_iterator begin() const
{
return parts.begin();
}
const_iterator end() const
{
return parts.end();
}
void popFirst(size_t parts_to_remove_size)
{
assert(parts_to_remove_size <= parts.size());
size_t parts_size = parts.size();
std::vector<std::string> result_parts;
result_parts.reserve(parts_size - parts_to_remove_size);
for (size_t i = parts_to_remove_size; i < parts_size; ++i)
result_parts.push_back(std::move(parts[i]));
parts = std::move(result_parts);
full_name = boost::algorithm::join(parts, ".");
}
void popFirst()
{
return popFirst(1);
}
void popLast(size_t parts_to_remove_size)
{
assert(parts_to_remove_size <= parts.size());
for (size_t i = 0; i < parts_to_remove_size; ++i)
parts.pop_back();
full_name = boost::algorithm::join(parts, ".");
}
void popLast()
{
return popLast(1);
}
private:
std::string full_name;
std::vector<std::string> parts;
};
inline bool operator==(const Identifier & lhs, const Identifier & rhs)
{
return lhs.getFullName() == rhs.getFullName();
}
inline bool operator!=(const Identifier & lhs, const Identifier & rhs)
{
return !(lhs == rhs);
}
inline std::ostream & operator<<(std::ostream & stream, const Identifier & identifier)
{
stream << identifier.getFullName();
return stream;
}
using Identifiers = std::vector<Identifier>;
/// View for Identifier
class IdentifierView
{
public:
IdentifierView() = default;
IdentifierView(const Identifier & identifier) /// NOLINT
: full_name_view(identifier.getFullName())
, parts_start_it(identifier.begin())
, parts_end_it(identifier.end())
{}
std::string_view getFullName() const
{
return full_name_view;
}
size_t getPartsSize() const
{
return parts_end_it - parts_start_it;
}
bool empty() const
{
return parts_start_it == parts_end_it;
}
bool isEmpty() const
{
return parts_start_it == parts_end_it;
}
bool isShort() const
{
return getPartsSize() == 1;
}
bool isCompound() const
{
return getPartsSize() > 1;
}
std::string_view at(size_t index) const
{
if (index >= getPartsSize())
throw std::out_of_range("identifier access part is out of range");
return *(parts_start_it + index);
}
std::string_view operator[](size_t index) const
{
return *(parts_start_it + index);
}
std::string_view front() const
{
return *parts_start_it;
}
std::string_view back() const
{
return *(parts_end_it - 1);
}
bool startsWith(std::string_view part) const
{
return !isEmpty() && *parts_start_it == part;
}
bool endsWith(std::string_view part) const
{
return !isEmpty() && *(parts_end_it - 1) == part;
}
void popFirst(size_t parts_to_remove_size)
{
assert(parts_to_remove_size <= getPartsSize());
for (size_t i = 0; i < parts_to_remove_size; ++i)
{
size_t part_size = parts_start_it->size();
++parts_start_it;
bool is_not_last = parts_start_it != parts_end_it;
full_name_view.remove_prefix(part_size + is_not_last);
}
}
void popFirst()
{
popFirst(1);
}
void popLast(size_t parts_to_remove_size)
{
assert(parts_to_remove_size <= getPartsSize());
for (size_t i = 0; i < parts_to_remove_size; ++i)
{
size_t last_part_size = (parts_end_it - 1)->size();
--parts_end_it;
bool is_not_last = parts_start_it != parts_end_it;
full_name_view.remove_suffix(last_part_size + is_not_last);
}
}
void popLast()
{
popLast(1);
}
using const_iterator = Identifier::const_iterator;
const_iterator begin() const
{
return parts_start_it;
}
const_iterator end() const
{
return parts_end_it;
}
private:
std::string_view full_name_view;
const_iterator parts_start_it;
const_iterator parts_end_it;
};
inline bool operator==(const IdentifierView & lhs, const IdentifierView & rhs)
{
return lhs.getFullName() == rhs.getFullName();
}
inline bool operator!=(const IdentifierView & lhs, const IdentifierView & rhs)
{
return !(lhs == rhs);
}
inline std::ostream & operator<<(std::ostream & stream, const IdentifierView & identifier_view)
{
stream << identifier_view.getFullName();
return stream;
}
}

View File

@ -0,0 +1,38 @@
#include <Analyzer/IdentifierNode.h>
#include <Common/SipHash.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <Parsers/ASTIdentifier.h>
namespace DB
{
void IdentifierNode::dumpTree(WriteBuffer & buffer, size_t indent) const
{
buffer << std::string(indent, ' ') << "IDENTIFIER ";
writePointerHex(this, buffer);
buffer << ' ' << identifier.getFullName();
}
void IdentifierNode::updateTreeHashImpl(HashState & state) const
{
const auto & identifier_name = identifier.getFullName();
state.update(identifier_name.size());
state.update(identifier_name);
}
ASTPtr IdentifierNode::toASTImpl() const
{
auto identifier_parts = identifier.getParts();
return std::make_shared<ASTIdentifier>(std::move(identifier_parts));
}
QueryTreeNodePtr IdentifierNode::cloneImpl() const
{
return std::make_shared<IdentifierNode>(identifier);
}
}

View File

@ -0,0 +1,53 @@
#pragma once
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/Identifier.h>
namespace DB
{
/** Identifier node represents identifier in query tree.
* Example: SELECT a FROM test_table.
* a - is identifier.
* test_table - is identifier.
*
* Identifier resolution must be done during query analysis pass.
*/
class IdentifierNode final : public IQueryTreeNode
{
public:
/// Construct identifier node with identifier
explicit IdentifierNode(Identifier identifier_)
: identifier(std::move(identifier_))
{}
/// Get identifier
const Identifier & getIdentifier() const
{
return identifier;
}
QueryTreeNodeType getNodeType() const override
{
return QueryTreeNodeType::IDENTIFIER;
}
void dumpTree(WriteBuffer & buffer, size_t indent) const override;
String getName() const override
{
return identifier.getFullName();
}
protected:
void updateTreeHashImpl(HashState & state) const override;
ASTPtr toASTImpl() const override;
QueryTreeNodePtr cloneImpl() const override;
private:
Identifier identifier;
};
}

View File

@ -0,0 +1,67 @@
#pragma once
#include <Common/Exception.h>
#include <Analyzer/IQueryTreeNode.h>
namespace DB
{
/** Visit query tree in depth.
* Matcher need to define `visit`, `needChildVisit` methods and `Data` type.
*/
template <typename Matcher, bool top_to_bottom, bool need_child_accept_data = false>
class InDepthQueryTreeVisitor
{
public:
using Data = typename Matcher::Data;
/// Initialize visitor with matchers data
explicit InDepthQueryTreeVisitor(Data & data_)
: data(data_)
{}
/// Visit query tree node
void visit(QueryTreeNodePtr & query_tree_node)
{
if constexpr (!top_to_bottom)
visitChildren(query_tree_node);
try
{
Matcher::visit(query_tree_node, data);
}
catch (Exception & e)
{
e.addMessage("While processing {}", query_tree_node->formatASTForErrorMessage());
throw;
}
if constexpr (top_to_bottom)
visitChildren(query_tree_node);
}
private:
Data & data;
void visitChildren(QueryTreeNodePtr & expression)
{
for (auto & child : expression->getChildren())
{
if (!child)
continue;
bool need_visit_child = false;
if constexpr (need_child_accept_data)
need_visit_child = Matcher::needChildVisit(expression, child, data);
else
need_visit_child = Matcher::needChildVisit(expression, child);
if (need_visit_child)
visit(child);
}
}
};
}

View File

@ -0,0 +1,92 @@
#include <Analyzer/LambdaNode.h>
#include <Common/SipHash.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
namespace DB
{
LambdaNode::LambdaNode(Names argument_names_, QueryTreeNodePtr expression_)
: argument_names(std::move(argument_names_))
{
children.resize(2);
auto arguments_list_node = std::make_shared<ListNode>();
auto & nodes = arguments_list_node->getNodes();
size_t argument_names_size = argument_names.size();
nodes.reserve(argument_names_size);
for (size_t i = 0; i < argument_names_size; ++i)
nodes.push_back(std::make_shared<IdentifierNode>(Identifier{argument_names[i]}));
children[arguments_child_index] = std::move(arguments_list_node);
children[expression_child_index] = std::move(expression_);
}
void LambdaNode::dumpTree(WriteBuffer & buffer, size_t indent) const
{
auto result_type = getExpression()->getResultType();
buffer << std::string(indent, ' ') << "LAMBDA ";
writePointerHex(this, buffer);
buffer << (result_type ? (" : " + result_type->getName()) : "") << '\n';
buffer << std::string(indent + 2, ' ') << "ARGUMENTS " << '\n';
getArguments().dumpTree(buffer, indent + 4);
buffer << '\n';
buffer << std::string(indent + 2, ' ') << "EXPRESSION " << '\n';
getExpression()->dumpTree(buffer, indent + 4);
}
String LambdaNode::getName() const
{
return "lambda(" + children[arguments_child_index]->getName() + ") -> " + children[expression_child_index]->getName();
}
void LambdaNode::updateTreeHashImpl(HashState & state) const
{
state.update(argument_names.size());
for (const auto & argument_name : argument_names)
{
state.update(argument_name.size());
state.update(argument_name);
}
}
ASTPtr LambdaNode::toASTImpl() const
{
auto lambda_function_arguments_ast = std::make_shared<ASTExpressionList>();
auto tuple_function = std::make_shared<ASTFunction>();
tuple_function->name = "tuple";
tuple_function->children.push_back(children[arguments_child_index]->toAST());
tuple_function->arguments = tuple_function->children.back();
lambda_function_arguments_ast->children.push_back(std::move(tuple_function));
lambda_function_arguments_ast->children.push_back(children[expression_child_index]->toAST());
auto lambda_function_ast = std::make_shared<ASTFunction>();
lambda_function_ast->name = "lambda";
lambda_function_ast->is_lambda_function = true;
lambda_function_ast->children.push_back(std::move(lambda_function_arguments_ast));
lambda_function_ast->arguments = lambda_function_ast->children.back();
return lambda_function_ast;
}
QueryTreeNodePtr LambdaNode::cloneImpl() const
{
LambdaNodePtr result_lambda(new LambdaNode());
result_lambda->argument_names = argument_names;
return result_lambda;
}
}

116
src/Analyzer/LambdaNode.h Normal file
View File

@ -0,0 +1,116 @@
#pragma once
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/ListNode.h>
#include <Analyzer/IdentifierNode.h>
#include <Parsers/ASTFunction.h>
namespace DB
{
/** Lambda node represents lambda expression in query tree.
*
* Lambda consist of argument names and lambda expression body.
* Lambda expression body does not necessary use lambda arguments. Example: SELECT arrayMap(x -> 1, [1, 2, 3])
*
* Initially lambda is initialized with argument names and expression query tree node.
* During query analysis if expression is not resolved lambda must be resolved.
* Lambda is resolved if lambda expression is resolved.
*
* It is important that lambda expression result type can depend on arguments types.
* Example: WITH (x -> x) as lambda SELECT lambda(1), lambda('string_value').
*
* During query analysis pass lambdas must be resolved.
* Lambda resolve must set concrete lambda arguments and resolve lambda expression body.
* In query tree lambda arguments are represented by ListNode.
* If client modified lambda arguments array its size must be equal to initial lambda argument names array.
*
* Examples:
* WITH (x -> x + 1) as lambda SELECT lambda(1).
* SELECT arrayMap(x -> x + 1, [1,2,3]).
*/
class LambdaNode;
using LambdaNodePtr = std::shared_ptr<LambdaNode>;
class LambdaNode final : public IQueryTreeNode
{
public:
/// Initialize lambda with argument names and expression query tree node
explicit LambdaNode(Names argument_names_, QueryTreeNodePtr expression_);
/// Get argument names
const Names & getArgumentNames() const
{
return argument_names;
}
/// Get arguments
const ListNode & getArguments() const
{
return children[arguments_child_index]->as<const ListNode &>();
}
/// Get arguments
ListNode & getArguments()
{
return children[arguments_child_index]->as<ListNode &>();
}
/// Get arguments node
const QueryTreeNodePtr & getArgumentsNode() const
{
return children[arguments_child_index];
}
/** Get arguments node.
* If arguments array is modified its result size must be equal to lambd argument names size.
*/
QueryTreeNodePtr & getArgumentsNode()
{
return children[arguments_child_index];
}
/// Get expression
const QueryTreeNodePtr & getExpression() const
{
return children[expression_child_index];
}
/// Get expression
QueryTreeNodePtr & getExpression()
{
return children[expression_child_index];
}
QueryTreeNodeType getNodeType() const override
{
return QueryTreeNodeType::LAMBDA;
}
void dumpTree(WriteBuffer & buffer, size_t indent) const override;
String getName() const override;
DataTypePtr getResultType() const override
{
return getExpression()->getResultType();
}
protected:
void updateTreeHashImpl(HashState & state) const override;
ASTPtr toASTImpl() const override;
QueryTreeNodePtr cloneImpl() const override;
private:
LambdaNode() = default;
static constexpr size_t arguments_child_index = 0;
static constexpr size_t expression_child_index = 1;
Names argument_names;
};
}

72
src/Analyzer/ListNode.cpp Normal file
View File

@ -0,0 +1,72 @@
#include <Analyzer/ListNode.h>
#include <Common/SipHash.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <Parsers/ASTExpressionList.h>
namespace DB
{
void ListNode::dumpTree(WriteBuffer & buffer, size_t indent) const
{
size_t children_size = children.size();
buffer << std::string(indent, ' ') << "LIST ";
writePointerHex(this, buffer);
buffer << ' ' << children_size << '\n';
for (size_t i = 0; i < children_size; ++i)
{
const auto & node = children[i];
node->dumpTree(buffer, indent + 2);
if (i + 1 != children_size)
buffer << '\n';
}
}
String ListNode::getName() const
{
if (children.empty())
return "";
std::string result;
for (const auto & node : children)
{
result += node->getName();
result += ", ";
}
result.pop_back();
result.pop_back();
return result;
}
void ListNode::updateTreeHashImpl(HashState &) const
{
/// Hash automatically updated with children size in IQueryTreeNode.h
}
ASTPtr ListNode::toASTImpl() const
{
auto expression_list_ast = std::make_shared<ASTExpressionList>();
size_t children_size = children.size();
expression_list_ast->children.resize(children_size);
for (size_t i = 0; i < children_size; ++i)
expression_list_ast->children[i] = children[i]->toAST();
return expression_list_ast;
}
QueryTreeNodePtr ListNode::cloneImpl() const
{
return std::make_shared<ListNode>();
}
}

48
src/Analyzer/ListNode.h Normal file
View File

@ -0,0 +1,48 @@
#pragma once
#include <Analyzer/IQueryTreeNode.h>
namespace DB
{
/** List node represents list of query tree nodes in query tree.
*
* Example: SELECT column_1, 1, 'constant_value' FROM table.
* column_1, 1, 'constant_value' is list query tree node.
*/
class ListNode;
using ListNodePtr = std::shared_ptr<ListNode>;
class ListNode final : public IQueryTreeNode
{
public:
/// Get list nodes
const QueryTreeNodes & getNodes() const
{
return children;
}
/// Get list nodes
QueryTreeNodes & getNodes()
{
return children;
}
QueryTreeNodeType getNodeType() const override
{
return QueryTreeNodeType::LIST;
}
void dumpTree(WriteBuffer & buffer, size_t indent) const override;
String getName() const override;
protected:
void updateTreeHashImpl(HashState &) const override;
ASTPtr toASTImpl() const override;
QueryTreeNodePtr cloneImpl() const override;
};
}

View File

@ -0,0 +1,267 @@
#include <Analyzer/MatcherNode.h>
#include <Common/SipHash.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTQualifiedAsterisk.h>
#include <Parsers/ASTColumnsMatcher.h>
namespace DB
{
const char * toString(MatcherNodeType matcher_node_type)
{
switch (matcher_node_type)
{
case MatcherNodeType::ASTERISK:
return "ASTERISK";
case MatcherNodeType::COLUMNS_LIST:
return "COLUMNS_LIST";
case MatcherNodeType::COLUMNS_REGEXP:
return "COLUMNS_REGEXP";
}
}
MatcherNode::MatcherNode(ColumnTransformersNodes column_transformers_)
: MatcherNode(MatcherNodeType::ASTERISK,
{} /*qualified_identifier*/,
{} /*columns_identifiers*/,
{} /*columns_matcher*/,
std::move(column_transformers_) /*column_transformers*/)
{
}
MatcherNode::MatcherNode(Identifier qualified_identifier_, ColumnTransformersNodes column_transformers_)
: MatcherNode(MatcherNodeType::ASTERISK,
std::move(qualified_identifier_),
{} /*columns_identifiers*/,
{} /*columns_matcher*/,
std::move(column_transformers_))
{
}
MatcherNode::MatcherNode(std::shared_ptr<re2::RE2> columns_matcher_, ColumnTransformersNodes column_transformers_)
: MatcherNode(MatcherNodeType::COLUMNS_REGEXP,
{} /*qualified_identifier*/,
{} /*columns_identifiers*/,
std::move(columns_matcher_),
std::move(column_transformers_))
{
}
MatcherNode::MatcherNode(Identifier qualified_identifier_, std::shared_ptr<re2::RE2> columns_matcher_, ColumnTransformersNodes column_transformers_)
: MatcherNode(MatcherNodeType::COLUMNS_REGEXP,
std::move(qualified_identifier_),
{} /*columns_identifiers*/,
std::move(columns_matcher_),
std::move(column_transformers_))
{
}
MatcherNode::MatcherNode(Identifiers columns_identifiers_, ColumnTransformersNodes column_transformers_)
: MatcherNode(MatcherNodeType::COLUMNS_LIST,
{} /*qualified_identifier*/,
std::move(columns_identifiers_),
{} /*columns_matcher*/,
std::move(column_transformers_))
{
}
MatcherNode::MatcherNode(Identifier qualified_identifier_, Identifiers columns_identifiers_, ColumnTransformersNodes column_transformers_)
: MatcherNode(MatcherNodeType::COLUMNS_LIST,
std::move(qualified_identifier_),
std::move(columns_identifiers_),
{} /*columns_matcher*/,
std::move(column_transformers_))
{
}
MatcherNode::MatcherNode(MatcherNodeType matcher_type_,
Identifier qualified_identifier_,
Identifiers columns_identifiers_,
std::shared_ptr<re2::RE2> columns_matcher_,
ColumnTransformersNodes column_transformers_)
: matcher_type(matcher_type_)
, qualified_identifier(qualified_identifier_)
, columns_identifiers(columns_identifiers_)
, columns_matcher(columns_matcher_)
{
auto column_transformers_list_node = std::make_shared<ListNode>();
auto & column_transformers_nodes = column_transformers_list_node->getNodes();
column_transformers_nodes.reserve(column_transformers_.size());
for (auto && column_transformer : column_transformers_)
column_transformers_nodes.emplace_back(std::move(column_transformer));
children.resize(1);
children[column_transformers_child_index] = std::move(column_transformers_list_node);
columns_identifiers_set.reserve(columns_identifiers.size());
for (auto & column_identifier : columns_identifiers)
columns_identifiers_set.insert(column_identifier.getFullName());
}
bool MatcherNode::isMatchingColumn(const std::string & column_name)
{
if (matcher_type == MatcherNodeType::ASTERISK)
return true;
if (columns_matcher)
return RE2::PartialMatch(column_name, *columns_matcher);
return columns_identifiers_set.find(column_name) != columns_identifiers_set.end();
}
void MatcherNode::dumpTree(WriteBuffer & buffer, size_t indent) const
{
buffer << std::string(indent, ' ') << "MATCHER ";
writePointerHex(this, buffer);
if (!qualified_identifier.empty())
buffer << ' ' << qualified_identifier.getFullName();
buffer << ' ' << toString(matcher_type);
if (columns_matcher)
{
buffer << ' ' << columns_matcher->pattern();
}
else if (matcher_type == MatcherNodeType::COLUMNS_LIST)
{
buffer << ' ';
size_t columns_identifiers_size = columns_identifiers.size();
for (size_t i = 0; i < columns_identifiers_size; ++i)
{
buffer << columns_identifiers[i].getFullName();
if (i + 1 != columns_identifiers_size)
buffer << ", ";
}
}
const auto & column_transformers_list = getColumnTransformers();
if (!column_transformers_list.getNodes().empty())
{
buffer << '\n';
column_transformers_list.dumpTree(buffer, indent + 2);
}
}
String MatcherNode::getName() const
{
if (matcher_type == MatcherNodeType::ASTERISK)
return "*";
WriteBufferFromOwnString buffer;
buffer << "COLUMNS(";
if (columns_matcher)
{
buffer << ' ' << columns_matcher->pattern();
}
else if (matcher_type == MatcherNodeType::COLUMNS_LIST)
{
size_t columns_identifiers_size = columns_identifiers.size();
for (size_t i = 0; i < columns_identifiers_size; ++i)
{
buffer << columns_identifiers[i].getFullName();
if (i + 1 != columns_identifiers_size)
buffer << ", ";
}
}
buffer << ')';
return buffer.str();
}
void MatcherNode::updateTreeHashImpl(HashState & hash_state) const
{
hash_state.update(static_cast<size_t>(matcher_type));
const auto & qualified_identifier_full_name = qualified_identifier.getFullName();
hash_state.update(qualified_identifier_full_name.size());
hash_state.update(qualified_identifier_full_name);
for (const auto & identifier : columns_identifiers)
{
const auto & identifier_full_name = identifier.getFullName();
hash_state.update(identifier_full_name.size());
hash_state.update(identifier_full_name.data(), identifier_full_name.size());
}
if (columns_matcher)
{
const auto & columns_matcher_pattern = columns_matcher->pattern();
hash_state.update(columns_matcher_pattern.size());
hash_state.update(columns_matcher_pattern);
}
}
ASTPtr MatcherNode::toASTImpl() const
{
ASTPtr result;
if (matcher_type == MatcherNodeType::ASTERISK)
{
/// For COLUMNS qualified identifier is not supported
if (qualified_identifier.empty())
{
result = std::make_shared<ASTAsterisk>();
}
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)));
result = qualified_asterisk;
}
}
else if (columns_matcher)
{
auto regexp_matcher = std::make_shared<ASTColumnsRegexpMatcher>();
regexp_matcher->setPattern(columns_matcher->pattern());
result = regexp_matcher;
}
else
{
auto columns_list_matcher = std::make_shared<ASTColumnsListMatcher>();
columns_list_matcher->children.reserve(columns_identifiers.size());
for (const auto & identifier : columns_identifiers)
{
auto identifier_parts = identifier.getParts();
columns_list_matcher->children.push_back(std::make_shared<ASTIdentifier>(std::move(identifier_parts)));
}
result = columns_list_matcher;
}
for (const auto & child : children)
result->children.push_back(child->toAST());
return result;
}
QueryTreeNodePtr MatcherNode::cloneImpl() const
{
MatcherNodePtr matcher_node = std::make_shared<MatcherNode>();
matcher_node->matcher_type = matcher_type;
matcher_node->qualified_identifier = qualified_identifier;
matcher_node->columns_identifiers = columns_identifiers;
matcher_node->columns_matcher = columns_matcher;
matcher_node->columns_identifiers_set = columns_identifiers_set;
return matcher_node;
}
}

171
src/Analyzer/MatcherNode.h Normal file
View File

@ -0,0 +1,171 @@
#pragma once
#include <re2/re2.h>
#include <Analyzer/Identifier.h>
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/ColumnTransformers.h>
#include <Parsers/ASTAsterisk.h>
namespace DB
{
/** Matcher query tree node.
* Matcher can be unqualified with identifier and qualified with identifier.
* It can be asterisk or COLUMNS('regexp') or COLUMNS(column_name_1, ...).
* In result we have 6 possible options:
* Unqualified
* 1. *
* 2. COLUMNS('regexp')
* 3. COLUMNS(column_name_1, ...)
*
* Qualified:
* 1. identifier.*
* 2. identifier.COLUMNS('regexp')
* 3. identifier.COLUMNS(column_name_1, ...)
*
* The main difference between matcher and identifier is that matcher cannot have alias.
* This simplifies analysis for matchers.
*
* How to resolve matcher during query analysis pass:
* 1. If matcher is unqualified, we use tables of current scope and try to resolve matcher from it.
* 2. If matcher is qualified:
* First try to resolve identifier part as query expression.
* Try expressions from aliases, then from tables (can be changed using prefer_column_name_to_alias setting).
* If identifier is resolved as expression. If expression is compound apply matcher to it, otherwise throw exception.
* Example: SELECT compound_column AS a, a.* FROM test_table.
* Example: SELECT compound_column.* FROM test_table.
*
* If identifier is not resolved as expression try to resolve it as table.
* If identifier is resolved as table then apply matcher to it.
* Example: SELECT test_table.* FROM test_table.
* Example: SELECT a.* FROM test_table AS a.
*
* Additionaly each matcher can contain transformers, check ColumnTransformers.h.
* In query tree matchers column transformers are represended as ListNode.
*/
enum class MatcherNodeType
{
ASTERISK,
COLUMNS_REGEXP,
COLUMNS_LIST
};
const char * toString(MatcherNodeType matcher_node_type);
class MatcherNode;
using MatcherNodePtr = std::shared_ptr<MatcherNode>;
class MatcherNode final : public IQueryTreeNode
{
public:
/// Variant unqualified asterisk
explicit MatcherNode(ColumnTransformersNodes column_transformers_ = {});
/// Variant qualified asterisk
explicit MatcherNode(Identifier qualified_identifier_, ColumnTransformersNodes column_transformers_ = {});
/// Variant unqualified COLUMNS('regexp')
explicit MatcherNode(std::shared_ptr<re2::RE2> columns_matcher_, ColumnTransformersNodes column_transformers_ = {});
/// Variant qualified COLUMNS('regexp')
explicit MatcherNode(Identifier qualified_identifier_, std::shared_ptr<re2::RE2> columns_matcher_, ColumnTransformersNodes column_transformers_ = {});
/// Variant unqualified COLUMNS(column_name_1, ...)
explicit MatcherNode(Identifiers columns_identifiers_, ColumnTransformersNodes column_transformers_ = {});
/// Variant qualified COLUMNS(column_name_1, ...)
explicit MatcherNode(Identifier qualified_identifier_, Identifiers columns_identifiers_, ColumnTransformersNodes column_transformers_ = {});
/// Get matcher type
MatcherNodeType getMatcherType() const
{
return matcher_type;
}
/// Is this matcher represented by asterisk
bool isAsteriskMatcher() const
{
return matcher_type == MatcherNodeType::ASTERISK;
}
/// Is this matcher represented by COLUMNS
bool isColumnsMatcher() const
{
return matcher_type == MatcherNodeType::COLUMNS_REGEXP || matcher_type == MatcherNodeType::COLUMNS_LIST;
}
/// Returns true if matcher qualified with identifier, false otherwise
bool isQualified() const
{
return !qualified_identifier.empty();
}
/// Get qualified identifier
const Identifier & getQualifiedIdentifier() const
{
return qualified_identifier;
}
/// Get columns matcher. Valid only if this matcher has type COLUMNS_REGEXP.
const std::shared_ptr<re2::RE2> & getColumnsMatcher() const
{
return columns_matcher;
}
/// Get columns matcher. Valid only if this matcher has type COLUMNS_LIST.
const Identifiers & getColumnsIdentifiers() const
{
return columns_identifiers;
}
/** Get column transformers
* Client can expect that node in this list is subclass of IColumnTransformerNode.
*/
const ListNode & getColumnTransformers() const
{
return children[column_transformers_child_index]->as<const ListNode &>();
}
const QueryTreeNodePtr & getColumnTransformersNode() const
{
return children[column_transformers_child_index];
}
/// Returns true if matcher match column name, false otherwise
bool isMatchingColumn(const std::string & column_name);
QueryTreeNodeType getNodeType() const override
{
return QueryTreeNodeType::ASTERISK;
}
void dumpTree(WriteBuffer & buffer, size_t indent) const override;
String getName() const override;
protected:
void updateTreeHashImpl(HashState & hash_state) const override;
ASTPtr toASTImpl() const override;
QueryTreeNodePtr cloneImpl() const override;
private:
explicit MatcherNode(MatcherNodeType matcher_type_,
Identifier qualified_identifier_,
Identifiers columns_identifiers_,
std::shared_ptr<re2::RE2> columns_matcher_,
ColumnTransformersNodes column_transformers_);
MatcherNodeType matcher_type;
Identifier qualified_identifier;
Identifiers columns_identifiers;
std::shared_ptr<re2::RE2> columns_matcher;
std::unordered_set<std::string> columns_identifiers_set;
static constexpr size_t column_transformers_child_index = 0;
};
}

View File

@ -0,0 +1,49 @@
#include <Analyzer/MultiIfToIfPass.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/FunctionNode.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
namespace
{
class MultiIfToIfVisitorMatcher
{
public:
using Visitor = InDepthQueryTreeVisitor<MultiIfToIfVisitorMatcher, true>;
struct Data
{
FunctionOverloadResolverPtr if_function_overload_resolver;
};
static void visit(QueryTreeNodePtr & node, Data & data)
{
auto * function_node = node->as<FunctionNode>();
if (!function_node || function_node->getFunctionName() != "multiIf")
return;
if (function_node->getArguments().getNodes().size() != 3)
return;
auto result_type = function_node->getResultType();
function_node->resolveAsFunction(data.if_function_overload_resolver, result_type);
}
static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &)
{
return true;
}
};
}
void MultiIfToIfPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
{
MultiIfToIfVisitorMatcher::Data data{FunctionFactory::instance().get("if", context)};
MultiIfToIfVisitorMatcher::Visitor visitor(data);
visitor.visit(query_tree_node);
}
}

View File

@ -0,0 +1,25 @@
#pragma once
#include <Analyzer/IQueryTreePass.h>
namespace DB
{
/** Convert multiIf with single argument into if.
* Example: SELECT multiIf(x, 1, 0);
* Result: SELECT if(x, 1, 0);
*/
class MultiIfToIfPass : public IQueryTreePass
{
public:
String getName() override { return "MultiIfToIf"; }
String getDescription() override { return "Optimize multiIf to if for single argument"; }
void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override;
};
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,49 @@
#pragma once
#include <Parsers/IAST_fwd.h>
#include <Interpreters/Context_fwd.h>
#include <Analyzer/IQueryTreePass.h>
namespace DB
{
/** This pass make initial query analysis.
*
* 1. All identifiers are resolved. Next passes can expect that there will be no IdentifierNode in query tree.
* 2. All matchers are resolved. Next passes can expect that there will be no MatcherNode in query tree.
* 3. All functions are resolved. Next passes can expect that for each FunctionNode its result type will be set, and it will be resolved
* as aggregate or non aggregate function.
* 4. All lambda expressions that are function arguments are resolved. Next passes can expect that LambaNode expression is resolved, and lambda has concrete arguments.
* 5. All standalone lambda expressions are resolved. Next passes can expect that there will be no standalone LambaNode expressions in query.
* 6. Constants are folded. Example: SELECT plus(1, 1). After step will be: SELECT 2.
* Motivation for this, there are places in query tree that must contain constant:
* Function parameters Example: SELECT quantile(0.5)(x).
* Functions in which result type depends on constant expression. Example: cast(x, 'type_name').
* Expressions that are part of LIMIT. Example: SELECT * FROM test_table LIMIT expr.
*
* 7. All scalar subqueries are evaluated.
* TODO: Scalar subqueries must be evaluated only if they are part of query tree where we must have constant. This is currently not done
* because execution layer does not support scalar subqueries execution.
*
* 8. Special function `untuple` is handled properly.
* 9. Sets are build for `in` function.
*/
class QueryAnalysisPass final : public IQueryTreePass
{
public:
String getName() override
{
return "QueryAnalysis";
}
String getDescription() override
{
return "Resolve type for each query expression. Replace identifiers, matchers with query expressions. Perform constant folding. Evaluate scalar subqueries.";
}
void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override;
};
}

100
src/Analyzer/QueryNode.cpp Normal file
View File

@ -0,0 +1,100 @@
#include <Analyzer/QueryNode.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
namespace DB
{
QueryNode::QueryNode()
{
children.resize(children_size);
children[with_child_index] = std::make_shared<ListNode>();
children[projection_child_index] = std::make_shared<ListNode>();
}
void QueryNode::dumpTree(WriteBuffer & buffer, size_t indent) const
{
buffer << std::string(indent, ' ') << "QUERY ";
writePointerHex(this, buffer);
buffer << '\n';
if (!getWith().getNodes().empty())
{
buffer << std::string(indent, ' ') << "WITH\n";
getWith().dumpTree(buffer, indent + 2);
buffer << '\n';
}
buffer << std::string(indent, ' ') << "PROJECTION\n";
getProjection().dumpTree(buffer, indent + 2);
buffer << '\n';
if (getFrom())
{
buffer << std::string(indent, ' ') << "JOIN TREE\n";
getFrom()->dumpTree(buffer, indent + 2);
buffer << '\n';
}
if (getPrewhere())
{
buffer << std::string(indent, ' ') << "PREWHERE\n";
getPrewhere()->dumpTree(buffer, indent + 2);
buffer << '\n';
}
if (getWhere())
{
buffer << std::string(indent, ' ') << "WHERE\n";
getWhere()->dumpTree(buffer, indent + 2);
buffer << '\n';
}
}
void QueryNode::updateTreeHashImpl(HashState &) const
{
/// TODO: No state
}
ASTPtr QueryNode::toASTImpl() const
{
auto select_query = std::make_shared<ASTSelectQuery>();
if (!getWith().getNodes().empty())
select_query->setExpression(ASTSelectQuery::Expression::WITH, getWithNode()->toAST());
select_query->setExpression(ASTSelectQuery::Expression::SELECT, children[projection_child_index]->toAST());
auto table_expression_ast = std::make_shared<ASTTableExpression>();
table_expression_ast->children.push_back(children[from_child_index]->toAST());
table_expression_ast->database_and_table_name = table_expression_ast->children.back();
auto tables_in_select_query_element_ast = std::make_shared<ASTTablesInSelectQueryElement>();
tables_in_select_query_element_ast->children.push_back(std::move(table_expression_ast));
tables_in_select_query_element_ast->table_expression = tables_in_select_query_element_ast->children.back();
auto tables_in_select_query_ast = std::make_shared<ASTTablesInSelectQuery>();
tables_in_select_query_ast->children.push_back(std::move(tables_in_select_query_element_ast));
select_query->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables_in_select_query_ast));
if (getPrewhere())
select_query->setExpression(ASTSelectQuery::Expression::PREWHERE, getPrewhere()->toAST());
if (getWhere())
select_query->setExpression(ASTSelectQuery::Expression::WHERE, getWhere()->toAST());
return select_query;
}
QueryTreeNodePtr QueryNode::cloneImpl() const
{
return std::make_shared<QueryNode>();
}
}

118
src/Analyzer/QueryNode.h Normal file
View File

@ -0,0 +1,118 @@
#pragma once
#include <Analyzer/Identifier.h>
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/ListNode.h>
namespace DB
{
/** Query node represents query in query tree.
* TODO: CTE.
*/
class QueryNode;
using QueryNodePtr = std::shared_ptr<QueryNode>;
class QueryNode final : public IQueryTreeNode
{
public:
QueryNode();
const ListNode & getWith() const
{
return children[with_child_index]->as<const ListNode &>();
}
ListNode & getWith()
{
return children[with_child_index]->as<ListNode &>();
}
const QueryTreeNodePtr & getWithNode() const
{
return children[with_child_index];
}
QueryTreeNodePtr & getWithNode()
{
return children[with_child_index];
}
const ListNode & getProjection() const
{
return children[projection_child_index]->as<const ListNode &>();
}
ListNode & getProjection()
{
return children[projection_child_index]->as<ListNode &>();
}
const QueryTreeNodePtr & getProjectionNode() const
{
return children[projection_child_index];
}
QueryTreeNodePtr & getProjectionNode()
{
return children[projection_child_index];
}
const QueryTreeNodePtr & getFrom() const
{
return children[from_child_index];
}
QueryTreeNodePtr & getFrom()
{
return children[from_child_index];
}
const QueryTreeNodePtr & getPrewhere() const
{
return children[prewhere_child_index];
}
QueryTreeNodePtr & getPrewhere()
{
return children[prewhere_child_index];
}
const QueryTreeNodePtr & getWhere() const
{
return children[where_child_index];
}
QueryTreeNodePtr & getWhere()
{
return children[where_child_index];
}
QueryTreeNodeType getNodeType() const override
{
return QueryTreeNodeType::QUERY;
}
void dumpTree(WriteBuffer & buffer, size_t indent) const override;
protected:
void updateTreeHashImpl(HashState &) const override;
ASTPtr toASTImpl() const override;
QueryTreeNodePtr cloneImpl() const override;
private:
static constexpr size_t with_child_index = 0;
static constexpr size_t projection_child_index = 1;
static constexpr size_t from_child_index = 2;
static constexpr size_t prewhere_child_index = 3;
static constexpr size_t where_child_index = 4;
static constexpr size_t group_by_child_index = 5;
static constexpr size_t having_child_index = 6;
static constexpr size_t order_by_child_index = 7;
static constexpr size_t limit_child_index = 8;
static constexpr size_t children_size = where_child_index + 1;
};
}

View File

@ -0,0 +1,550 @@
#include <Analyzer/QueryTreeBuilder.h>
#include <Common/FieldVisitorToString.h>
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTQualifiedAsterisk.h>
#include <Parsers/ASTColumnsMatcher.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTWithElement.h>
#include <Parsers/ASTColumnsTransformers.h>
#include <Analyzer/IdentifierNode.h>
#include <Analyzer/MatcherNode.h>
#include <Analyzer/ColumnTransformers.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/LambdaNode.h>
#include <Analyzer/TableNode.h>
#include <Analyzer/QueryNode.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Databases/IDatabase.h>
#include <Interpreters/StorageID.h>
#include <Interpreters/Context.h>
#include <Functions/FunctionFactory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
extern const int UNKNOWN_DATABASE;
extern const int UNKNOWN_TABLE;
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
}
class QueryTreeBuilder : public WithContext
{
public:
QueryTreeBuilder(ASTPtr query_, ContextPtr context_);
QueryTreeNodePtr getQueryTreeNode()
{
return query_tree_node;
}
private:
QueryTreeNodePtr getSelectWithUnionExpression(const ASTPtr & select_with_union_query, bool is_scalar_query) const;
QueryTreeNodePtr getSelectExpression(const ASTPtr & select_query, bool is_scalar_query) const;
QueryTreeNodePtr getExpressionList(const ASTPtr & expression_list) const;
QueryTreeNodePtr getExpression(const ASTPtr & expression) const;
QueryTreeNodePtr getFromNode(const ASTPtr & tables_in_select_query) const;
ColumnTransformersNodes getColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index) const;
StoragePtr resolveStorage(const Identifier & storage_identifier) const;
ASTPtr query;
QueryTreeNodePtr query_tree_node;
};
QueryTreeBuilder::QueryTreeBuilder(ASTPtr query_, ContextPtr context_)
: WithContext(context_)
, query(query_->clone())
{
if (query->as<ASTSelectWithUnionQuery>())
query_tree_node = getSelectWithUnionExpression(query, false /*is_scalar_query*/);
else if (query->as<ASTSelectQuery>())
query_tree_node = getSelectExpression(query, false /*is_scalar_query*/);
else if (query->as<ASTExpressionList>())
query_tree_node = getExpressionList(query);
else
query_tree_node = getExpression(query);
}
QueryTreeNodePtr QueryTreeBuilder::getSelectWithUnionExpression(const ASTPtr & select_with_union_query, bool is_scalar_query) const
{
auto & select_with_union_query_typed = select_with_union_query->as<ASTSelectWithUnionQuery &>();
auto & select_lists = select_with_union_query_typed.list_of_selects->as<ASTExpressionList &>();
if (select_lists.children.size() == 1)
{
return getSelectExpression(select_with_union_query->children[0]->children[0], is_scalar_query);
}
else
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "UNION is not supported");
// auto union_expression = UnionExpression::create(is_scalar_query);
// union_expression->getModes() = select_with_union_query_typed.list_of_modes;
// union_expression->getModesSet() = select_with_union_query_typed.set_of_modes;
// auto & select_expressions = union_expression->getSelectExpressions();
// select_expressions.reserve(select_lists.children.size());
// for (const auto & select : select_lists.children)
// {
// auto expression = getSelectExpression(select, false);
// select_expressions.emplace_back(std::move(expression));
// }
// return union_expression;
}
}
QueryTreeNodePtr QueryTreeBuilder::getSelectExpression(const ASTPtr & select_query, bool) const
{
const auto & select_query_typed = select_query->as<ASTSelectQuery &>();
auto current_query_tree = std::make_shared<QueryNode>();
current_query_tree->getFrom() = getFromNode(select_query_typed.tables());
current_query_tree->setOriginalAST(select_query);
auto select_with_list = select_query_typed.with();
if (select_with_list)
{
auto & select_with_list_typed = select_with_list->as<ASTExpressionList &>();
for (auto & expression_part : select_with_list_typed.children)
{
auto expression_node = getExpression(expression_part);
current_query_tree->getWith().getNodes().push_back(expression_node);
}
}
auto select_expression_list = select_query_typed.select();
if (select_expression_list)
{
auto & select_expression_list_typed = select_expression_list->as<ASTExpressionList &>();
for (auto & expression_part : select_expression_list_typed.children)
{
auto expression_node = getExpression(expression_part);
current_query_tree->getProjection().getNodes().push_back(expression_node);
}
}
auto prewhere_expression = select_query_typed.prewhere();
if (prewhere_expression)
current_query_tree->getPrewhere() = getExpression(prewhere_expression);
auto where_expression = select_query_typed.where();
if (where_expression)
current_query_tree->getWhere() = getExpression(where_expression);
return current_query_tree;
}
QueryTreeNodePtr QueryTreeBuilder::getExpressionList(const ASTPtr & expression_list) const
{
auto list_node = std::make_shared<ListNode>();
auto & expression_list_typed = expression_list->as<ASTExpressionList &>();
list_node->getNodes().reserve(expression_list_typed.children.size());
for (auto & expression : expression_list_typed.children)
{
auto expression_node = getExpression(expression);
list_node->getNodes().push_back(std::move(expression_node));
}
return list_node;
}
QueryTreeNodePtr QueryTreeBuilder::getExpression(const ASTPtr & expression) const
{
QueryTreeNodePtr result;
if (const auto * ast_identifier = expression->as<ASTIdentifier>())
{
/// TODO: Identifier as query parameter
auto identifier = Identifier(ast_identifier->name_parts);
result = std::make_shared<IdentifierNode>(std::move(identifier));
}
else if (const auto * asterisk = expression->as<ASTAsterisk>())
{
auto column_transformers = getColumnTransformers(expression, 0 /*start_child_index*/);
result = std::make_shared<MatcherNode>(column_transformers);
}
else if (const auto * qualified_asterisk = expression->as<ASTQualifiedAsterisk>())
{
/// TODO: Identifier with UUID
/// TODO: Currently during query analysis stage we support qualified matchers with any identifier length
/// but ASTTableIdentifier can contain only 2 parts.
auto & qualified_identifier = qualified_asterisk->children.at(0)->as<ASTTableIdentifier &>();
auto column_transformers = getColumnTransformers(expression, 1 /*start_child_index*/);
result = std::make_shared<MatcherNode>(Identifier(qualified_identifier.name_parts), column_transformers);
}
else if (const auto * ast_literal = expression->as<ASTLiteral>())
{
result = std::make_shared<ConstantNode>(ast_literal->value);
}
else if (const auto * function = expression->as<ASTFunction>())
{
if (function->is_lambda_function)
{
const auto & lambda_arguments_and_expression = function->arguments->as<ASTExpressionList &>().children;
auto & lambda_arguments_tuple = lambda_arguments_and_expression.at(0)->as<ASTFunction &>();
auto lambda_arguments_nodes = std::make_shared<ListNode>();
Names lambda_arguments;
NameSet lambda_arguments_set;
if (lambda_arguments_tuple.arguments)
{
const auto & lambda_arguments_list = lambda_arguments_tuple.arguments->as<ASTExpressionList>()->children;
for (const auto & lambda_argument : lambda_arguments_list)
{
const auto * lambda_argument_identifier = lambda_argument->as<ASTIdentifier>();
if (!lambda_argument_identifier)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Lambda {} argument is not identifier",
function->formatForErrorMessage());
if (lambda_argument_identifier->name_parts.size() > 1)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Lambda {} argument identifier must contain only argument name. Actual {}",
function->formatForErrorMessage(),
lambda_argument_identifier->full_name);
const auto & argument_name = lambda_argument_identifier->name_parts[0];
auto [_, inserted] = lambda_arguments_set.insert(argument_name);
if (!inserted)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Lambda {} multiple arguments with same name {}",
function->formatForErrorMessage(),
argument_name);
lambda_arguments.push_back(argument_name);
}
}
const auto & lambda_expression = lambda_arguments_and_expression.at(1);
auto lambda_expression_node = getExpression(lambda_expression);
result = std::make_shared<LambdaNode>(std::move(lambda_arguments), std::move(lambda_expression_node));
}
else
{
auto function_node = std::make_shared<FunctionNode>(function->name);
if (function->parameters)
{
const auto & function_parameters_list = function->parameters->as<ASTExpressionList>()->children;
for (const auto & argument : function_parameters_list)
function_node->getParameters().getNodes().push_back(getExpression(argument));
}
if (function->arguments)
{
const auto & function_arguments_list = function->arguments->as<ASTExpressionList>()->children;
for (const auto & argument : function_arguments_list)
function_node->getArguments().getNodes().push_back(getExpression(argument));
}
result = function_node;
}
}
else if (const auto * subquery = expression->as<ASTSubquery>())
{
auto subquery_query = subquery->children[0];
auto query_node = getSelectWithUnionExpression(subquery_query, false);
result = query_node;
}
else if (const auto * columns_regexp_matcher = expression->as<ASTColumnsRegexpMatcher>())
{
auto column_transformers = getColumnTransformers(expression, 0 /*start_child_index*/);
result = std::make_shared<MatcherNode>(columns_regexp_matcher->getMatcher(), std::move(column_transformers));
}
else if (const auto * columns_list_matcher = expression->as<ASTColumnsListMatcher>())
{
Identifiers column_list_identifiers;
column_list_identifiers.reserve(columns_list_matcher->column_list->children.size());
for (auto & column_list_child : columns_list_matcher->column_list->children)
{
auto & column_list_identifier = column_list_child->as<ASTIdentifier &>();
column_list_identifiers.emplace_back(Identifier{column_list_identifier.name_parts});
}
auto column_transformers = getColumnTransformers(expression, 0 /*start_child_index*/);
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 = getColumnTransformers(expression, 1 /*start_child_index*/);
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 &>();
Identifiers column_list_identifiers;
column_list_identifiers.reserve(qualified_columns_list_matcher->column_list->children.size());
for (auto & column_list_child : qualified_columns_list_matcher->column_list->children)
{
auto & column_list_identifier = column_list_child->as<ASTIdentifier &>();
column_list_identifiers.emplace_back(Identifier{column_list_identifier.name_parts});
}
auto column_transformers = getColumnTransformers(expression, 1 /*start_child_index*/);
result = std::make_shared<MatcherNode>(Identifier(qualified_identifier.name_parts), column_list_identifiers, std::move(column_transformers));
}
else
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Only literals and constants are supported as expression. Actual {}", expression->formatForErrorMessage());
}
result->setAlias(expression->tryGetAlias());
result->setOriginalAST(expression);
return result;
}
QueryTreeNodePtr QueryTreeBuilder::getFromNode(const ASTPtr & tables_in_select_query) const
{
if (!tables_in_select_query)
{
/** If no table is specified in SELECT query we substitude system.one table.
* SELECT * FROM system.one;
*/
Identifier storage_identifier("system.one");
auto table = resolveStorage(storage_identifier);
return std::make_shared<TableNode>(table, getContext());
}
auto & tables = tables_in_select_query->as<ASTTablesInSelectQuery &>();
if (tables.children.size() > 1)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Multiple tables are not supported");
for (const auto & table_element_untyped : tables.children)
{
const auto & table_element = table_element_untyped->as<ASTTablesInSelectQueryElement &>();
if (table_element.table_expression)
{
auto * table_expression = table_element.table_expression->as<ASTTableExpression>();
if (table_expression->database_and_table_name)
{
/// Add CTE support
auto & table_identifier_typed = table_expression->database_and_table_name->as<ASTTableIdentifier &>();
auto storage_identifier = Identifier(table_identifier_typed.name_parts);
auto table = resolveStorage(storage_identifier);
auto node = std::make_shared<TableNode>(table, getContext());
node->setAlias(table_identifier_typed.tryGetAlias());
node->setOriginalAST(table_element.table_expression);
return node;
}
// else if (auto * subquery_expression = table_expression->subquery->as<ASTSubquery>())
// {
// const auto & select_with_union_query = subquery_expression->children[0];
// auto expression = getSelectWithUnionExpression(select_with_union_query, false /*scalar query*/);
// expression->setAlias(subquery_expression->tryGetAlias());
// if (auto * select_expression = expression->as<SelectExpression>())
// scope->addInnerScope(select_expression->getScope());
// table_expressions.emplace_back(std::move(expression));
// }
else
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Only table is supported");
}
}
// if (table_element.table_join)
// {
// const auto & table_join = table_element.table_join->as<ASTTableJoin>();
// auto right_table_expression = std::move(table_expressions.back());
// table_expressions.pop_back();
// auto left_table_expression = std::move(table_expressions.back());
// table_expressions.pop_back();
// auto join_expression = JoinExpression::create();
// join_expression->getLeftTableExpression() = left_table_expression;
// join_expression->getRightTableExpression() = right_table_expression;
// if (table_join->using_expression_list)
// join_expression->getUsingExpressions() = getExpressionElements(table_join->using_expression_list, scope);
// if (table_join->on_expression)
// {
// join_expression->getOnExpression() = getExpressionElement(table_join->on_expression, scope);
// }
// table_expressions.emplace_back(std::move(join_expression));
// }
// if (table_element.array_join)
// {
// auto array_join_array_expression = table_element.array_join->children[0]->children[0];
// auto expression_element = getExpressionElement(array_join_array_expression, scope);
// expression_element->setAlias(array_join_array_expression->tryGetAlias());
// auto last_table_expression = std::move(table_expressions.back());
// table_expressions.pop_back();
// auto array_join_expression = ArrayJoinExpression::create();
// array_join_expression->getLeftTableExpression() = std::move(last_table_expression);
// array_join_expression->getArrayExpression() = std::move(expression_element);
// table_expressions.push_back(array_join_expression);
// }
}
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "From section is unsupported");
// if (table_expressions.empty())
// throw Exception(ErrorCodes::LOGICAL_ERROR, "QueryAnalyzer from cannot be empty");
// return table_expressions.back();
}
ColumnTransformersNodes QueryTreeBuilder::getColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index) const
{
ColumnTransformersNodes column_transformers;
size_t children_size = matcher_expression->children.size();
for (; start_child_index < children_size; ++start_child_index)
{
const auto & child = matcher_expression->children[start_child_index];
if (auto * apply_transformer = child->as<ASTColumnsApplyTransformer>())
{
if (apply_transformer->lambda)
{
auto lambda_query_tree_node = getExpression(apply_transformer->lambda);
column_transformers.emplace_back(std::make_shared<ApplyColumnTransformerNode>(std::move(lambda_query_tree_node)));
}
else
{
auto function_node = std::make_shared<FunctionNode>(apply_transformer->func_name);
if (apply_transformer->parameters)
function_node->getParametersNode() = getExpressionList(apply_transformer->parameters);
column_transformers.emplace_back(std::make_shared<ApplyColumnTransformerNode>(std::move(function_node)));
}
}
else if (auto * except_transformer = child->as<ASTColumnsExceptTransformer>())
{
auto matcher = except_transformer->getMatcher();
if (matcher)
{
column_transformers.emplace_back(std::make_shared<ExceptColumnTransformerNode>(std::move(matcher)));
}
else
{
Names except_column_names;
except_column_names.reserve(except_transformer->children.size());
for (auto & except_transformer_child : except_transformer->children)
except_column_names.push_back(except_transformer_child->as<ASTIdentifier &>().full_name);
column_transformers.emplace_back(std::make_shared<ExceptColumnTransformerNode>(std::move(except_column_names)));
}
}
else if (auto * replace_transformer = child->as<ASTColumnsReplaceTransformer>())
{
std::vector<ReplaceColumnTransformerNode::Replacement> replacements;
replacements.reserve(replace_transformer->children.size());
for (const auto & replace_transformer_child : replace_transformer->children)
{
auto & replacement = replace_transformer_child->as<ASTColumnsReplaceTransformer::Replacement &>();
replacements.emplace_back(ReplaceColumnTransformerNode::Replacement{replacement.name, getExpression(replacement.expr)});
}
column_transformers.emplace_back(std::make_shared<ReplaceColumnTransformerNode>(replacements));
}
else
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Unsupported column matcher {}", child->formatForErrorMessage());
}
}
return column_transformers;
}
StoragePtr QueryTreeBuilder::resolveStorage(const Identifier & storage_identifier) const
{
size_t parts_size = storage_identifier.getPartsSize();
if (parts_size < 1 || parts_size > 2)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table identifier should consist of 1 or 2 parts");
std::string database_name;
std::string table_name;
if (storage_identifier.isCompound())
{
database_name = storage_identifier[0];
table_name = storage_identifier[1];
}
else
{
table_name = storage_identifier[0];
}
auto current_context = getContext();
if (database_name.empty())
database_name = current_context->getCurrentDatabase();
auto & database_catalog = DatabaseCatalog::instance();
auto database = database_catalog.tryGetDatabase(database_name);
if (!database)
throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database {} doesn't exists", database_name);
auto table = database->tryGetTable(table_name, current_context);
if (!table)
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exists", table_name);
return table;
}
QueryTreeNodePtr buildQueryTree(ASTPtr query, ContextPtr context)
{
QueryTreeBuilder builder(query, context);
return builder.getQueryTreeNode();
}
}

View File

@ -0,0 +1,19 @@
#pragma once
#include <Parsers/IAST_fwd.h>
#include <Storages/IStorage_fwd.h>
#include <Interpreters/Context_fwd.h>
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/ColumnTransformers.h>
namespace DB
{
/** Build query tree from AST.
* AST that represent query ASTSelectWithUnionQuery, ASTSelectQuery.
* AST that represent a list of expressions ASTExpressionList.
* AST that represent expression ASTIdentifier, ASTAsterisk, ASTLiteral, ASTFunction.
*/
QueryTreeNodePtr buildQueryTree(ASTPtr query, ContextPtr context);
}

View File

@ -0,0 +1,91 @@
#include <Analyzer/QueryTreePassManager.h>
#include <Analyzer/QueryAnalysisPass.h>
#include <Analyzer/MultiIfToIfPass.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <Interpreters/Context.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
QueryTreePassManager::QueryTreePassManager(ContextPtr context_) : WithContext(context_) {}
void QueryTreePassManager::addPass(QueryTreePassPtr pass)
{
passes.push_back(std::move(pass));
}
void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node)
{
auto current_context = getContext();
size_t optimizations_size = passes.size();
for (size_t i = 0; i < optimizations_size; ++i)
passes[i]->run(query_tree_node, current_context);
}
void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node, size_t up_to_pass_index)
{
size_t optimizations_size = passes.size();
if (up_to_pass_index > optimizations_size)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Requested to run optimizations up to {} pass. There are only {} pass",
up_to_pass_index,
optimizations_size);
auto current_context = getContext();
for (size_t i = 0; i < up_to_pass_index; ++i)
passes[i]->run(query_tree_node, current_context);
}
void QueryTreePassManager::dump(WriteBuffer & buffer)
{
size_t passes_size = passes.size();
for (size_t i = 0; i < passes_size; ++i)
{
auto & pass = passes[i];
buffer << "Pass " << (i + 1) << ' ' << pass->getName() << " - " << pass->getDescription();
if (i < passes_size)
buffer << '\n';
}
}
void QueryTreePassManager::dump(WriteBuffer & buffer, size_t up_to_pass_index)
{
size_t optimizations_size = passes.size();
if (up_to_pass_index > optimizations_size)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Requested to dump optimizations up to {} pass. There are only {} pass",
up_to_pass_index,
optimizations_size);
for (size_t i = 0; i < up_to_pass_index; ++i)
{
auto & pass = passes[i];
buffer << "Pass " << (i + 1) << " " << pass->getName() << " - " << pass->getDescription();
if (i < up_to_pass_index)
buffer << '\n';
}
}
void addQueryTreePasses(QueryTreePassManager & manager)
{
auto context = manager.getContext();
const auto & settings = context->getSettingsRef();
manager.addPass(std::make_shared<QueryAnalysisPass>());
if (settings.optimize_multiif_to_if)
manager.addPass(std::make_shared<MultiIfToIfPass>());
}
}

View File

@ -0,0 +1,49 @@
#pragma once
#include <Analyzer/IQueryTreePass.h>
#include <Interpreters/Context_fwd.h>
namespace DB
{
/** Query tree pass manager provide functionality to register and run passes
* on query tree.
*/
class QueryTreePassManager : public WithContext
{
public:
explicit QueryTreePassManager(ContextPtr context_);
/// Get registered passes
const std::vector<QueryTreePassPtr> & getPasses() const
{
return passes;
}
/// Add query tree pass
void addPass(QueryTreePassPtr pass);
/// Run query tree passes on query tree
void run(QueryTreeNodePtr query_tree_node);
/** Run query tree passes on query tree up to up_to_pass_index.
* If up_to_pass_index is greater than passes size throws exception.
*/
void run(QueryTreeNodePtr query_tree_node, size_t up_to_pass_index);
/// Dump query tree passes
void dump(WriteBuffer & buffer);
/** Dump query tree passes to up_to_pass_index.
* If up_to_pass_index is greater than passes size throws exception.
*/
void dump(WriteBuffer & buffer, size_t up_to_pass_index);
private:
std::vector<QueryTreePassPtr> passes;
};
void addQueryTreePasses(QueryTreePassManager & manager);
}

View File

@ -0,0 +1,60 @@
#include <Analyzer/TableNode.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <Parsers/ASTIdentifier.h>
#include <Storages/IStorage.h>
#include <Interpreters/Context.h>
namespace DB
{
TableNode::TableNode(StoragePtr storage_, ContextPtr context)
: storage(std::move(storage_))
, storage_id(storage->getStorageID())
, table_lock(storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout))
, storage_snapshot(storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context))
{
}
void TableNode::dumpTree(WriteBuffer & buffer, size_t indent) const
{
buffer << std::string(indent, ' ') << "TABLE ";
writePointerHex(this, buffer);
buffer << ' ' << storage_id.getFullNameNotQuoted();
}
void TableNode::updateTreeHashImpl(HashState & state) const
{
auto full_name = storage_id.getFullNameNotQuoted();
state.update(full_name.size());
state.update(full_name);
}
String TableNode::getName() const
{
return storage->getStorageID().getFullNameNotQuoted();
}
ASTPtr TableNode::toASTImpl() const
{
return std::make_shared<ASTTableIdentifier>(storage_id.getDatabaseName(), storage_id.getTableName());
}
QueryTreeNodePtr TableNode::cloneImpl() const
{
TableNodePtr result_table_node(new TableNode());
result_table_node->storage = storage;
result_table_node->storage_id = storage_id;
result_table_node->table_lock = table_lock;
result_table_node->storage_snapshot = storage_snapshot;
return result_table_node;
}
}

90
src/Analyzer/TableNode.h Normal file
View File

@ -0,0 +1,90 @@
#pragma once
#include <Storages/IStorage_fwd.h>
#include <Storages/TableLockHolder.h>
#include <Storages/StorageSnapshot.h>
#include <Interpreters/Context_fwd.h>
#include <Interpreters/StorageID.h>
#include <Analyzer/IQueryTreeNode.h>
namespace DB
{
/** Table node represents table in query tree.
* Example: SELECT a FROM test_table.
* test_table - is identifier, that during query analysis pass must be resolved into table node.
*
* During construction table node:
* 1. Lock storage for share. Later lock can be moved out of node using `moveTableLock` method.
* 2. Take storage snapshot.
*/
class TableNode;
using TableNodePtr = std::shared_ptr<TableNode>;
class TableNode : public IQueryTreeNode
{
public:
/// Construct table node with storage and context
explicit TableNode(StoragePtr storage_, ContextPtr context);
/// Get storage
const StoragePtr & getStorage() const
{
return storage;
}
/// Get storage id
const StorageID & getStorageID() const
{
return storage_id;
}
/// Get storage snapshot
const StorageSnapshotPtr & getStorageSnapshot() const
{
return storage_snapshot;
}
/// Get table lock
const TableLockHolder & getTableLock() const
{
return table_lock;
}
/** Move table lock out of table node.
* After using this method table node state becomes invalid.
*/
TableLockHolder && moveTableLock()
{
return std::move(table_lock);
}
QueryTreeNodeType getNodeType() const override
{
return QueryTreeNodeType::TABLE;
}
void dumpTree(WriteBuffer & buffer, size_t indent) const override;
String getName() const override;
protected:
void updateTreeHashImpl(HashState & state) const override;
ASTPtr toASTImpl() const override;
QueryTreeNodePtr cloneImpl() const override;
private:
TableNode() : storage_id("", "") {}
StoragePtr storage;
StorageID storage_id;
TableLockHolder table_lock;
StorageSnapshotPtr storage_snapshot;
};
}

View File

@ -0,0 +1,3 @@
add_executable (query_analyzer query_analyzer.cpp)
target_include_directories (query_analyzer SYSTEM BEFORE PRIVATE ${SPARSEHASH_INCLUDE_DIR})
target_link_libraries (query_analyzer PRIVATE dbms)

View File

@ -0,0 +1,11 @@
#include <iostream>
using namespace DB;
int main(int argc, char ** argv)
{
(void)(argc);
(void)(argv);
return 0;
}

View File

View File

@ -0,0 +1,227 @@
#include <gtest/gtest.h>
#include <Analyzer/Identifier.h>
using namespace DB;
TEST(Identifier, IdentifierBasics)
{
{
Identifier identifier;
ASSERT_TRUE(identifier.empty());
ASSERT_TRUE(identifier.isEmpty());
ASSERT_EQ(identifier.getPartsSize(), 0);
ASSERT_FALSE(identifier.isShort());
ASSERT_FALSE(identifier.isCompound());
ASSERT_FALSE(identifier.startsWith("test"));
ASSERT_FALSE(identifier.endsWith("test"));
ASSERT_EQ(identifier.begin(), identifier.end());
ASSERT_EQ(identifier.getFullName(), "");
}
{
Identifier identifier("value");
ASSERT_FALSE(identifier.empty());
ASSERT_FALSE(identifier.isEmpty());
ASSERT_EQ(identifier.getPartsSize(), 1);
ASSERT_TRUE(identifier.isShort());
ASSERT_FALSE(identifier.isCompound());
ASSERT_EQ(identifier.front(), "value");
ASSERT_EQ(identifier.back(), "value");
ASSERT_FALSE(identifier.startsWith("test"));
ASSERT_FALSE(identifier.endsWith("test"));
ASSERT_TRUE(identifier.startsWith("value"));
ASSERT_TRUE(identifier.endsWith("value"));
ASSERT_EQ(identifier[0], "value");
ASSERT_NE(identifier.begin(), identifier.end());
ASSERT_EQ(identifier.getFullName(), "value");
}
{
Identifier identifier("value1.value2");
ASSERT_FALSE(identifier.empty());
ASSERT_FALSE(identifier.isEmpty());
ASSERT_EQ(identifier.getPartsSize(), 2);
ASSERT_FALSE(identifier.isShort());
ASSERT_TRUE(identifier.isCompound());
ASSERT_EQ(identifier.front(), "value1");
ASSERT_EQ(identifier.back(), "value2");
ASSERT_FALSE(identifier.startsWith("test"));
ASSERT_FALSE(identifier.endsWith("test"));
ASSERT_TRUE(identifier.startsWith("value1"));
ASSERT_TRUE(identifier.endsWith("value2"));
ASSERT_EQ(identifier[0], "value1");
ASSERT_EQ(identifier[1], "value2");
ASSERT_NE(identifier.begin(), identifier.end());
ASSERT_EQ(identifier.getFullName(), "value1.value2");
}
{
Identifier identifier1("value1.value2");
Identifier identifier2("value1.value2");
ASSERT_EQ(identifier1, identifier2);
}
{
Identifier identifier1("value1.value2");
Identifier identifier2("value1.value3");
ASSERT_NE(identifier1, identifier2);
}
}
TEST(Identifier, IdentifierPopParts)
{
{
Identifier identifier("value1.value2.value3");
ASSERT_EQ(identifier.getFullName(), "value1.value2.value3");
identifier.popLast();
ASSERT_EQ(identifier.getFullName(), "value1.value2");
identifier.popLast();
ASSERT_EQ(identifier.getFullName(), "value1");
identifier.popLast();
ASSERT_EQ(identifier.getFullName(), "");
ASSERT_TRUE(identifier.isEmpty());
}
{
Identifier identifier("value1.value2.value3");
ASSERT_EQ(identifier.getFullName(), "value1.value2.value3");
identifier.popFirst();
ASSERT_EQ(identifier.getFullName(), "value2.value3");
identifier.popFirst();
ASSERT_EQ(identifier.getFullName(), "value3");
identifier.popFirst();
ASSERT_EQ(identifier.getFullName(), "");
ASSERT_TRUE(identifier.isEmpty());
}
{
Identifier identifier("value1.value2.value3");
ASSERT_EQ(identifier.getFullName(), "value1.value2.value3");
identifier.popLast();
ASSERT_EQ(identifier.getFullName(), "value1.value2");
identifier.popFirst();
ASSERT_EQ(identifier.getFullName(), "value2");
identifier.popLast();
ASSERT_EQ(identifier.getFullName(), "");
ASSERT_TRUE(identifier.isEmpty());
}
}
TEST(Identifier, IdentifierViewBasics)
{
{
Identifier identifier;
IdentifierView identifier_view(identifier);
ASSERT_TRUE(identifier_view.empty());
ASSERT_TRUE(identifier_view.isEmpty());
ASSERT_EQ(identifier_view.getPartsSize(), 0);
ASSERT_FALSE(identifier_view.isShort());
ASSERT_FALSE(identifier_view.isCompound());
ASSERT_FALSE(identifier_view.startsWith("test"));
ASSERT_FALSE(identifier_view.endsWith("test"));
ASSERT_EQ(identifier_view.begin(), identifier_view.end());
ASSERT_EQ(identifier_view.getFullName(), "");
}
{
Identifier identifier("value");
IdentifierView identifier_view(identifier);
ASSERT_FALSE(identifier_view.empty());
ASSERT_FALSE(identifier_view.isEmpty());
ASSERT_EQ(identifier_view.getPartsSize(), 1);
ASSERT_TRUE(identifier_view.isShort());
ASSERT_FALSE(identifier_view.isCompound());
ASSERT_EQ(identifier_view.front(), "value");
ASSERT_EQ(identifier_view.back(), "value");
ASSERT_FALSE(identifier_view.startsWith("test"));
ASSERT_FALSE(identifier_view.endsWith("test"));
ASSERT_TRUE(identifier_view.startsWith("value"));
ASSERT_TRUE(identifier_view.endsWith("value"));
ASSERT_EQ(identifier_view[0], "value");
ASSERT_NE(identifier_view.begin(), identifier_view.end());
ASSERT_EQ(identifier_view.getFullName(), "value");
}
{
Identifier identifier("value1.value2");
IdentifierView identifier_view(identifier);
ASSERT_FALSE(identifier_view.empty());
ASSERT_FALSE(identifier_view.isEmpty());
ASSERT_EQ(identifier_view.getPartsSize(), 2);
ASSERT_FALSE(identifier_view.isShort());
ASSERT_TRUE(identifier_view.isCompound());
ASSERT_FALSE(identifier_view.startsWith("test"));
ASSERT_FALSE(identifier_view.endsWith("test"));
ASSERT_TRUE(identifier_view.startsWith("value1"));
ASSERT_TRUE(identifier_view.endsWith("value2"));
ASSERT_EQ(identifier_view[0], "value1");
ASSERT_EQ(identifier_view[1], "value2");
ASSERT_NE(identifier_view.begin(), identifier_view.end());
ASSERT_EQ(identifier_view.getFullName(), "value1.value2");
}
{
Identifier identifier1("value1.value2");
IdentifierView identifier_view1(identifier1);
Identifier identifier2("value1.value2");
IdentifierView identifier_view2(identifier2);
ASSERT_EQ(identifier_view1, identifier_view2);
}
{
Identifier identifier1("value1.value2");
IdentifierView identifier_view1(identifier1);
Identifier identifier2("value1.value3");
IdentifierView identifier_view2(identifier2);
ASSERT_NE(identifier_view1, identifier_view2);
}
}
TEST(Identifier, IdentifierViewPopParts)
{
{
Identifier identifier("value1.value2.value3");
IdentifierView identifier_view(identifier);
ASSERT_EQ(identifier_view.getFullName(), "value1.value2.value3");
identifier_view.popLast();
ASSERT_EQ(identifier_view.getFullName(), "value1.value2");
identifier_view.popLast();
ASSERT_EQ(identifier_view.getFullName(), "value1");
identifier_view.popLast();
ASSERT_EQ(identifier_view.getFullName(), "");
ASSERT_TRUE(identifier_view.isEmpty());
}
{
Identifier identifier("value1.value2.value3");
IdentifierView identifier_view(identifier);
ASSERT_EQ(identifier_view.getFullName(), "value1.value2.value3");
identifier_view.popFirst();
ASSERT_EQ(identifier_view.getFullName(), "value2.value3");
identifier_view.popFirst();
ASSERT_EQ(identifier_view.getFullName(), "value3");
identifier_view.popFirst();
ASSERT_EQ(identifier_view.getFullName(), "");
ASSERT_TRUE(identifier_view.isEmpty());
}
{
Identifier identifier("value1.value2.value3");
IdentifierView identifier_view(identifier);
ASSERT_EQ(identifier_view.getFullName(), "value1.value2.value3");
identifier_view.popLast();
ASSERT_EQ(identifier_view.getFullName(), "value1.value2");
identifier_view.popFirst();
ASSERT_EQ(identifier_view.getFullName(), "value2");
identifier_view.popLast();
ASSERT_EQ(identifier_view.getFullName(), "");
ASSERT_TRUE(identifier_view.isEmpty());
}
}

View File

@ -0,0 +1,82 @@
#include <gtest/gtest.h>
#include <DataTypes/DataTypesNumber.h>
#include <Analyzer/Identifier.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/ListNode.h>
using namespace DB;
class SourceNode final : public IQueryTreeNode
{
public:
void updateTreeHashImpl(HashState & hash_state) const override
{
(void)(hash_state);
}
void dumpTree(WriteBuffer & buffer, size_t indent) const override
{
(void)(buffer);
(void)(indent);
}
QueryTreeNodeType getNodeType() const override
{
return QueryTreeNodeType::TABLE;
}
ASTPtr toASTImpl() const override
{
return nullptr;
}
QueryTreeNodePtr cloneImpl() const override
{
return std::make_shared<SourceNode>();
}
};
TEST(QueryTreeNode, Clone)
{
{
auto source_node = std::make_shared<SourceNode>();
NameAndTypePair column_name_and_type("value", std::make_shared<DataTypeUInt64>());
auto column_node = std::make_shared<ColumnNode>(column_name_and_type, source_node);
ASSERT_EQ(column_node->getColumnSource().get(), source_node.get());
auto cloned_column_node = column_node->clone();
/// If in subtree source was not cloned, source pointer must remain same
ASSERT_NE(column_node.get(), cloned_column_node.get());
ASSERT_EQ(cloned_column_node->as<ColumnNode &>().getColumnSource().get(), source_node.get());
}
{
auto root_node = std::make_shared<ListNode>();
auto source_node = std::make_shared<SourceNode>();
NameAndTypePair column_name_and_type("value", std::make_shared<DataTypeUInt64>());
auto column_node = std::make_shared<ColumnNode>(column_name_and_type, source_node);
root_node->getNodes().push_back(source_node);
root_node->getNodes().push_back(column_node);
ASSERT_EQ(column_node->getColumnSource().get(), source_node.get());
auto cloned_root_node = std::static_pointer_cast<ListNode>(root_node->clone());
auto cloned_source_node = cloned_root_node->getNodes()[0];
auto cloned_column_node = std::static_pointer_cast<ColumnNode>(cloned_root_node->getNodes()[1]);
/** If in subtree source was cloned.
* Source pointer for node that was cloned must remain same.
* Source pointer for cloned node must be updated.
*/
ASSERT_NE(column_node.get(), cloned_column_node.get());
ASSERT_NE(source_node.get(), cloned_source_node.get());
ASSERT_EQ(column_node->getColumnSource().get(), source_node.get());
ASSERT_EQ(cloned_column_node->getColumnSource().get(), cloned_source_node.get());
}
}

View File

@ -66,6 +66,7 @@ add_subdirectory (Storages)
add_subdirectory (Parsers)
add_subdirectory (IO)
add_subdirectory (Functions)
add_subdirectory (Analyzer)
add_subdirectory (Interpreters)
add_subdirectory (AggregateFunctions)
add_subdirectory (Client)
@ -254,6 +255,7 @@ add_object_library(clickhouse_datatypes_serializations DataTypes/Serializations)
add_object_library(clickhouse_databases Databases)
add_object_library(clickhouse_databases_mysql Databases/MySQL)
add_object_library(clickhouse_disks Disks)
add_object_library(clickhouse_analyzer Analyzer)
add_object_library(clickhouse_interpreters Interpreters)
add_object_library(clickhouse_interpreters_cache Interpreters/Cache)
add_object_library(clickhouse_interpreters_access Interpreters/Access)

View File

@ -302,6 +302,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(Float, opentelemetry_start_trace_probability, 0., "Probability to start an OpenTelemetry trace for an incoming query.", 0) \
M(Bool, opentelemetry_trace_processors, false, "Collect OpenTelemetry spans for processors.", 0) \
M(Bool, prefer_column_name_to_alias, false, "Prefer using column names instead of aliases if possible.", 0) \
M(Bool, use_analyzer, false, "Use analyzer", 0) \
M(Bool, prefer_global_in_and_join, false, "If enabled, all IN/JOIN operators will be rewritten as GLOBAL IN/JOIN. It's useful when the to-be-joined tables are only available on the initiator and we need to always scatter their data on-the-fly during distributed processing with the GLOBAL keyword. It's also useful to reduce the need to access the external sources joining external tables.", 0) \
\
\

View File

@ -185,8 +185,10 @@ public:
const auto * data_type_function = checkAndGetDataType<DataTypeFunction>(arguments[0].type.get());
if (!data_type_function)
throw Exception("First argument for function " + getName() + " must be a function",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"First argument for function {} must be a function. Actual {}",
getName(),
arguments[0].type->getName());
/// The types of the remaining arguments are already checked in getLambdaArgumentTypes.

View File

@ -82,7 +82,10 @@ public:
const DataTypeTuple * tuple = checkAndGetDataType<DataTypeTuple>(tuple_col);
if (!tuple)
throw Exception("First argument for function " + getName() + " must be tuple or array of tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"First argument for function {} must be tuple or array of tuple. Actual {}",
getName(),
arguments[0].type->getName());
auto index = getElementNum(arguments[1].column, *tuple, number_of_arguments);
if (index.has_value())
@ -137,7 +140,10 @@ public:
const DataTypeTuple * tuple_type_concrete = checkAndGetDataType<DataTypeTuple>(tuple_type);
const ColumnTuple * tuple_col_concrete = checkAndGetColumn<ColumnTuple>(tuple_col);
if (!tuple_type_concrete || !tuple_col_concrete)
throw Exception("First argument for function " + getName() + " must be tuple or array of tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"First argument for function {} must be tuple or array of tuple. Actual {}",
getName(),
first_arg.type->getName());
auto index = getElementNum(arguments[1].column, *tuple_type_concrete, arguments.size());
@ -221,20 +227,18 @@ private:
std::optional<size_t> getElementNum(const ColumnPtr & index_column, const DataTypeTuple & tuple, const size_t argument_size) const
{
if (
checkAndGetColumnConst<ColumnUInt8>(index_column.get())
|| checkAndGetColumnConst<ColumnUInt16>(index_column.get())
|| checkAndGetColumnConst<ColumnUInt32>(index_column.get())
|| checkAndGetColumnConst<ColumnUInt64>(index_column.get())
)
if (checkAndGetColumnConst<ColumnUInt8>(index_column.get())
|| checkAndGetColumnConst<ColumnUInt16>(index_column.get())
|| checkAndGetColumnConst<ColumnUInt32>(index_column.get())
|| checkAndGetColumnConst<ColumnUInt64>(index_column.get()))
{
size_t index = index_column->getUInt(0);
if (index == 0)
throw Exception("Indices in tuples are 1-based.", ErrorCodes::ILLEGAL_INDEX);
throw Exception(ErrorCodes::ILLEGAL_INDEX, "Indices in tuples are 1-based.");
if (index > tuple.getElements().size())
throw Exception("Index for tuple element is out of range.", ErrorCodes::ILLEGAL_INDEX);
throw Exception(ErrorCodes::ILLEGAL_INDEX, "Index for tuple element is out of range.");
return std::optional<size_t>(index - 1);
}
@ -253,7 +257,9 @@ private:
return std::nullopt;
}
else
throw Exception("Second argument to " + getName() + " must be a constant UInt or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Second argument to {} must be a constant UInt or String",
getName());
}
};

View File

@ -7,6 +7,7 @@
#include <TableFunctions/ITableFunction.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterDescribeQuery.h>
#include <Interpreters/IdentifierSemantic.h>
@ -17,7 +18,6 @@
#include <Parsers/TablePropertiesQueriesASTs.h>
#include <DataTypes/NestedUtils.h>
namespace DB
{
@ -60,7 +60,6 @@ Block InterpreterDescribeQuery::getSampleBlock(bool include_subcolumns)
return block;
}
BlockIO InterpreterDescribeQuery::execute()
{
ColumnsDescription columns;
@ -72,8 +71,20 @@ BlockIO InterpreterDescribeQuery::execute()
if (table_expression.subquery)
{
auto names_and_types = InterpreterSelectWithUnionQuery::getSampleBlock(
table_expression.subquery->children.at(0), getContext()).getNamesAndTypesList();
NamesAndTypesList names_and_types;
auto select_query = table_expression.subquery->children.at(0);
auto current_context = getContext();
if (settings.use_analyzer)
{
SelectQueryOptions select_query_options;
names_and_types = InterpreterSelectQueryAnalyzer(select_query, select_query_options, current_context).getSampleBlock().getNamesAndTypesList();
}
else
{
names_and_types = InterpreterSelectWithUnionQuery::getSampleBlock(select_query, current_context).getNamesAndTypesList();
}
columns = ColumnsDescription(std::move(names_and_types));
}
else if (table_expression.table_function)

View File

@ -28,6 +28,9 @@
#include <Common/JSONBuilder.h>
#include <Analyzer/QueryTreeBuilder.h>
#include <Analyzer/QueryTreePassManager.h>
namespace DB
{
@ -155,6 +158,28 @@ struct QueryASTSettings
{"graph", graph},
{"optimize", optimize}
};
std::unordered_map<std::string, std::reference_wrapper<Int64>> integer_settings;
};
struct QueryTreeSettings
{
bool run_passes = false;
Int64 passes = -1;
bool ast = false;
constexpr static char name[] = "QUERYTREE";
std::unordered_map<std::string, std::reference_wrapper<bool>> boolean_settings =
{
{"run_passes", run_passes},
{"ast", ast}
};
std::unordered_map<std::string, std::reference_wrapper<Int64>> integer_settings =
{
{"passes", passes}
};
};
struct QueryPlanSettings
@ -177,6 +202,8 @@ struct QueryPlanSettings
{"json", json},
{"sorting", query_plan_options.sorting},
};
std::unordered_map<std::string, std::reference_wrapper<Int64>> integer_settings;
};
struct QueryPipelineSettings
@ -193,18 +220,31 @@ struct QueryPipelineSettings
{"graph", graph},
{"compact", compact},
};
std::unordered_map<std::string, std::reference_wrapper<Int64>> integer_settings;
};
template <typename Settings>
struct ExplainSettings : public Settings
{
using Settings::boolean_settings;
using Settings::integer_settings;
bool has(const std::string & name_) const
{
return hasBooleanSetting(name_) || hasIntegerSetting(name_);
}
bool hasBooleanSetting(const std::string & name_) const
{
return boolean_settings.count(name_) > 0;
}
bool hasIntegerSetting(const std::string & name_) const
{
return integer_settings.count(name_) > 0;
}
void setBooleanSetting(const std::string & name_, bool value)
{
auto it = boolean_settings.find(name_);
@ -214,6 +254,15 @@ struct ExplainSettings : public Settings
it->second.get() = value;
}
void setIntegerSetting(const std::string & name_, Int64 value)
{
auto it = integer_settings.find(name_);
if (it == integer_settings.end())
throw Exception("Unknown setting for ExplainSettings: " + name_, ErrorCodes::LOGICAL_ERROR);
it->second.get() = value;
}
std::string getSettingsList() const
{
std::string res;
@ -224,6 +273,13 @@ struct ExplainSettings : public Settings
res += setting.first;
}
for (const auto & setting : integer_settings)
{
if (!res.empty())
res += ", ";
res += setting.first;
}
return res;
}
@ -246,15 +302,23 @@ ExplainSettings<Settings> checkAndGetSettings(const ASTPtr & ast_settings)
if (change.value.getType() != Field::Types::UInt64)
throw Exception(ErrorCodes::INVALID_SETTING_VALUE,
"Invalid type {} for setting \"{}\" only boolean settings are supported",
"Invalid type {} for setting \"{}\" only integer settings are supported",
change.value.getTypeName(), change.name);
auto value = change.value.get<UInt64>();
if (value > 1)
throw Exception("Invalid value " + std::to_string(value) + " for setting \"" + change.name +
"\". Only boolean settings are supported", ErrorCodes::INVALID_SETTING_VALUE);
if (settings.hasBooleanSetting(change.name))
{
auto value = change.value.get<UInt64>();
if (value > 1)
throw Exception("Invalid value " + std::to_string(value) + " for setting \"" + change.name +
"\". Expected boolean type", ErrorCodes::INVALID_SETTING_VALUE);
settings.setBooleanSetting(change.name, value);
settings.setBooleanSetting(change.name, value);
}
else
{
auto value = change.value.get<UInt64>();
settings.setIntegerSetting(change.name, value);
}
}
return settings;
@ -304,6 +368,50 @@ QueryPipeline InterpreterExplainQuery::executeImpl()
ast.getExplainedQuery()->format(IAST::FormatSettings(buf, false));
break;
}
case ASTExplainQuery::QueryTree:
{
if (ast.getExplainedQuery()->as<ASTSelectWithUnionQuery>() == nullptr)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Only SELECT is supported for EXPLAIN QUERYTREE query");
auto settings = checkAndGetSettings<QueryTreeSettings>(ast.getSettings());
auto query_tree = buildQueryTree(ast.getExplainedQuery(), getContext());
if (settings.run_passes)
{
auto query_tree_pass_manager = QueryTreePassManager(getContext());
addQueryTreePasses(query_tree_pass_manager);
if (settings.passes < 0)
{
query_tree_pass_manager.dump(buf);
buf << '\n';
query_tree_pass_manager.run(query_tree);
}
else
{
size_t pass_index = static_cast<size_t>(settings.passes);
query_tree_pass_manager.dump(buf, pass_index);
if (pass_index > 0)
buf << '\n';
query_tree_pass_manager.run(query_tree, pass_index);
}
query_tree->dumpTree(buf, 0);
}
else
{
query_tree->dumpTree(buf, 0);
}
if (settings.ast)
{
buf << '\n';
query_tree->toAST()->format(IAST::FormatSettings(buf, false));
}
break;
}
case ASTExplainQuery::QueryPlan:
{
if (!dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))

View File

@ -63,6 +63,7 @@
#include <Interpreters/InterpreterOptimizeQuery.h>
#include <Interpreters/InterpreterRenameQuery.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/InterpreterSetQuery.h>
#include <Interpreters/InterpreterShowCreateQuery.h>
@ -118,6 +119,9 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, ContextMut
if (query->as<ASTSelectQuery>())
{
if (context->getSettingsRef().use_analyzer)
return std::make_unique<InterpreterSelectQueryAnalyzer>(query, options, context);
/// This is internal part of ASTSelectWithUnionQuery.
/// Even if there is SELECT without union, it is represented by ASTSelectWithUnionQuery with single ASTSelectQuery as a child.
return std::make_unique<InterpreterSelectQuery>(query, context, options);
@ -125,6 +129,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, ContextMut
else if (query->as<ASTSelectWithUnionQuery>())
{
ProfileEvents::increment(ProfileEvents::SelectQuery);
if (context->getSettingsRef().use_analyzer)
return std::make_unique<InterpreterSelectQueryAnalyzer>(query, options, context);
return std::make_unique<InterpreterSelectWithUnionQuery>(query, context, options);
}
else if (query->as<ASTSelectIntersectExceptQuery>())

View File

@ -0,0 +1,609 @@
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTExpressionList.h>
#include <Core/QueryProcessingStage.h>
#include <Common/FieldVisitorToString.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/FieldToDataType.h>
#include <Storages/SelectQueryInfo.h>
#include <Storages/IStorage.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/TableNode.h>
#include <Analyzer/LambdaNode.h>
#include <Analyzer/QueryNode.h>
#include <Analyzer/QueryTreeBuilder.h>
#include <Analyzer/QueryTreePassManager.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Functions/FunctionsMiscellaneous.h>
#include <QueryPipeline/Pipe.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <Processors/Sources/NullSource.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
#include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Interpreters/Context.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNSUPPORTED_METHOD;
extern const int LOGICAL_ERROR;
}
QueryPipeline buildDummyPipeline()
{
ColumnsWithTypeAndName columns;
auto string_data_type = std::make_shared<DataTypeString>();
auto string_column = string_data_type->createColumn();
string_column->insert("TestValue");
columns.emplace_back(ColumnWithTypeAndName{std::move(string_column), string_data_type, "test_column"});
Block block(columns);
auto source = std::make_shared<SourceFromSingleChunk>(block);
auto shell_input_pipe = Pipe(std::move(source));
QueryPipeline pipeline(std::move(shell_input_pipe));
return pipeline;
}
String dumpQueryPlan(QueryPlan & query_plan)
{
WriteBufferFromOwnString query_plan_buffer;
query_plan.explainPlan(query_plan_buffer, QueryPlan::ExplainPlanOptions{});
return query_plan_buffer.str();
}
String dumpQueryPipeline(QueryPlan & query_plan)
{
QueryPlan::ExplainPipelineOptions explain_pipeline;
WriteBufferFromOwnString query_pipeline_buffer;
query_plan.explainPipeline(query_pipeline_buffer, explain_pipeline);
return query_pipeline_buffer.str();
}
struct QueryTreeActionsScopeNode
{
explicit QueryTreeActionsScopeNode(ActionsDAGPtr actions_dag_, QueryTreeNodePtr scope_node_)
: actions_dag(std::move(actions_dag_))
, scope_node(std::move(scope_node_))
{
for (const auto & node : actions_dag->getNodes())
node_name_to_node[node.result_name] = &node;
}
bool containsNode(const std::string & node_name)
{
return node_name_to_node.find(node_name) != node_name_to_node.end();
}
const ActionsDAG::Node * tryGetNode(const std::string & node_name)
{
auto it = node_name_to_node.find(node_name);
if (it == node_name_to_node.end())
return {};
return it->second;
}
const ActionsDAG::Node * getNodeOrThrow(const std::string & node_name)
{
auto it = node_name_to_node.find(node_name);
if (it == node_name_to_node.end())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"No node with name {}. There are only nodes {}",
node_name,
actions_dag->dumpNames());
return it->second;
}
const ActionsDAG::Node * addInputColumnIfNecessary(const std::string & node_name, const DataTypePtr & column_type)
{
auto it = node_name_to_node.find(node_name);
if (it != node_name_to_node.end())
return it->second;
const auto * node = &actions_dag->addInput(node_name, column_type);
// std::cout << "QueryTreeActionsScopeNode::addInputColumnIfNecessary dag " << actions_dag << " node name " << node_name;
// std::cout << " result node ptr " << node << std::endl;
node_name_to_node[node->result_name] = node;
return node;
}
const ActionsDAG::Node * addConstantIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column)
{
auto it = node_name_to_node.find(node_name);
if (it != node_name_to_node.end())
return it->second;
const auto * node = &actions_dag->addColumn(column);
// std::cout << "QueryTreeActionsScopeNode::addConstantIfNecessary dag " << actions_dag << " node name " << node_name;
// std::cout << " result node ptr " << node << std::endl;
node_name_to_node[node->result_name] = node;
return node;
}
const ActionsDAG::Node * addFunctionIfNecessary(const std::string & node_name, ActionsDAG::NodeRawConstPtrs children, FunctionOverloadResolverPtr function)
{
auto it = node_name_to_node.find(node_name);
if (it != node_name_to_node.end())
return it->second;
const auto * node = &actions_dag->addFunction(function, children, node_name);
// std::cout << "QueryTreeActionsScopeNode::addFunctionIfNecessary dag " << actions_dag << " node name " << node_name;
// std::cout << " result node ptr " << node << std::endl;
node_name_to_node[node->result_name] = node;
return node;
}
std::unordered_map<std::string_view, const ActionsDAG::Node *> node_name_to_node;
ActionsDAGPtr actions_dag;
QueryTreeNodePtr scope_node;
};
class QueryTreeActionsVisitor : public WithContext
{
public:
explicit QueryTreeActionsVisitor(
ActionsDAGPtr actions_dag,
ContextPtr context_)
: WithContext(context_)
{
actions_stack.emplace_back(std::move(actions_dag), nullptr);
}
ActionsDAG::NodeRawConstPtrs visit(QueryTreeNodePtr expression_node)
{
ActionsDAG::NodeRawConstPtrs result;
if (auto * expression_list_node = expression_node->as<ListNode>())
{
for (auto & node : expression_list_node->getNodes())
{
auto [node_name, _] = visitImpl(node);
result.push_back(actions_stack.front().getNodeOrThrow(node_name));
}
}
else
{
auto [node_name, _] = visitImpl(expression_node);
result.push_back(actions_stack.front().getNodeOrThrow(node_name));
}
return result;
}
private:
std::pair<std::string, size_t> visitImpl(QueryTreeNodePtr node)
{
if (auto * column_node = node->as<ColumnNode>())
return visitColumn(*column_node);
else if (auto * constant_node = node->as<ConstantNode>())
return visitConstant(*constant_node);
else if (auto * function_node = node->as<FunctionNode>())
return visitFunction(*function_node);
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Expected only column, constant or function node. Actual {}", node->formatASTForErrorMessage());
}
std::pair<std::string, size_t> visitColumn(ColumnNode & column)
{
const auto & column_name = column.getColumnName();
Int64 actions_stack_size = static_cast<Int64>(actions_stack.size() - 1);
for (Int64 i = actions_stack_size; i >= 0; --i)
{
actions_stack[i].addInputColumnIfNecessary(column_name, column.getColumnType());
if (column.getColumnSource()->getNodeType() == QueryTreeNodeType::LAMBDA &&
actions_stack[i].scope_node.get() == column.getColumnSource().get())
{
return {column_name, i};
}
}
return {column_name, 0};
}
std::pair<std::string, size_t> visitConstant(ConstantNode & constant_node)
{
const auto & literal = constant_node.getConstantValue();
auto constant_name = constant_node.getName();
ColumnWithTypeAndName column;
column.name = constant_name;
column.type = constant_node.getResultType();
column.column = column.type->createColumnConst(1, literal);
actions_stack[0].addConstantIfNecessary(constant_name, column);
size_t actions_stack_size = actions_stack.size();
for (size_t i = 1; i < actions_stack_size; ++i)
{
auto & actions_stack_node = actions_stack[i];
actions_stack_node.addInputColumnIfNecessary(constant_name, column.type);
}
return {constant_name, 0};
}
std::pair<std::string, size_t> visitLambda(QueryTreeNodePtr lambda_node_untyped)
{
auto & lambda_node = lambda_node_untyped->as<LambdaNode &>();
auto result_type = lambda_node.getResultType();
if (!result_type)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Lambda {} is not resolved during query analysis",
lambda_node.formatASTForErrorMessage());
// std::cout << "QueryTreeActionsVisitor::visitLambda " << lambda_node.formatASTForErrorMessage() << std::endl;
// std::cout << "Lambda arguments nodes size " << lambda_node.getArguments().getNodes().size() << std::endl;
NamesAndTypesList lambda_arguments_names_and_types;
for (auto & lambda_node_argument : lambda_node.getArguments().getNodes())
{
auto lambda_argument_name = lambda_node_argument->getName();
auto lambda_argument_type = lambda_node_argument->getResultType();
// std::cout << "Lambda argument name " << lambda_argument_name;
// std::cout << " type " << lambda_argument_type->getName() << std::endl;
lambda_arguments_names_and_types.emplace_back(lambda_argument_name, lambda_argument_type);
}
size_t previous_scope_node_actions_stack_index = actions_stack.size() - 1;
auto lambda_actions_dag = std::make_shared<ActionsDAG>();
actions_stack.emplace_back(lambda_actions_dag, lambda_node_untyped);
auto [node_name, level] = visitImpl(lambda_node.getExpression());
auto lambda_result_node_name = node_name;
lambda_actions_dag->getIndex().push_back(actions_stack.back().getNodeOrThrow(node_name));
// std::cout << "Previous DAG nodes " << actions_stack[previous_scope_node_actions_stack_index].actions_dag.get() << std::endl;
// for (const auto & previous_actions_node : actions_stack[previous_scope_node_actions_stack_index].actions_dag->getNodes())
// {
// std::cout << "Node " << &previous_actions_node << " result name " << previous_actions_node.result_name << std::endl;
// std::cout << "Children " << previous_actions_node.children.size() << std::endl;
// for (const auto * child_node : previous_actions_node.children)
// {
// std::cout << "Child node " << child_node << " result name " << child_node->result_name << std::endl;
// }
// }
lambda_actions_dag->removeUnusedActions(Names(1, lambda_result_node_name));
// std::cout << "Lambda actions DAG Node " << node_name << " level " << level << std::endl;
// std::cout << "Lambda actions DAG " << lambda_actions_dag.get() << std::endl;
// std::cout << lambda_actions_dag->dumpDAG() << std::endl;
auto lambda_actions = std::make_shared<ExpressionActions>(
lambda_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes));
Names captured_column_names;
ActionsDAG::NodeRawConstPtrs lambda_children;
Names required_column_names = lambda_actions->getRequiredColumns();
const auto & lambda_argument_names = lambda_node.getArgumentNames();
for (const auto & required_column_name : required_column_names)
{
auto it = std::find_if(
lambda_argument_names.begin(), lambda_argument_names.end(), [&](auto & value) { return value == required_column_name; });
if (it == lambda_argument_names.end())
{
lambda_children.push_back(actions_stack[previous_scope_node_actions_stack_index].getNodeOrThrow(required_column_name));
captured_column_names.push_back(required_column_name);
}
}
auto lambda_hash = lambda_node.getTreeHash();
std::string lambda_name = "__lambda_" + toString(lambda_hash.first) + '_' + toString(lambda_hash.second);
auto function_capture = std::make_shared<FunctionCaptureOverloadResolver>(
lambda_actions, captured_column_names, lambda_arguments_names_and_types, result_type, lambda_result_node_name);
actions_stack.pop_back();
if (level == actions_stack.size())
--level;
actions_stack[level].addFunctionIfNecessary(lambda_name, lambda_children, function_capture);
size_t actions_stack_size = actions_stack.size();
for (size_t i = level + 1; i < actions_stack_size; ++i)
{
auto & actions_stack_node = actions_stack[i];
actions_stack_node.addInputColumnIfNecessary(lambda_name, result_type);
}
return {lambda_name, level};
}
std::pair<std::string, size_t> visitFunction(FunctionNode & function_node)
{
auto function_node_name = function_node.getName();
if (function_node.isAggregateFunction())
{
size_t actions_stack_size = actions_stack.size();
for (size_t i = 0; i < actions_stack_size; ++i)
{
auto & actions_stack_node = actions_stack[i];
actions_stack_node.addInputColumnIfNecessary(function_node_name, function_node.getResultType());
}
return {function_node_name, 0};
}
auto & function_arguments = function_node.getArguments().getNodes();
size_t function_arguments_size = function_arguments.size();
Names function_arguments_node_names;
function_arguments_node_names.reserve(function_arguments_size);
size_t level = 0;
for (auto & argument : function_arguments)
{
if (argument->getNodeType() == QueryTreeNodeType::LAMBDA)
{
auto [node_name, node_min_level] = visitLambda(argument);
function_arguments_node_names.push_back(std::move(node_name));
level = std::max(level, node_min_level);
continue;
}
auto [node_name, node_min_level] = visitImpl(argument);
function_arguments_node_names.push_back(std::move(node_name));
level = std::max(level, node_min_level);
}
ActionsDAG::NodeRawConstPtrs children;
children.reserve(function_arguments_size);
for (auto & function_argument_node_name : function_arguments_node_names)
children.push_back(actions_stack[level].getNodeOrThrow(function_argument_node_name));
actions_stack[level].addFunctionIfNecessary(function_node_name, children, function_node.getFunction());
size_t actions_stack_size = actions_stack.size();
for (size_t i = level + 1; i < actions_stack_size; ++i)
{
auto & actions_stack_node = actions_stack[i];
actions_stack_node.addInputColumnIfNecessary(function_node_name, function_node.getResultType());
}
return {function_node_name, level};
}
std::vector<QueryTreeActionsScopeNode> actions_stack;
};
class CollectSourceColumnsMatcher
{
public:
using Visitor = InDepthQueryTreeVisitor<CollectSourceColumnsMatcher, true, false>;
struct Data
{
NameSet source_columns_set;
};
static void visit(QueryTreeNodePtr & node, Data & data)
{
auto * column_node = node->as<ColumnNode>();
if (!column_node)
return;
auto * table_node = column_node->getColumnSource()->as<TableNode>();
if (!table_node)
return;
data.source_columns_set.insert(column_node->getColumnName());
}
static bool needChildVisit(const QueryTreeNodePtr &, const QueryTreeNodePtr &)
{
return true;
}
};
using CollectSourceColumnsVisitor = CollectSourceColumnsMatcher::Visitor;
InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer(const ASTPtr & query_ptr_,
const SelectQueryOptions & select_query_options_,
ContextPtr context_)
: WithContext(context_)
, query_ptr(query_ptr_)
, select_query_options(select_query_options_)
, query_tree_pass_manager(context_)
{
addQueryTreePasses(query_tree_pass_manager);
if (auto * select_with_union_query_typed = query_ptr->as<ASTSelectWithUnionQuery>())
{
auto & select_lists = select_with_union_query_typed->list_of_selects->as<ASTExpressionList &>();
if (select_lists.children.size() == 1)
{
query_ptr = select_lists.children[0];
}
else
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "UNION is not supported");
}
}
else if (auto * select_query_typed = query_ptr->as<ASTSelectQuery>())
{
}
else
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Expected ASTSelectWithUnionQuery or ASTSelectQuery. Actual {}",
query_ptr_->formatForErrorMessage());
}
}
Block InterpreterSelectQueryAnalyzer::getSampleBlock()
{
initializeQueryPlanIfNeeded();
return query_plan.getCurrentDataStream().header;
}
BlockIO InterpreterSelectQueryAnalyzer::execute()
{
initializeQueryPlanIfNeeded();
QueryPlanOptimizationSettings optimization_settings;
BuildQueryPipelineSettings build_pipeline_settings;
auto pipeline_builder = query_plan.buildQueryPipeline(optimization_settings, build_pipeline_settings);
BlockIO res;
res.pipeline = QueryPipelineBuilder::getPipeline(std::move(*pipeline_builder));
return res;
}
void InterpreterSelectQueryAnalyzer::initializeQueryPlanIfNeeded()
{
if (query_plan.isInitialized())
return;
auto query_tree_untyped = buildQueryTree(query_ptr, getContext());
auto query_tree = std::static_pointer_cast<QueryNode>(query_tree_untyped);
auto * table_node = query_tree->getFrom()->as<TableNode>();
if (!table_node)
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Only single table is supported");
query_tree_pass_manager.run(query_tree);
ActionsDAGPtr action_dag = std::make_shared<ActionsDAG>();
ColumnsWithTypeAndName inputs;
CollectSourceColumnsVisitor::Data data;
CollectSourceColumnsVisitor collect_source_columns_visitor(data);
collect_source_columns_visitor.visit(query_tree_untyped);
NameSet source_columns_set = std::move(data.source_columns_set);
// std::cout << "DAG before " << action_dag.get() << " nodes " << action_dag->getNodes().size() << std::endl;
// std::cout << action_dag->dumpDAG() << std::endl;
QueryTreeActionsVisitor visitor(action_dag, getContext());
auto projection_action_dag_nodes = visitor.visit(query_tree->getProjectionNode());
size_t projection_action_dag_nodes_size = projection_action_dag_nodes.size();
// std::cout << "Projection action dag nodes size " << projection_action_dag_nodes_size << std::endl;
// for (size_t i = 0; i < projection_action_dag_nodes_size; ++i)
// {
// std::cout << "DAG node " << projection_action_dag_nodes[i] << std::endl;
// }
// std::cout << "DAG after " << action_dag.get() << " nodes " << action_dag->getNodes().size() << std::endl;
// std::cout << action_dag->dumpDAG() << std::endl;
auto & projection_nodes = query_tree->getProjection().getNodes();
size_t projection_nodes_size = projection_nodes.size();
if (projection_nodes_size != projection_action_dag_nodes_size)
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"QueryTree projection nodes size mismatch. Expected {}. Actual {}",
projection_action_dag_nodes_size,
projection_nodes_size);
NamesWithAliases projection_names;
for (size_t i = 0; i < projection_nodes_size; ++i)
{
auto & node = projection_nodes[i];
const auto * action_dag_node = projection_action_dag_nodes[i];
auto action_dag_node_name = action_dag_node->result_name;
action_dag->getIndex().push_back(action_dag_node);
if (node->hasAlias())
projection_names.push_back({action_dag_node_name, node->getAlias()});
else
projection_names.push_back({action_dag_node_name, action_dag_node_name});
}
action_dag->project(projection_names);
// std::cout << "Final DAG " << action_dag.get() << " nodes " << action_dag->getNodes().size() << std::endl;
// std::cout << action_dag->dumpDAG() << std::endl;
// std::cout << "Names " << action_dag->dumpNames() << std::endl;
// std::cout << "Final DAG nodes " << std::endl;
// for (const auto & node : action_dag->getNodes())
// {
// std::cout << "Node " << &node << " result name " << node.result_name << std::endl;
// }
// std::cout << "Source columns " << source_columns_set.size() << std::endl;
// for (const auto & source_column : source_columns_set)
// std::cout << source_column << std::endl;
auto current_context = getContext();
size_t max_block_size = current_context->getSettingsRef().max_block_size;
size_t max_streams = current_context->getSettingsRef().max_threads;
SelectQueryInfo query_info;
query_info.original_query = query_ptr;
query_info.query = query_ptr;
auto from_stage = table_node->getStorage()->getQueryProcessingStage(
current_context, select_query_options.to_stage, table_node->getStorageSnapshot(), query_info);
Names column_names(source_columns_set.begin(), source_columns_set.end());
if (column_names.empty() && table_node->getStorage()->getName() == "SystemOne")
column_names.push_back("dummy");
if (!column_names.empty())
table_node->getStorage()->read(
query_plan, column_names, table_node->getStorageSnapshot(), query_info, getContext(), from_stage, max_block_size, max_streams);
/// Create step which reads from empty source if storage has no data.
if (!query_plan.isInitialized())
{
auto source_header = table_node->getStorageSnapshot()->getSampleBlockForColumns(column_names);
Pipe pipe(std::make_shared<NullSource>(source_header));
auto read_from_pipe = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
read_from_pipe->setStepDescription("Read from NullSource");
query_plan.addStep(std::move(read_from_pipe));
}
auto projection_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), action_dag);
projection_step->setStepDescription("Projection");
query_plan.addStep(std::move(projection_step));
}
}

View File

@ -0,0 +1,36 @@
#pragma once
#include <Interpreters/IInterpreter.h>
#include <Interpreters/SelectQueryOptions.h>
#include <Analyzer/QueryTreePassManager.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Interpreters/Context_fwd.h>
namespace DB
{
class InterpreterSelectQueryAnalyzer : public IInterpreter, public WithContext
{
public:
InterpreterSelectQueryAnalyzer(
const ASTPtr & query_ptr_,
const SelectQueryOptions & select_query_options_,
ContextPtr context);
Block getSampleBlock();
BlockIO execute() override;
bool supportsTransactions() const override { return true; }
private:
void initializeQueryPlanIfNeeded();
ASTPtr query_ptr;
QueryPlan query_plan;
SelectQueryOptions select_query_options;
QueryTreePassManager query_tree_pass_manager;
};
}

View File

@ -60,6 +60,11 @@ void ASTColumnsRegexpMatcher::setPattern(String pattern)
DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
}
const std::shared_ptr<re2::RE2> & ASTColumnsRegexpMatcher::getMatcher() const
{
return column_matcher;
}
bool ASTColumnsRegexpMatcher::isColumnMatching(const String & column_name) const
{
return RE2::PartialMatch(column_name, *column_matcher);
@ -114,4 +119,120 @@ void ASTColumnsListMatcher::formatImpl(const FormatSettings & settings, FormatSt
}
}
ASTPtr ASTQualifiedColumnsRegexpMatcher::clone() const
{
auto clone = std::make_shared<ASTQualifiedColumnsRegexpMatcher>(*this);
clone->cloneChildren();
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)
{
original_pattern = std::move(pattern);
column_matcher = std::make_shared<RE2>(original_pattern, RE2::Quiet);
if (!column_matcher->ok())
throw DB::Exception(
"COLUMNS pattern " + original_pattern + " cannot be compiled: " + column_matcher->error(),
DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
}
void ASTQualifiedColumnsRegexpMatcher::setMatcher(std::shared_ptr<re2::RE2> matcher)
{
column_matcher = std::move(matcher);
}
const std::shared_ptr<re2::RE2> & ASTQualifiedColumnsRegexpMatcher::getMatcher() const
{
return column_matcher;
}
void ASTQualifiedColumnsRegexpMatcher::updateTreeHashImpl(SipHash & hash_state) const
{
hash_state.update(original_pattern.size());
hash_state.update(original_pattern);
IAST::updateTreeHashImpl(hash_state);
}
void ASTQualifiedColumnsRegexpMatcher::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
const auto & qualifier = children.at(0);
qualifier->formatImpl(settings, state, frame);
settings.ostr << (settings.hilite ? hilite_keyword : "");
settings.ostr << ".COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
settings.ostr << quoteString(original_pattern);
settings.ostr << ")";
/// Format column transformers
for (const auto & child : children)
{
settings.ostr << ' ';
child->formatImpl(settings, state, frame);
}
}
ASTPtr ASTQualifiedColumnsListMatcher::clone() const
{
auto clone = std::make_shared<ASTQualifiedColumnsListMatcher>(*this);
clone->column_list = column_list->clone();
clone->cloneChildren();
return clone;
}
void ASTQualifiedColumnsListMatcher::appendColumnName(WriteBuffer & ostr) const
{
const auto & qualifier = children.at(0);
qualifier->appendColumnName(ostr);
writeCString(".COLUMNS(", ostr);
for (auto it = column_list->children.begin(); it != column_list->children.end(); ++it)
{
if (it != column_list->children.begin())
writeCString(", ", ostr);
(*it)->appendColumnName(ostr);
}
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
{
const auto & qualifier = children.at(0);
qualifier->formatImpl(settings, state, frame);
settings.ostr << (settings.hilite ? hilite_keyword : "") << ".COLUMNS" << (settings.hilite ? hilite_none : "") << "(";
for (ASTs::const_iterator it = column_list->children.begin(); it != column_list->children.end(); ++it)
{
if (it != column_list->children.begin())
{
settings.ostr << ", ";
}
(*it)->formatImpl(settings, state, frame);
}
settings.ostr << ")";
/// Format column transformers
for (const auto & child : children)
{
settings.ostr << ' ';
child->formatImpl(settings, state, frame);
}
}
}

View File

@ -24,6 +24,7 @@ public:
void appendColumnName(WriteBuffer & ostr) const override;
void setPattern(String pattern);
const std::shared_ptr<re2::RE2> & getMatcher() const;
bool isColumnMatching(const String & column_name) const;
void updateTreeHashImpl(SipHash & hash_state) const override;
@ -49,5 +50,39 @@ protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
/// Same as ASTColumnsRegexpMatcher. Qualified identifier is first child.
class ASTQualifiedColumnsRegexpMatcher : public IAST
{
public:
String getID(char) const override { return "QualifiedColumnsRegexpMatcher"; }
ASTPtr clone() const override;
void appendColumnName(WriteBuffer & ostr) const override;
const std::shared_ptr<re2::RE2> & getMatcher() const;
void setPattern(String pattern);
void setMatcher(std::shared_ptr<re2::RE2> matcher);
void updateTreeHashImpl(SipHash & hash_state) const override;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
private:
std::shared_ptr<re2::RE2> column_matcher;
String original_pattern;
};
/// Same as ASTColumnsListMatcher. Qualified identifier is first child.
class ASTQualifiedColumnsListMatcher : public IAST
{
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 column_list;
protected:
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
};
}

View File

@ -270,6 +270,11 @@ void ASTColumnsExceptTransformer::setPattern(String pattern)
DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
}
const std::shared_ptr<re2::RE2> & ASTColumnsExceptTransformer::getMatcher() const
{
return column_matcher;
}
bool ASTColumnsExceptTransformer::isColumnMatching(const String & column_name) const
{
return RE2::PartialMatch(column_name, *column_matcher);

View File

@ -60,6 +60,7 @@ public:
}
void transform(ASTs & nodes) const override;
void setPattern(String pattern);
const std::shared_ptr<re2::RE2> & getMatcher() const;
bool isColumnMatching(const String & column_name) const;
void appendColumnName(WriteBuffer & ostr) const override;
void updateTreeHashImpl(SipHash & hash_state) const override;

View File

@ -15,6 +15,7 @@ public:
{
ParsedAST, /// 'EXPLAIN AST SELECT ...'
AnalyzedSyntax, /// 'EXPLAIN SYNTAX SELECT ...'
QueryTree, /// 'EXPLAIN QUERYTREE SELECT ...'
QueryPlan, /// 'EXPLAIN SELECT ...'
QueryPipeline, /// 'EXPLAIN PIPELINE ...'
QueryEstimates, /// 'EXPLAIN ESTIMATE ...'
@ -109,6 +110,7 @@ private:
{
case ParsedAST: return "EXPLAIN AST";
case AnalyzedSyntax: return "EXPLAIN SYNTAX";
case QueryTree: return "EXPLAIN QUERYTREE";
case QueryPlan: return "EXPLAIN";
case QueryPipeline: return "EXPLAIN PIPELINE";
case QueryEstimates: return "EXPLAIN ESTIMATE";

View File

@ -24,6 +24,8 @@ public:
bool compute_after_window_functions = false;
bool is_lambda_function = false;
// We have to make these fields ASTPtr because this is what the visitors
// expect. Some of them take const ASTPtr & (makes no sense), and some
// take ASTPtr & and modify it. I don't understand how the latter is

View File

@ -49,9 +49,10 @@ public:
void restoreTable(); // TODO(ilezhankin): get rid of this
std::shared_ptr<ASTTableIdentifier> createTable() const; // returns |nullptr| if identifier is not table.
protected:
String full_name;
std::vector<String> name_parts;
protected:
std::shared_ptr<IdentifierSemanticImpl> semantic; /// pimpl
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;

View File

@ -1192,54 +1192,6 @@ bool ParserAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return true;
}
bool ParserColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword columns("COLUMNS");
ParserList columns_p(std::make_unique<ParserCompoundIdentifier>(false, true), std::make_unique<ParserToken>(TokenType::Comma), false);
ParserStringLiteral regex;
if (!columns.ignore(pos, expected))
return false;
if (pos->type != TokenType::OpeningRoundBracket)
return false;
++pos;
ASTPtr column_list;
ASTPtr regex_node;
if (!columns_p.parse(pos, column_list, expected) && !regex.parse(pos, regex_node, expected))
return false;
if (pos->type != TokenType::ClosingRoundBracket)
return false;
++pos;
ASTPtr res;
if (column_list)
{
auto list_matcher = std::make_shared<ASTColumnsListMatcher>();
list_matcher->column_list = column_list;
res = 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))
{
res->children.push_back(transformer);
}
node = std::move(res);
return true;
}
bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword apply("APPLY");
@ -1488,6 +1440,122 @@ bool ParserQualifiedAsterisk::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
return true;
}
/// Parse (columns_list) or ('REGEXP').
static bool parseColumnsMatcherBody(IParser::Pos & pos, ASTPtr & node, Expected & expected, ParserColumnsTransformers::ColumnTransformers allowed_transformers)
{
if (pos->type != TokenType::OpeningRoundBracket)
return false;
++pos;
ParserList columns_p(std::make_unique<ParserCompoundIdentifier>(false, true), std::make_unique<ParserToken>(TokenType::Comma), false);
ParserStringLiteral regex;
ASTPtr column_list;
ASTPtr regex_node;
if (!columns_p.parse(pos, column_list, expected) && !regex.parse(pos, regex_node, expected))
return false;
if (pos->type != TokenType::ClosingRoundBracket)
return false;
++pos;
ASTPtr res;
if (column_list)
{
auto list_matcher = std::make_shared<ASTColumnsListMatcher>();
list_matcher->column_list = column_list;
res = 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))
{
res->children.push_back(transformer);
}
node = std::move(res);
return true;
}
bool ParserColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword columns("COLUMNS");
if (!columns.ignore(pos, expected))
return false;
return parseColumnsMatcherBody(pos, node, expected, allowed_transformers);
}
bool ParserQualifiedColumnsMatcher::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserCompoundIdentifier(true, true).parse(pos, node, expected))
return false;
auto identifier_node = node;
const auto & identifier_node_typed = identifier_node->as<ASTTableIdentifier &>();
/// ParserCompoundIdentifier parse identifier.COLUMNS
if (identifier_node_typed.name_parts.size() == 1 || identifier_node_typed.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);
}
if (!parseColumnsMatcherBody(pos, node, expected, allowed_transformers))
return false;
if (auto * columns_list_matcher = node->as<ASTColumnsListMatcher>())
{
auto result = std::make_shared<ASTQualifiedColumnsListMatcher>();
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));
for (auto && child : columns_list_matcher->children)
result->children.push_back(std::move(child));
node = result;
}
else if (auto * column_regexp_matcher = node->as<ASTColumnsRegexpMatcher>())
{
auto result = std::make_shared<ASTQualifiedColumnsRegexpMatcher>();
result->setMatcher(column_regexp_matcher->getMatcher());
result->children.reserve(column_regexp_matcher->children.size() + 1);
result->children.push_back(std::move(identifier_node));
for (auto && child : column_regexp_matcher->children)
result->children.push_back(std::move(child));
node = result;
}
else
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Qualified COLUMNS matcher expected to be list or regexp");
}
return true;
}
bool ParserSubstitution::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{

View File

@ -104,7 +104,7 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** COLUMNS('<regular expression>')
/** COLUMNS(columns_names) or COLUMNS('<regular expression>')
*/
class ParserColumnsMatcher : public IParserBase
{
@ -121,6 +121,23 @@ protected:
ColumnTransformers allowed_transformers;
};
/** Qualified columns matcher identifier.COLUMNS(columns_names) or identifier.COLUMNS('<regular expression>')
*/
class ParserQualifiedColumnsMatcher : public IParserBase
{
public:
using ColumnTransformers = ParserColumnsTransformers::ColumnTransformers;
explicit ParserQualifiedColumnsMatcher(ColumnTransformers allowed_transformers_ = ParserColumnsTransformers::AllTransformers)
: allowed_transformers(allowed_transformers_)
{}
protected:
const char * getName() const override { return "qualified COLUMNS matcher"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
ColumnTransformers allowed_transformers;
};
// Allows to make queries like SELECT SUM(<expr>) FILTER(WHERE <cond>) FROM ...
class ParserFilterClause : public IParserBase
{

View File

@ -19,6 +19,7 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
ParserKeyword s_ast("AST");
ParserKeyword s_explain("EXPLAIN");
ParserKeyword s_syntax("SYNTAX");
ParserKeyword s_query_tree("QUERYTREE");
ParserKeyword s_pipeline("PIPELINE");
ParserKeyword s_plan("PLAN");
ParserKeyword s_estimates("ESTIMATE");
@ -33,6 +34,8 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
kind = ASTExplainQuery::ExplainKind::ParsedAST;
else if (s_syntax.ignore(pos, expected))
kind = ASTExplainQuery::ExplainKind::AnalyzedSyntax;
else if (s_query_tree.ignore(pos, expected))
kind = ASTExplainQuery::ExplainKind::QueryTree;
else if (s_pipeline.ignore(pos, expected))
kind = ASTExplainQuery::ExplainKind::QueryPipeline;
else if (s_plan.ignore(pos, expected))
@ -84,6 +87,15 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
explain_query->setTableFunction(table_function);
explain_query->setTableOverride(table_override);
}
if (kind == ASTExplainQuery::ExplainKind::QueryTree)
{
if (select_p.parse(pos, query, expected))
explain_query->setExplainedQuery(std::move(query));
else
{
return false;
}
}
else if (kind == ASTExplainQuery::ExplainKind::CurrentTransaction)
{
/// Nothing to parse
@ -103,7 +115,9 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
explain_query->setExplainedQuery(std::move(query));
}
else
{
return false;
}
node = std::move(explain_query);
return true;

View File

@ -0,0 +1,24 @@
#include <Parsers/SelectUnionMode.h>
namespace DB
{
const char * toString(SelectUnionMode mode)
{
switch (mode)
{
case SelectUnionMode::ALL:
return "ALL";
case SelectUnionMode::DISTINCT:
return "DISTINCT";
case SelectUnionMode::EXCEPT:
return "EXCEPT";
case SelectUnionMode::INTERSECT:
return "INTERSECT";
case SelectUnionMode::Unspecified:
return "Unspecified";
}
}
}

View File

@ -18,6 +18,8 @@ enum class SelectUnionMode
INTERSECT_DISTINCT
};
const char * toString(SelectUnionMode mode);
using SelectUnionModes = std::vector<SelectUnionMode>;
using SelectUnionModesSet = std::unordered_set<SelectUnionMode>;