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>
|
2023-02-16 10:31:24 +00:00
|
|
|
#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>
|
2022-09-04 15:20:59 +00:00
|
|
|
#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>
|
2022-09-15 11:45:01 +00:00
|
|
|
#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;
|
2022-10-06 19:36:06 +00:00
|
|
|
extern const int BAD_ARGUMENTS;
|
2023-02-16 10:31:24 +00:00
|
|
|
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
|
|
|
|
2023-01-13 16:35:36 +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
|
2023-01-23 21:13:58 +00:00
|
|
|
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
|
|
|
|
2022-10-26 14:47:40 +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)
|
2022-10-26 14:47:40 +00:00
|
|
|
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)
|
2022-10-26 14:47:40 +00:00
|
|
|
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)
|
2022-10-26 14:47:40 +00:00
|
|
|
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)
|
2022-10-26 14:47:40 +00:00
|
|
|
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);
|
2022-10-26 14:47:40 +00:00
|
|
|
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;
|
|
|
|
|
2023-02-21 03:29:14 +00:00
|
|
|
if (is_subquery)
|
|
|
|
{
|
|
|
|
if (const Settings & settings_ref = updated_context->getSettingsRef(); settings_ref.limit || settings_ref.offset)
|
|
|
|
{
|
|
|
|
Settings settings = updated_context->getSettings();
|
|
|
|
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 &>();
|
|
|
|
updated_context->applySettingsChanges(set_query.changes);
|
|
|
|
settings_changes = set_query.changes;
|
|
|
|
}
|
|
|
|
|
|
|
|
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);
|
2022-09-02 14:36:35 +00:00
|
|
|
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);
|
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-09-04 15:20:59 +00:00
|
|
|
|
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
|
|
|
|
2022-08-26 16:23:26 +00:00
|
|
|
auto select_limit = select_query_typed.limitLength();
|
|
|
|
if (select_limit)
|
2022-12-06 09:44:38 +00:00
|
|
|
current_query_tree->getLimit() = buildExpression(select_limit, current_context);
|
2022-08-26 16:23:26 +00:00
|
|
|
|
|
|
|
auto select_offset = select_query_typed.limitOffset();
|
|
|
|
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);
|
2022-09-15 11:45:01 +00:00
|
|
|
auto interpolate_node = std::make_shared<InterpolateNode>(std::move(expression_to_interpolate), std::move(interpolate_expression));
|
2022-09-03 16:20:09 +00:00
|
|
|
|
2022-09-15 11:45:01 +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>())
|
|
|
|
{
|
2023-01-13 16:35:36 +00:00
|
|
|
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>())
|
|
|
|
{
|
2022-11-02 06:11:37 +00:00
|
|
|
auto & qualified_identifier = qualified_asterisk->qualifier->as<ASTIdentifier &>();
|
2023-01-13 16:35:36 +00:00
|
|
|
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);
|
|
|
|
|
|
|
|
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
|
|
|
}
|
2023-02-16 10:31:24 +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>())
|
|
|
|
{
|
2023-01-13 16:35:36 +00:00
|
|
|
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});
|
|
|
|
}
|
|
|
|
|
2023-01-13 16:35:36 +00:00
|
|
|
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>())
|
|
|
|
{
|
2022-11-02 06:11:37 +00:00
|
|
|
auto & qualified_identifier = qualified_columns_regexp_matcher->qualifier->as<ASTIdentifier &>();
|
2023-01-13 16:35:36 +00:00
|
|
|
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>())
|
|
|
|
{
|
2022-11-02 06:11:37 +00:00
|
|
|
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});
|
|
|
|
}
|
|
|
|
|
2023-01-13 16:35:36 +00:00
|
|
|
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
|
|
|
}
|
2023-02-16 10:31:24 +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 &>();
|
2022-09-04 15:20:59 +00:00
|
|
|
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);
|
2022-09-04 15:20:59 +00:00
|
|
|
QueryTreeNodePtr table_identifier_node;
|
2022-07-14 11:20:16 +00:00
|
|
|
|
2022-09-04 15:20:59 +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
|
|
|
|
2022-09-04 15:20:59 +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);
|
|
|
|
|
2022-09-04 15:20:59 +00:00
|
|
|
if (table_expression_modifiers)
|
|
|
|
{
|
2022-10-26 15:19:29 +00:00
|
|
|
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(),
|
2022-10-26 15:19:29 +00:00
|
|
|
node->formatASTForErrorMessage());
|
2022-09-04 15:20:59 +00:00
|
|
|
}
|
|
|
|
|
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 (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));
|
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
|
|
|
}
|
|
|
|
|
2022-09-04 15:20:59 +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
|
|
|
|
{
|
2023-01-23 21:13:58 +00:00
|
|
|
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
|
|
|
}
|
|
|
|
|
|
|
|
|
2023-01-13 16:35:36 +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;
|
|
|
|
|
2022-11-02 06:11:37 +00:00
|
|
|
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);
|
|
|
|
|
2022-07-14 16:02:47 +00:00
|
|
|
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 &>();
|
2023-02-14 22:02:54 +00:00
|
|
|
replacements.emplace_back(ReplaceColumnTransformerNode::Replacement{replacement.name, buildExpression(replacement.children[0], context)});
|
2022-07-14 11:20:16 +00:00
|
|
|
}
|
|
|
|
|
2022-07-14 16:02:47 +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();
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|