mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Added Analyzer
This commit is contained in:
parent
729692f862
commit
75885ce2e1
7
src/Analyzer/CMakeLists.txt
Normal file
7
src/Analyzer/CMakeLists.txt
Normal file
@ -0,0 +1,7 @@
|
||||
if (ENABLE_TESTS)
|
||||
add_subdirectory(tests)
|
||||
endif()
|
||||
|
||||
if (ENABLE_EXAMPLES)
|
||||
add_subdirectory(examples)
|
||||
endif()
|
70
src/Analyzer/ColumnNode.cpp
Normal file
70
src/Analyzer/ColumnNode.cpp
Normal 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
96
src/Analyzer/ColumnNode.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
302
src/Analyzer/ColumnTransformers.cpp
Normal file
302
src/Analyzer/ColumnTransformers.cpp
Normal 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;
|
||||
}
|
||||
|
||||
}
|
267
src/Analyzer/ColumnTransformers.h
Normal file
267
src/Analyzer/ColumnTransformers.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
56
src/Analyzer/ConstantNode.cpp
Normal file
56
src/Analyzer/ConstantNode.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
62
src/Analyzer/ConstantNode.h
Normal file
62
src/Analyzer/ConstantNode.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
123
src/Analyzer/FunctionNode.cpp
Normal file
123
src/Analyzer/FunctionNode.cpp
Normal 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
181
src/Analyzer/FunctionNode.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
158
src/Analyzer/IQueryTreeNode.cpp
Normal file
158
src/Analyzer/IQueryTreeNode.cpp
Normal 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();
|
||||
}
|
||||
|
||||
}
|
234
src/Analyzer/IQueryTreeNode.h
Normal file
234
src/Analyzer/IQueryTreeNode.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
38
src/Analyzer/IQueryTreePass.h
Normal file
38
src/Analyzer/IQueryTreePass.h
Normal 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
320
src/Analyzer/Identifier.h
Normal 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;
|
||||
}
|
||||
|
||||
}
|
38
src/Analyzer/IdentifierNode.cpp
Normal file
38
src/Analyzer/IdentifierNode.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
53
src/Analyzer/IdentifierNode.h
Normal file
53
src/Analyzer/IdentifierNode.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
67
src/Analyzer/InDepthQueryTreeVisitor.h
Normal file
67
src/Analyzer/InDepthQueryTreeVisitor.h
Normal 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);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
}
|
92
src/Analyzer/LambdaNode.cpp
Normal file
92
src/Analyzer/LambdaNode.cpp
Normal 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
116
src/Analyzer/LambdaNode.h
Normal 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
72
src/Analyzer/ListNode.cpp
Normal 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
48
src/Analyzer/ListNode.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
267
src/Analyzer/MatcherNode.cpp
Normal file
267
src/Analyzer/MatcherNode.cpp
Normal 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
171
src/Analyzer/MatcherNode.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
49
src/Analyzer/MultiIfToIfPass.cpp
Normal file
49
src/Analyzer/MultiIfToIfPass.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
25
src/Analyzer/MultiIfToIfPass.h
Normal file
25
src/Analyzer/MultiIfToIfPass.h
Normal 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;
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
2024
src/Analyzer/QueryAnalysisPass.cpp
Normal file
2024
src/Analyzer/QueryAnalysisPass.cpp
Normal file
File diff suppressed because it is too large
Load Diff
49
src/Analyzer/QueryAnalysisPass.h
Normal file
49
src/Analyzer/QueryAnalysisPass.h
Normal 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
100
src/Analyzer/QueryNode.cpp
Normal 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
118
src/Analyzer/QueryNode.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
550
src/Analyzer/QueryTreeBuilder.cpp
Normal file
550
src/Analyzer/QueryTreeBuilder.cpp
Normal 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();
|
||||
}
|
||||
|
||||
}
|
19
src/Analyzer/QueryTreeBuilder.h
Normal file
19
src/Analyzer/QueryTreeBuilder.h
Normal 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);
|
||||
|
||||
}
|
91
src/Analyzer/QueryTreePassManager.cpp
Normal file
91
src/Analyzer/QueryTreePassManager.cpp
Normal 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>());
|
||||
}
|
||||
|
||||
}
|
49
src/Analyzer/QueryTreePassManager.h
Normal file
49
src/Analyzer/QueryTreePassManager.h
Normal 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);
|
||||
|
||||
}
|
60
src/Analyzer/TableNode.cpp
Normal file
60
src/Analyzer/TableNode.cpp
Normal 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
90
src/Analyzer/TableNode.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
||||
|
3
src/Analyzer/examples/CMakeLists.txt
Normal file
3
src/Analyzer/examples/CMakeLists.txt
Normal 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)
|
11
src/Analyzer/examples/query_analyzer.cpp
Normal file
11
src/Analyzer/examples/query_analyzer.cpp
Normal file
@ -0,0 +1,11 @@
|
||||
#include <iostream>
|
||||
|
||||
using namespace DB;
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
{
|
||||
(void)(argc);
|
||||
(void)(argv);
|
||||
|
||||
return 0;
|
||||
}
|
0
src/Analyzer/tests/CMakeLists.txt
Normal file
0
src/Analyzer/tests/CMakeLists.txt
Normal file
227
src/Analyzer/tests/gtest_identifier.cpp
Normal file
227
src/Analyzer/tests/gtest_identifier.cpp
Normal 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());
|
||||
}
|
||||
}
|
82
src/Analyzer/tests/gtest_query_tree_node.cpp
Normal file
82
src/Analyzer/tests/gtest_query_tree_node.cpp
Normal 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());
|
||||
}
|
||||
}
|
@ -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)
|
||||
|
@ -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) \
|
||||
\
|
||||
\
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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());
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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()))
|
||||
|
@ -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>())
|
||||
|
609
src/Interpreters/InterpreterSelectQueryAnalyzer.cpp
Normal file
609
src/Interpreters/InterpreterSelectQueryAnalyzer.cpp
Normal 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));
|
||||
}
|
||||
|
||||
}
|
36
src/Interpreters/InterpreterSelectQueryAnalyzer.h
Normal file
36
src/Interpreters/InterpreterSelectQueryAnalyzer.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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";
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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;
|
||||
|
24
src/Parsers/SelectUnionMode.cpp
Normal file
24
src/Parsers/SelectUnionMode.cpp
Normal 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";
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -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>;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user