ClickHouse/src/Analyzer/QueryTreeBuilder.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

1036 lines
46 KiB
C++
Raw Normal View History

2022-07-14 11:20:16 +00:00
#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>
2022-08-23 09:50:02 +00:00
#include <Parsers/ASTSelectIntersectExceptQuery.h>
2022-07-14 11:20:16 +00:00
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTQueryParameter.h>
2022-07-14 11:20:16 +00:00
#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>
2022-08-26 13:06:14 +00:00
#include <Parsers/ASTOrderByElement.h>
2022-09-03 16:20:09 +00:00
#include <Parsers/ASTInterpolateElement.h>
#include <Parsers/ASTSampleRatio.h>
2022-09-12 14:14:40 +00:00
#include <Parsers/ASTWindowDefinition.h>
2022-10-12 13:46:50 +00:00
#include <Parsers/ASTSetQuery.h>
2022-07-14 11:20:16 +00:00
#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>
2022-09-15 11:25:11 +00:00
#include <Analyzer/SortNode.h>
#include <Analyzer/InterpolateNode.h>
2022-09-12 14:14:40 +00:00
#include <Analyzer/WindowNode.h>
2022-07-14 11:20:16 +00:00
#include <Analyzer/TableNode.h>
2022-07-20 15:25:24 +00:00
#include <Analyzer/TableFunctionNode.h>
2022-07-14 11:20:16 +00:00
#include <Analyzer/QueryNode.h>
2022-08-15 16:34:10 +00:00
#include <Analyzer/ArrayJoinNode.h>
#include <Analyzer/JoinNode.h>
2022-08-23 09:50:02 +00:00
#include <Analyzer/UnionNode.h>
2022-07-14 11:20:16 +00:00
#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 LOGICAL_ERROR;
2022-09-27 15:04:03 +00:00
extern const int EXPECTED_ALL_OR_ANY;
2022-10-04 20:45:47 +00:00
extern const int NOT_IMPLEMENTED;
extern const int BAD_ARGUMENTS;
extern const int UNKNOWN_QUERY_PARAMETER;
2022-07-14 11:20:16 +00:00
}
2022-08-23 09:50:02 +00:00
namespace
{
2022-09-27 15:04:03 +00:00
class QueryTreeBuilder
2022-07-14 11:20:16 +00:00
{
public:
2022-09-27 15:04:03 +00:00
explicit QueryTreeBuilder(ASTPtr query_, ContextPtr context_);
2022-07-14 11:20:16 +00:00
QueryTreeNodePtr getQueryTreeNode()
{
return query_tree_node;
}
private:
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr buildSelectOrUnionExpression(const ASTPtr & select_or_union_query,
bool is_subquery,
const std::string & cte_name,
const ContextPtr & context) const;
2022-08-23 09:50:02 +00:00
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr buildSelectWithUnionExpression(const ASTPtr & select_with_union_query,
bool is_subquery,
const std::string & cte_name,
const ContextPtr & context) const;
2022-08-23 09:50:02 +00:00
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr buildSelectIntersectExceptQuery(const ASTPtr & select_intersect_except_query,
bool is_subquery,
const std::string & cte_name,
const ContextPtr & context) const;
2022-07-14 11:20:16 +00:00
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr buildSelectExpression(const ASTPtr & select_query,
bool is_subquery,
const std::string & cte_name,
const ContextPtr & context) const;
2022-07-14 11:20:16 +00:00
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr buildSortList(const ASTPtr & order_by_expression_list, const ContextPtr & context) const;
2022-08-26 13:06:14 +00:00
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr buildInterpolateList(const ASTPtr & interpolate_expression_list, const ContextPtr & context) const;
2022-09-03 16:20:09 +00:00
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr buildWindowList(const ASTPtr & window_definition_list, const ContextPtr & context) const;
2022-09-12 14:14:40 +00:00
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr buildExpressionList(const ASTPtr & expression_list, const ContextPtr & context) const;
2022-07-14 11:20:16 +00:00
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr buildExpression(const ASTPtr & expression, const ContextPtr & context) const;
2022-07-14 11:20:16 +00:00
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr buildWindow(const ASTPtr & window_definition, const ContextPtr & context) const;
2022-09-12 14:14:40 +00:00
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr buildJoinTree(const ASTPtr & tables_in_select_query, const ContextPtr & context) const;
2022-07-14 11:20:16 +00:00
ColumnTransformersNodes buildColumnTransformers(const ASTPtr & matcher_expression, const ContextPtr & context) const;
2022-07-14 11:20:16 +00:00
ASTPtr query;
QueryTreeNodePtr query_tree_node;
};
2022-09-27 15:04:03 +00:00
QueryTreeBuilder::QueryTreeBuilder(ASTPtr query_, ContextPtr context_)
2022-09-07 17:09:37 +00:00
: query(query_->clone())
2022-07-14 11:20:16 +00:00
{
2022-08-23 09:50:02 +00:00
if (query->as<ASTSelectWithUnionQuery>() ||
query->as<ASTSelectIntersectExceptQuery>() ||
query->as<ASTSelectQuery>())
2022-12-06 09:44:38 +00:00
query_tree_node = buildSelectOrUnionExpression(query, false /*is_subquery*/, {} /*cte_name*/, context_);
2022-07-14 11:20:16 +00:00
else if (query->as<ASTExpressionList>())
2022-12-06 09:44:38 +00:00
query_tree_node = buildExpressionList(query, context_);
2022-07-14 11:20:16 +00:00
else
2022-12-06 09:44:38 +00:00
query_tree_node = buildExpression(query, context_);
2022-07-14 11:20:16 +00:00
}
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr QueryTreeBuilder::buildSelectOrUnionExpression(const ASTPtr & select_or_union_query,
bool is_subquery,
const std::string & cte_name,
const ContextPtr & context) const
2022-08-23 09:50:02 +00:00
{
QueryTreeNodePtr query_node;
if (select_or_union_query->as<ASTSelectWithUnionQuery>())
2022-12-06 09:44:38 +00:00
query_node = buildSelectWithUnionExpression(select_or_union_query, is_subquery /*is_subquery*/, cte_name /*cte_name*/, context);
2022-08-23 09:50:02 +00:00
else if (select_or_union_query->as<ASTSelectIntersectExceptQuery>())
2022-12-06 09:44:38 +00:00
query_node = buildSelectIntersectExceptQuery(select_or_union_query, is_subquery /*is_subquery*/, cte_name /*cte_name*/, context);
2022-08-23 09:50:02 +00:00
else if (select_or_union_query->as<ASTSelectQuery>())
2022-12-06 09:44:38 +00:00
query_node = buildSelectExpression(select_or_union_query, is_subquery /*is_subquery*/, cte_name /*cte_name*/, context);
2022-08-23 09:50:02 +00:00
else
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "SELECT or UNION query {} is not supported",
select_or_union_query->formatForErrorMessage());
2022-08-23 09:50:02 +00:00
return query_node;
}
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr QueryTreeBuilder::buildSelectWithUnionExpression(const ASTPtr & select_with_union_query,
bool is_subquery,
const std::string & cte_name,
const ContextPtr & context) const
2022-07-14 11:20:16 +00:00
{
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)
2022-12-06 09:44:38 +00:00
return buildSelectOrUnionExpression(select_lists.children[0], is_subquery, cte_name, context);
2022-08-23 09:50:02 +00:00
2022-12-06 09:44:38 +00:00
auto union_node = std::make_shared<UnionNode>(Context::createCopy(context), select_with_union_query_typed.union_mode);
2022-08-23 09:50:02 +00:00
union_node->setIsSubquery(is_subquery);
2022-10-19 10:25:27 +00:00
union_node->setIsCTE(!cte_name.empty());
2022-08-23 09:50:02 +00:00
union_node->setCTEName(cte_name);
union_node->setOriginalAST(select_with_union_query);
size_t select_lists_children_size = select_lists.children.size();
for (size_t i = 0; i < select_lists_children_size; ++i)
2022-07-14 11:20:16 +00:00
{
2022-08-23 09:50:02 +00:00
auto & select_list_node = select_lists.children[i];
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr query_node = buildSelectOrUnionExpression(select_list_node, false /*is_subquery*/, {} /*cte_name*/, context);
2022-08-23 09:50:02 +00:00
union_node->getQueries().getNodes().push_back(std::move(query_node));
2022-07-14 11:20:16 +00:00
}
2022-08-23 09:50:02 +00:00
return union_node;
}
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr QueryTreeBuilder::buildSelectIntersectExceptQuery(const ASTPtr & select_intersect_except_query,
bool is_subquery,
const std::string & cte_name,
const ContextPtr & context) const
2022-08-23 09:50:02 +00:00
{
auto & select_intersect_except_query_typed = select_intersect_except_query->as<ASTSelectIntersectExceptQuery &>();
auto select_lists = select_intersect_except_query_typed.getListOfSelects();
if (select_lists.size() == 1)
2022-12-06 09:44:38 +00:00
return buildSelectExpression(select_lists[0], is_subquery, cte_name, context);
2022-08-23 09:50:02 +00:00
SelectUnionMode union_mode;
2022-09-06 16:46:30 +00:00
if (select_intersect_except_query_typed.final_operator == ASTSelectIntersectExceptQuery::Operator::INTERSECT_ALL)
union_mode = SelectUnionMode::INTERSECT_ALL;
2022-09-06 16:46:30 +00:00
else if (select_intersect_except_query_typed.final_operator == ASTSelectIntersectExceptQuery::Operator::INTERSECT_DISTINCT)
union_mode = SelectUnionMode::INTERSECT_DISTINCT;
2022-09-06 16:46:30 +00:00
else if (select_intersect_except_query_typed.final_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT_ALL)
union_mode = SelectUnionMode::EXCEPT_ALL;
2022-09-06 16:46:30 +00:00
else if (select_intersect_except_query_typed.final_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT_DISTINCT)
union_mode = SelectUnionMode::EXCEPT_DISTINCT;
2022-08-23 09:50:02 +00:00
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "UNION type is not initialized");
2022-12-06 09:44:38 +00:00
auto union_node = std::make_shared<UnionNode>(Context::createCopy(context), union_mode);
union_node->setIsSubquery(is_subquery);
union_node->setIsCTE(!cte_name.empty());
union_node->setCTEName(cte_name);
2022-08-23 09:50:02 +00:00
union_node->setOriginalAST(select_intersect_except_query);
2022-07-14 11:20:16 +00:00
2022-08-23 09:50:02 +00:00
size_t select_lists_size = select_lists.size();
2022-07-14 11:20:16 +00:00
2022-08-23 09:50:02 +00:00
for (size_t i = 0; i < select_lists_size; ++i)
{
auto & select_list_node = select_lists[i];
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr query_node = buildSelectOrUnionExpression(select_list_node, false /*is_subquery*/, {} /*cte_name*/, context);
2022-08-23 09:50:02 +00:00
union_node->getQueries().getNodes().push_back(std::move(query_node));
2022-07-14 11:20:16 +00:00
}
2022-08-23 09:50:02 +00:00
return union_node;
2022-07-14 11:20:16 +00:00
}
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_query,
bool is_subquery,
const std::string & cte_name,
const ContextPtr & context) const
2022-07-14 11:20:16 +00:00
{
const auto & select_query_typed = select_query->as<ASTSelectQuery &>();
2022-12-06 09:44:38 +00:00
auto updated_context = Context::createCopy(context);
auto select_settings = select_query_typed.settings();
SettingsChanges settings_changes;
/// We are going to remove settings LIMIT and OFFSET and
/// further replace them with corresponding expression nodes
UInt64 limit = 0;
UInt64 offset = 0;
2023-02-22 17:22:13 +00:00
/// Remove global settings limit and offset
if (const auto & settings_ref = updated_context->getSettingsRef(); settings_ref.limit || settings_ref.offset)
{
Settings settings = updated_context->getSettings();
limit = settings.limit;
offset = settings.offset;
settings.limit = 0;
settings.offset = 0;
updated_context->setSettings(settings);
}
2022-12-06 09:44:38 +00:00
if (select_settings)
{
auto & set_query = select_settings->as<ASTSetQuery &>();
2023-02-22 17:22:13 +00:00
/// Remove expression settings limit and offset
if (auto * limit_field = set_query.changes.tryGet("limit"))
{
limit = limit_field->safeGet<UInt64>();
set_query.changes.removeSetting("limit");
}
if (auto * offset_field = set_query.changes.tryGet("offset"))
{
offset = offset_field->safeGet<UInt64>();
set_query.changes.removeSetting("offset");
}
if (!set_query.changes.empty())
{
updated_context->applySettingsChanges(set_query.changes);
settings_changes = set_query.changes;
}
2022-12-06 09:44:38 +00:00
}
auto current_query_tree = std::make_shared<QueryNode>(std::move(updated_context), std::move(settings_changes));
2022-07-18 17:20:28 +00:00
current_query_tree->setIsSubquery(is_subquery);
current_query_tree->setIsCTE(!cte_name.empty());
current_query_tree->setCTEName(cte_name);
2022-08-25 18:35:16 +00:00
current_query_tree->setIsDistinct(select_query_typed.distinct);
2022-08-26 16:23:26 +00:00
current_query_tree->setIsLimitWithTies(select_query_typed.limit_with_ties);
current_query_tree->setIsGroupByWithTotals(select_query_typed.group_by_with_totals);
current_query_tree->setIsGroupByWithCube(select_query_typed.group_by_with_cube);
current_query_tree->setIsGroupByWithRollup(select_query_typed.group_by_with_rollup);
current_query_tree->setIsGroupByWithGroupingSets(select_query_typed.group_by_with_grouping_sets);
2022-11-01 08:18:56 +00:00
current_query_tree->setIsGroupByAll(select_query_typed.group_by_all);
2023-12-19 07:18:27 +00:00
current_query_tree->setIsOrderByAll(select_query_typed.order_by_all);
2022-07-14 11:20:16 +00:00
current_query_tree->setOriginalAST(select_query);
2022-12-06 09:44:38 +00:00
auto current_context = current_query_tree->getContext();
2022-10-12 13:46:50 +00:00
2022-12-06 09:44:38 +00:00
current_query_tree->getJoinTree() = buildJoinTree(select_query_typed.tables(), current_context);
2022-07-14 11:20:16 +00:00
auto select_with_list = select_query_typed.with();
if (select_with_list)
2022-12-06 09:44:38 +00:00
current_query_tree->getWithNode() = buildExpressionList(select_with_list, current_context);
2022-07-14 11:20:16 +00:00
auto select_expression_list = select_query_typed.select();
if (select_expression_list)
2022-12-06 09:44:38 +00:00
current_query_tree->getProjectionNode() = buildExpressionList(select_expression_list, current_context);
2022-07-14 11:20:16 +00:00
auto prewhere_expression = select_query_typed.prewhere();
if (prewhere_expression)
2022-12-06 09:44:38 +00:00
current_query_tree->getPrewhere() = buildExpression(prewhere_expression, current_context);
2022-07-14 11:20:16 +00:00
auto where_expression = select_query_typed.where();
if (where_expression)
2022-12-06 09:44:38 +00:00
current_query_tree->getWhere() = buildExpression(where_expression, current_context);
2022-07-14 11:20:16 +00:00
2022-08-25 14:19:35 +00:00
auto group_by_list = select_query_typed.groupBy();
if (group_by_list)
2022-09-02 15:41:19 +00:00
{
auto & group_by_children = group_by_list->children;
if (current_query_tree->isGroupByWithGroupingSets())
{
auto grouping_sets_list_node = std::make_shared<ListNode>();
for (auto & grouping_sets_keys : group_by_children)
{
2022-12-06 09:44:38 +00:00
auto grouping_sets_keys_list_node = buildExpressionList(grouping_sets_keys, current_context);
2022-09-02 15:41:19 +00:00
current_query_tree->getGroupBy().getNodes().emplace_back(std::move(grouping_sets_keys_list_node));
}
}
else
{
2022-12-06 09:44:38 +00:00
current_query_tree->getGroupByNode() = buildExpressionList(group_by_list, current_context);
2022-09-02 15:41:19 +00:00
}
}
2022-08-25 14:19:35 +00:00
2022-09-02 10:30:12 +00:00
auto having_expression = select_query_typed.having();
if (having_expression)
2022-12-06 09:44:38 +00:00
current_query_tree->getHaving() = buildExpression(having_expression, current_context);
2022-09-02 10:30:12 +00:00
2022-09-12 14:14:40 +00:00
auto window_list = select_query_typed.window();
if (window_list)
2022-12-06 09:44:38 +00:00
current_query_tree->getWindowNode() = buildWindowList(window_list, current_context);
2022-09-12 14:14:40 +00:00
2022-08-26 13:06:14 +00:00
auto select_order_by_list = select_query_typed.orderBy();
if (select_order_by_list)
2022-12-06 09:44:38 +00:00
current_query_tree->getOrderByNode() = buildSortList(select_order_by_list, current_context);
2022-08-26 13:06:14 +00:00
2022-09-03 16:20:09 +00:00
auto interpolate_list = select_query_typed.interpolate();
if (interpolate_list)
2022-12-06 09:44:38 +00:00
current_query_tree->getInterpolate() = buildInterpolateList(interpolate_list, current_context);
2022-09-03 16:20:09 +00:00
2022-09-03 17:21:17 +00:00
auto select_limit_by_limit = select_query_typed.limitByLength();
if (select_limit_by_limit)
2022-12-06 09:44:38 +00:00
current_query_tree->getLimitByLimit() = buildExpression(select_limit_by_limit, current_context);
2022-09-03 17:21:17 +00:00
2023-01-31 10:34:17 +00:00
auto select_limit_by_offset = select_query_typed.limitByOffset();
2022-09-03 17:21:17 +00:00
if (select_limit_by_offset)
2022-12-06 09:44:38 +00:00
current_query_tree->getLimitByOffset() = buildExpression(select_limit_by_offset, current_context);
2022-09-03 17:21:17 +00:00
auto select_limit_by = select_query_typed.limitBy();
if (select_limit_by)
2022-12-06 09:44:38 +00:00
current_query_tree->getLimitByNode() = buildExpressionList(select_limit_by, current_context);
2022-09-03 17:21:17 +00:00
/// Combine limit expression with limit and offset settings into final limit expression
/// The sequence of application is the following - offset expression, limit expression, offset setting, limit setting.
2023-03-08 14:25:24 +00:00
/// Since offset setting is applied after limit expression, but we want to transfer settings into expression
/// we must decrease limit expression by offset setting and then add offset setting to offset expression.
/// select_limit - limit expression
/// limit - limit setting
/// offset - offset setting
///
/// if select_limit
/// -- if offset >= select_limit (expr 0)
/// then (0) (0 rows)
/// -- else if limit > 0 (expr 1)
/// then min(select_limit - offset, limit) (expr 2)
/// -- else
/// then (select_limit - offset) (expr 3)
/// else if limit > 0
/// then limit
///
/// offset = offset + of_expr
2022-08-26 16:23:26 +00:00
auto select_limit = select_query_typed.limitLength();
if (select_limit)
{
/// Shortcut
if (offset == 0 && limit == 0)
{
current_query_tree->getLimit() = buildExpression(select_limit, current_context);
}
else
{
/// expr 3
auto expr_3 = std::make_shared<FunctionNode>("minus");
expr_3->getArguments().getNodes().push_back(buildExpression(select_limit, current_context));
expr_3->getArguments().getNodes().push_back(std::make_shared<ConstantNode>(offset));
/// expr 2
auto expr_2 = std::make_shared<FunctionNode>("least");
expr_2->getArguments().getNodes().push_back(expr_3->clone());
expr_2->getArguments().getNodes().push_back(std::make_shared<ConstantNode>(limit));
/// expr 0
auto expr_0 = std::make_shared<FunctionNode>("greaterOrEquals");
expr_0->getArguments().getNodes().push_back(std::make_shared<ConstantNode>(offset));
expr_0->getArguments().getNodes().push_back(buildExpression(select_limit, current_context));
/// expr 1
auto expr_1 = std::make_shared<ConstantNode>(limit > 0);
auto function_node = std::make_shared<FunctionNode>("multiIf");
function_node->getArguments().getNodes().push_back(expr_0);
function_node->getArguments().getNodes().push_back(std::make_shared<ConstantNode>(0));
function_node->getArguments().getNodes().push_back(expr_1);
function_node->getArguments().getNodes().push_back(expr_2);
function_node->getArguments().getNodes().push_back(expr_3);
current_query_tree->getLimit() = std::move(function_node);
}
}
else if (limit > 0)
current_query_tree->getLimit() = std::make_shared<ConstantNode>(limit);
2022-08-26 16:23:26 +00:00
/// Combine offset expression with offset setting into final offset expression
2022-08-26 16:23:26 +00:00
auto select_offset = select_query_typed.limitOffset();
if (select_offset && offset)
{
auto function_node = std::make_shared<FunctionNode>("plus");
function_node->getArguments().getNodes().push_back(buildExpression(select_offset, current_context));
function_node->getArguments().getNodes().push_back(std::make_shared<ConstantNode>(offset));
2023-02-22 17:22:13 +00:00
current_query_tree->getOffset() = std::move(function_node);
}
else if (offset)
current_query_tree->getOffset() = std::make_shared<ConstantNode>(offset);
else if (select_offset)
2022-12-06 09:44:38 +00:00
current_query_tree->getOffset() = buildExpression(select_offset, current_context);
2022-08-26 16:23:26 +00:00
2022-07-14 11:20:16 +00:00
return current_query_tree;
}
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr QueryTreeBuilder::buildSortList(const ASTPtr & order_by_expression_list, const ContextPtr & context) const
2022-08-26 13:06:14 +00:00
{
auto list_node = std::make_shared<ListNode>();
auto & expression_list_typed = order_by_expression_list->as<ASTExpressionList &>();
list_node->getNodes().reserve(expression_list_typed.children.size());
for (auto & expression : expression_list_typed.children)
{
const auto & order_by_element = expression->as<const ASTOrderByElement &>();
auto sort_direction = order_by_element.direction == 1 ? SortDirection::ASCENDING : SortDirection::DESCENDING;
std::optional<SortDirection> nulls_sort_direction;
if (order_by_element.nulls_direction_was_explicitly_specified)
nulls_sort_direction = order_by_element.nulls_direction == 1 ? SortDirection::ASCENDING : SortDirection::DESCENDING;
std::shared_ptr<Collator> collator;
if (order_by_element.collation)
2022-10-19 10:25:27 +00:00
collator = std::make_shared<Collator>(order_by_element.collation->as<ASTLiteral &>().value.get<String &>());
2022-08-26 13:06:14 +00:00
const auto & sort_expression_ast = order_by_element.children.at(0);
2022-12-06 09:44:38 +00:00
auto sort_expression = buildExpression(sort_expression_ast, context);
2022-09-15 11:25:11 +00:00
auto sort_node = std::make_shared<SortNode>(std::move(sort_expression),
2022-09-03 17:34:52 +00:00
sort_direction,
nulls_sort_direction,
std::move(collator),
order_by_element.with_fill);
2022-08-26 13:06:14 +00:00
if (order_by_element.fill_from)
2022-12-06 09:44:38 +00:00
sort_node->getFillFrom() = buildExpression(order_by_element.fill_from, context);
2022-08-26 13:06:14 +00:00
if (order_by_element.fill_to)
2022-12-06 09:44:38 +00:00
sort_node->getFillTo() = buildExpression(order_by_element.fill_to, context);
2022-08-26 13:06:14 +00:00
if (order_by_element.fill_step)
2022-12-06 09:44:38 +00:00
sort_node->getFillStep() = buildExpression(order_by_element.fill_step, context);
2022-08-26 13:06:14 +00:00
2022-09-15 11:25:11 +00:00
list_node->getNodes().push_back(std::move(sort_node));
2022-08-26 13:06:14 +00:00
}
return list_node;
}
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr QueryTreeBuilder::buildInterpolateList(const ASTPtr & interpolate_expression_list, const ContextPtr & context) const
2022-09-03 16:20:09 +00:00
{
auto list_node = std::make_shared<ListNode>();
auto & expression_list_typed = interpolate_expression_list->as<ASTExpressionList &>();
list_node->getNodes().reserve(expression_list_typed.children.size());
for (auto & expression : expression_list_typed.children)
{
const auto & interpolate_element = expression->as<const ASTInterpolateElement &>();
auto expression_to_interpolate = std::make_shared<IdentifierNode>(Identifier(interpolate_element.column));
2022-12-06 09:44:38 +00:00
auto interpolate_expression = buildExpression(interpolate_element.expr, context);
auto interpolate_node = std::make_shared<InterpolateNode>(std::move(expression_to_interpolate), std::move(interpolate_expression));
2022-09-03 16:20:09 +00:00
list_node->getNodes().push_back(std::move(interpolate_node));
2022-09-03 16:20:09 +00:00
}
return list_node;
}
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr QueryTreeBuilder::buildWindowList(const ASTPtr & window_definition_list, const ContextPtr & context) const
2022-09-12 14:14:40 +00:00
{
auto list_node = std::make_shared<ListNode>();
auto & expression_list_typed = window_definition_list->as<ASTExpressionList &>();
list_node->getNodes().reserve(expression_list_typed.children.size());
for (auto & window_list_element : expression_list_typed.children)
{
const auto & window_list_element_typed = window_list_element->as<const ASTWindowListElement &>();
2022-12-06 09:44:38 +00:00
auto window_node = buildWindow(window_list_element_typed.definition, context);
2022-09-12 14:14:40 +00:00
window_node->setAlias(window_list_element_typed.name);
list_node->getNodes().push_back(std::move(window_node));
}
return list_node;
}
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr QueryTreeBuilder::buildExpressionList(const ASTPtr & expression_list, const ContextPtr & context) const
2022-07-14 11:20:16 +00:00
{
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)
{
2022-12-06 09:44:38 +00:00
auto expression_node = buildExpression(expression, context);
2022-07-14 11:20:16 +00:00
list_node->getNodes().push_back(std::move(expression_node));
}
return list_node;
}
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, const ContextPtr & context) const
2022-07-14 11:20:16 +00:00
{
QueryTreeNodePtr result;
if (const auto * ast_identifier = expression->as<ASTIdentifier>())
{
auto identifier = Identifier(ast_identifier->name_parts);
result = std::make_shared<IdentifierNode>(std::move(identifier));
}
2023-02-09 17:30:57 +00:00
else if (const auto * table_identifier = expression->as<ASTTableIdentifier>())
{
auto identifier = Identifier(table_identifier->name_parts);
result = std::make_shared<IdentifierNode>(std::move(identifier));
2022-07-14 11:20:16 +00:00
}
else if (const auto * asterisk = expression->as<ASTAsterisk>())
{
auto column_transformers = buildColumnTransformers(asterisk->transformers, context);
2022-10-19 10:25:27 +00:00
result = std::make_shared<MatcherNode>(std::move(column_transformers));
2022-07-14 11:20:16 +00:00
}
else if (const auto * qualified_asterisk = expression->as<ASTQualifiedAsterisk>())
{
auto & qualified_identifier = qualified_asterisk->qualifier->as<ASTIdentifier &>();
auto column_transformers = buildColumnTransformers(qualified_asterisk->transformers, context);
2022-10-19 10:25:27 +00:00
result = std::make_shared<MatcherNode>(Identifier(qualified_identifier.name_parts), std::move(column_transformers));
2022-07-14 11:20:16 +00:00
}
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)
{
2022-10-19 10:25:27 +00:00
const auto & lambda_arguments_list = lambda_arguments_tuple.arguments->as<ASTExpressionList &>().children;
2022-07-14 11:20:16 +00:00
for (const auto & lambda_argument : lambda_arguments_list)
{
const auto * lambda_argument_identifier = lambda_argument->as<ASTIdentifier>();
if (!lambda_argument_identifier)
2022-10-05 10:09:23 +00:00
throw Exception(ErrorCodes::BAD_ARGUMENTS,
2022-07-14 11:20:16 +00:00
"Lambda {} argument is not identifier",
function->formatForErrorMessage());
if (lambda_argument_identifier->name_parts.size() > 1)
2022-10-12 11:26:02 +00:00
throw Exception(ErrorCodes::BAD_ARGUMENTS,
2022-10-19 10:25:27 +00:00
"Lambda {} argument identifier must contain single part. Actual {}",
2022-07-14 11:20:16 +00:00
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)
2022-10-12 11:26:02 +00:00
throw Exception(ErrorCodes::BAD_ARGUMENTS,
2022-07-14 11:20:16 +00:00
"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);
2022-12-06 09:44:38 +00:00
auto lambda_expression_node = buildExpression(lambda_expression, context);
2022-07-14 11:20:16 +00:00
result = std::make_shared<LambdaNode>(std::move(lambda_arguments), std::move(lambda_expression_node));
}
else
{
auto function_node = std::make_shared<FunctionNode>(function->name);
function_node->setNullsAction(function->nulls_action);
2022-07-14 11:20:16 +00:00
if (function->parameters)
{
const auto & function_parameters_list = function->parameters->as<ASTExpressionList>()->children;
for (const auto & argument : function_parameters_list)
2022-12-06 09:44:38 +00:00
function_node->getParameters().getNodes().push_back(buildExpression(argument, context));
2022-07-14 11:20:16 +00:00
}
if (function->arguments)
{
const auto & function_arguments_list = function->arguments->as<ASTExpressionList>()->children;
for (const auto & argument : function_arguments_list)
2022-12-06 09:44:38 +00:00
function_node->getArguments().getNodes().push_back(buildExpression(argument, context));
2022-07-14 11:20:16 +00:00
}
2022-09-12 14:14:40 +00:00
if (function->is_window_function)
{
if (function->window_definition)
2022-12-06 09:44:38 +00:00
function_node->getWindowNode() = buildWindow(function->window_definition, context);
2022-09-12 14:14:40 +00:00
else
function_node->getWindowNode() = std::make_shared<IdentifierNode>(Identifier(function->window_name));
}
2022-10-19 10:25:27 +00:00
result = std::move(function_node);
2022-07-14 11:20:16 +00:00
}
}
else if (const auto * subquery = expression->as<ASTSubquery>())
{
auto subquery_query = subquery->children[0];
2022-12-06 09:44:38 +00:00
auto query_node = buildSelectWithUnionExpression(subquery_query, true /*is_subquery*/, {} /*cte_name*/, context);
2022-07-18 17:20:28 +00:00
2022-10-19 10:25:27 +00:00
result = std::move(query_node);
2022-07-18 17:20:28 +00:00
}
else if (const auto * select_with_union_query = expression->as<ASTSelectWithUnionQuery>())
{
auto query_node = buildSelectWithUnionExpression(expression, false /*is_subquery*/, {} /*cte_name*/, context);
result = std::move(query_node);
}
2022-07-18 17:20:28 +00:00
else if (const auto * with_element = expression->as<ASTWithElement>())
{
auto with_element_subquery = with_element->subquery->as<ASTSubquery &>().children.at(0);
2022-12-06 09:44:38 +00:00
auto query_node = buildSelectWithUnionExpression(with_element_subquery, true /*is_subquery*/, with_element->name /*cte_name*/, context);
2022-07-18 17:20:28 +00:00
2022-10-19 10:25:27 +00:00
result = std::move(query_node);
2022-07-14 11:20:16 +00:00
}
else if (const auto * columns_regexp_matcher = expression->as<ASTColumnsRegexpMatcher>())
{
auto column_transformers = buildColumnTransformers(columns_regexp_matcher->transformers, context);
2022-07-14 11:20:16 +00:00
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 = buildColumnTransformers(columns_list_matcher->transformers, context);
2022-07-14 11:20:16 +00:00
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->qualifier->as<ASTIdentifier &>();
auto column_transformers = buildColumnTransformers(qualified_columns_regexp_matcher->transformers, context);
2022-07-14 11:20:16 +00:00
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->qualifier->as<ASTIdentifier &>();
2022-07-14 11:20:16 +00:00
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 = buildColumnTransformers(qualified_columns_list_matcher->transformers, context);
2022-10-19 10:25:27 +00:00
result = std::make_shared<MatcherNode>(Identifier(qualified_identifier.name_parts), std::move(column_list_identifiers), std::move(column_transformers));
2022-07-14 11:20:16 +00:00
}
else if (const auto * query_parameter = expression->as<ASTQueryParameter>())
{
throw Exception(ErrorCodes::UNKNOWN_QUERY_PARAMETER,
"Query parameter {} was not set",
backQuote(query_parameter->name));
}
2022-07-14 11:20:16 +00:00
else
{
2022-08-25 18:35:16 +00:00
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Invalid expression. Expected identifier, literal, matcher, function, subquery. Actual {}",
expression->formatForErrorMessage());
2022-07-14 11:20:16 +00:00
}
result->setAlias(expression->tryGetAlias());
result->setOriginalAST(expression);
return result;
}
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr QueryTreeBuilder::buildWindow(const ASTPtr & window_definition, const ContextPtr & context) const
2022-09-12 14:14:40 +00:00
{
const auto & window_definition_typed = window_definition->as<const ASTWindowDefinition &>();
WindowFrame window_frame;
if (!window_definition_typed.frame_is_default)
{
window_frame.is_default = false;
window_frame.type = window_definition_typed.frame_type;
window_frame.begin_type = window_definition_typed.frame_begin_type;
window_frame.begin_preceding = window_definition_typed.frame_begin_preceding;
window_frame.end_type = window_definition_typed.frame_end_type;
window_frame.end_preceding = window_definition_typed.frame_end_preceding;
}
auto window_node = std::make_shared<WindowNode>(window_frame);
window_node->setParentWindowName(window_definition_typed.parent_window_name);
if (window_definition_typed.partition_by)
2022-12-06 09:44:38 +00:00
window_node->getPartitionByNode() = buildExpressionList(window_definition_typed.partition_by, context);
2022-09-12 14:14:40 +00:00
if (window_definition_typed.order_by)
2022-12-06 09:44:38 +00:00
window_node->getOrderByNode() = buildSortList(window_definition_typed.order_by, context);
2022-09-12 14:14:40 +00:00
if (window_definition_typed.frame_begin_offset)
2022-12-06 09:44:38 +00:00
window_node->getFrameBeginOffsetNode() = buildExpression(window_definition_typed.frame_begin_offset, context);
2022-09-12 14:14:40 +00:00
if (window_definition_typed.frame_end_offset)
2022-12-06 09:44:38 +00:00
window_node->getFrameEndOffsetNode() = buildExpression(window_definition_typed.frame_end_offset, context);
2022-09-12 14:14:40 +00:00
window_node->setOriginalAST(window_definition);
return window_node;
}
2022-12-06 09:44:38 +00:00
QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select_query, const ContextPtr & context) const
2022-07-14 11:20:16 +00:00
{
if (!tables_in_select_query)
{
2022-09-06 16:46:30 +00:00
/** If no table is specified in SELECT query we substitute system.one table.
2022-07-14 11:20:16 +00:00
* SELECT * FROM system.one;
*/
Identifier storage_identifier("system.one");
2022-07-18 17:20:28 +00:00
return std::make_shared<IdentifierNode>(storage_identifier);
2022-07-14 11:20:16 +00:00
}
auto & tables = tables_in_select_query->as<ASTTablesInSelectQuery &>();
2022-08-15 16:34:10 +00:00
QueryTreeNodes table_expressions;
2022-07-14 11:20:16 +00:00
for (const auto & table_element_untyped : tables.children)
{
const auto & table_element = table_element_untyped->as<ASTTablesInSelectQueryElement &>();
if (table_element.table_expression)
{
2022-07-18 17:20:28 +00:00
auto & table_expression = table_element.table_expression->as<ASTTableExpression &>();
std::optional<TableExpressionModifiers> table_expression_modifiers;
if (table_expression.final || table_expression.sample_size)
{
bool has_final = table_expression.final;
std::optional<TableExpressionModifiers::Rational> sample_size_ratio;
std::optional<TableExpressionModifiers::Rational> sample_offset_ratio;
if (table_expression.sample_size)
{
auto & ast_sample_size_ratio = table_expression.sample_size->as<ASTSampleRatio &>();
sample_size_ratio = ast_sample_size_ratio.ratio;
if (table_expression.sample_offset)
{
auto & ast_sample_offset_ratio = table_expression.sample_offset->as<ASTSampleRatio &>();
sample_offset_ratio = ast_sample_offset_ratio.ratio;
}
}
table_expression_modifiers = TableExpressionModifiers(has_final, sample_size_ratio, sample_offset_ratio);
}
2022-07-14 11:20:16 +00:00
2022-07-18 17:20:28 +00:00
if (table_expression.database_and_table_name)
2022-07-14 11:20:16 +00:00
{
2022-07-18 17:20:28 +00:00
auto & table_identifier_typed = table_expression.database_and_table_name->as<ASTTableIdentifier &>();
2022-07-14 11:20:16 +00:00
auto storage_identifier = Identifier(table_identifier_typed.name_parts);
QueryTreeNodePtr table_identifier_node;
2022-07-14 11:20:16 +00:00
if (table_expression_modifiers)
table_identifier_node = std::make_shared<IdentifierNode>(storage_identifier, *table_expression_modifiers);
else
table_identifier_node = std::make_shared<IdentifierNode>(storage_identifier);
2022-07-14 11:20:16 +00:00
table_identifier_node->setAlias(table_identifier_typed.tryGetAlias());
table_identifier_node->setOriginalAST(table_element.table_expression);
table_expressions.push_back(std::move(table_identifier_node));
2022-07-14 11:20:16 +00:00
}
2022-07-18 17:20:28 +00:00
else if (table_expression.subquery)
{
auto & subquery_expression = table_expression.subquery->as<ASTSubquery &>();
const auto & select_with_union_query = subquery_expression.children[0];
2022-07-14 11:20:16 +00:00
2022-12-06 09:44:38 +00:00
auto node = buildSelectWithUnionExpression(select_with_union_query, true /*is_subquery*/, {} /*cte_name*/, context);
2022-07-18 17:20:28 +00:00
node->setAlias(subquery_expression.tryGetAlias());
node->setOriginalAST(select_with_union_query);
if (table_expression_modifiers)
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD,
"Table expression modifiers {} are not supported for subquery {}",
2022-10-31 10:45:12 +00:00
table_expression_modifiers->formatForErrorMessage(),
node->formatASTForErrorMessage());
}
2022-08-15 16:34:10 +00:00
table_expressions.push_back(std::move(node));
2022-07-18 17:20:28 +00:00
}
2022-07-20 15:25:24 +00:00
else if (table_expression.table_function)
2022-07-14 11:20:16 +00:00
{
2022-07-20 15:25:24 +00:00
auto & table_function_expression = table_expression.table_function->as<ASTFunction &>();
auto node = std::make_shared<TableFunctionNode>(table_function_expression.name);
if (table_function_expression.arguments)
{
2022-10-19 10:25:27 +00:00
const auto & function_arguments_list = table_function_expression.arguments->as<ASTExpressionList &>().children;
2022-07-20 15:25:24 +00:00
for (const auto & argument : function_arguments_list)
2022-08-25 18:35:16 +00:00
{
if (!node->getSettingsChanges().empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table function '{}' has arguments after SETTINGS",
table_function_expression.formatForErrorMessage());
2022-08-25 18:35:16 +00:00
if (argument->as<ASTSelectQuery>() || argument->as<ASTSelectWithUnionQuery>() || argument->as<ASTSelectIntersectExceptQuery>())
2022-12-06 09:44:38 +00:00
node->getArguments().getNodes().push_back(buildSelectOrUnionExpression(argument, false /*is_subquery*/, {} /*cte_name*/, context));
else if (const auto * ast_set = argument->as<ASTSetQuery>())
node->setSettingsChanges(ast_set->changes);
2022-08-25 18:35:16 +00:00
else
2022-12-06 09:44:38 +00:00
node->getArguments().getNodes().push_back(buildExpression(argument, context));
2022-08-25 18:35:16 +00:00
}
2022-07-20 15:25:24 +00:00
}
if (table_expression_modifiers)
node->setTableExpressionModifiers(*table_expression_modifiers);
2022-07-20 15:25:24 +00:00
node->setAlias(table_function_expression.tryGetAlias());
node->setOriginalAST(table_expression.table_function);
2022-08-15 16:34:10 +00:00
table_expressions.push_back(std::move(node));
}
else
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Unsupported table expression node {}",
table_element.table_expression->formatForErrorMessage());
2022-07-14 11:20:16 +00:00
}
}
2022-08-15 16:34:10 +00:00
if (table_element.table_join)
{
const auto & table_join = table_element.table_join->as<ASTTableJoin &>();
2022-07-14 11:20:16 +00:00
2022-08-15 16:34:10 +00:00
auto right_table_expression = std::move(table_expressions.back());
table_expressions.pop_back();
2022-07-14 11:20:16 +00:00
2022-08-15 16:34:10 +00:00
auto left_table_expression = std::move(table_expressions.back());
table_expressions.pop_back();
2022-07-14 11:20:16 +00:00
2022-08-15 16:34:10 +00:00
QueryTreeNodePtr join_expression;
2022-07-14 11:20:16 +00:00
2022-08-15 16:34:10 +00:00
if (table_join.using_expression_list)
2022-12-06 09:44:38 +00:00
join_expression = buildExpressionList(table_join.using_expression_list, context);
2022-08-15 16:34:10 +00:00
else if (table_join.on_expression)
2022-12-06 09:44:38 +00:00
join_expression = buildExpression(table_join.on_expression, context);
2022-07-14 11:20:16 +00:00
2022-09-27 15:04:03 +00:00
const auto & settings = context->getSettingsRef();
auto join_default_strictness = settings.join_default_strictness;
auto any_join_distinct_right_table_keys = settings.any_join_distinct_right_table_keys;
JoinStrictness result_join_strictness = table_join.strictness;
JoinKind result_join_kind = table_join.kind;
if (result_join_strictness == JoinStrictness::Unspecified && (result_join_kind != JoinKind::Cross && result_join_kind != JoinKind::Comma))
{
if (join_default_strictness == JoinStrictness::Any)
result_join_strictness = JoinStrictness::Any;
else if (join_default_strictness == JoinStrictness::All)
result_join_strictness = JoinStrictness::All;
else
throw Exception(ErrorCodes::EXPECTED_ALL_OR_ANY,
"Expected ANY or ALL in JOIN section, because setting (join_default_strictness) is empty");
}
if (any_join_distinct_right_table_keys)
{
if (result_join_strictness == JoinStrictness::Any && result_join_kind == JoinKind::Inner)
{
result_join_strictness = JoinStrictness::Semi;
result_join_kind = JoinKind::Left;
}
if (result_join_strictness == JoinStrictness::Any)
result_join_strictness = JoinStrictness::RightAny;
}
else if (result_join_strictness == JoinStrictness::Any && result_join_kind == JoinKind::Full)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "ANY FULL JOINs are not implemented");
}
2022-08-15 16:34:10 +00:00
auto join_node = std::make_shared<JoinNode>(std::move(left_table_expression),
std::move(right_table_expression),
std::move(join_expression),
table_join.locality,
2022-09-27 15:04:03 +00:00
result_join_strictness,
result_join_kind);
2022-07-14 11:20:16 +00:00
2022-08-15 16:34:10 +00:00
/** Original AST is not set because it will contain only join part and does
* not include left table expression.
*/
table_expressions.emplace_back(std::move(join_node));
}
2022-07-14 11:20:16 +00:00
2022-08-15 16:34:10 +00:00
if (table_element.array_join)
{
auto & array_join_expression = table_element.array_join->as<ASTArrayJoin &>();
bool is_left_array_join = array_join_expression.kind == ASTArrayJoin::Kind::Left;
2022-07-14 11:20:16 +00:00
2022-08-15 16:34:10 +00:00
auto last_table_expression = std::move(table_expressions.back());
table_expressions.pop_back();
2022-07-14 11:20:16 +00:00
2022-12-06 09:44:38 +00:00
auto array_join_expressions_list = buildExpressionList(array_join_expression.expression_list, context);
2022-08-15 16:34:10 +00:00
auto array_join_node = std::make_shared<ArrayJoinNode>(std::move(last_table_expression), std::move(array_join_expressions_list), is_left_array_join);
2022-08-21 11:46:07 +00:00
/** Original AST is not set because it will contain only array join part and does
* not include left table expression.
*/
2022-08-15 16:34:10 +00:00
table_expressions.push_back(std::move(array_join_node));
}
2022-07-14 11:20:16 +00:00
}
2022-08-15 16:34:10 +00:00
if (table_expressions.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Query FROM section cannot be empty");
if (table_expressions.size() > 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Query FROM section cannot have more than 1 root table expression");
2022-07-14 11:20:16 +00:00
2022-08-15 16:34:10 +00:00
return table_expressions.back();
2022-07-14 11:20:16 +00:00
}
ColumnTransformersNodes QueryTreeBuilder::buildColumnTransformers(const ASTPtr & matcher_expression, const ContextPtr & context) const
2022-07-14 11:20:16 +00:00
{
ColumnTransformersNodes column_transformers;
2022-11-30 02:14:04 +00:00
if (!matcher_expression)
return column_transformers;
for (const auto & child : matcher_expression->children)
2022-07-14 11:20:16 +00:00
{
if (auto * apply_transformer = child->as<ASTColumnsApplyTransformer>())
{
if (apply_transformer->lambda)
{
2022-12-06 09:44:38 +00:00
auto lambda_query_tree_node = buildExpression(apply_transformer->lambda, context);
2022-07-14 11:20:16 +00:00
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)
2022-12-06 09:44:38 +00:00
function_node->getParametersNode() = buildExpressionList(apply_transformer->parameters, context);
2022-07-14 11:20:16 +00:00
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), except_transformer->is_strict));
2022-07-14 11:20:16 +00:00
}
}
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, buildExpression(replacement.children[0], context)});
2022-07-14 11:20:16 +00:00
}
column_transformers.emplace_back(std::make_shared<ReplaceColumnTransformerNode>(replacements, replace_transformer->is_strict));
2022-07-14 11:20:16 +00:00
}
else
{
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Unsupported column matcher {}", child->formatForErrorMessage());
}
}
return column_transformers;
}
2022-08-23 09:50:02 +00:00
}
2022-09-27 15:04:03 +00:00
QueryTreeNodePtr buildQueryTree(ASTPtr query, ContextPtr context)
2022-07-14 11:20:16 +00:00
{
2022-09-27 15:04:03 +00:00
QueryTreeBuilder builder(std::move(query), context);
2022-07-14 11:20:16 +00:00
return builder.getQueryTreeNode();
}
}