Change signature of ASTSelectQuery::arrayJoinExpressionList

This commit is contained in:
vdimir 2021-06-28 12:53:54 +03:00
parent 34b9bd7d33
commit e8e26463bf
No known key found for this signature in database
GPG Key ID: F57B3E10A21DBB31
8 changed files with 15 additions and 26 deletions

View File

@ -60,7 +60,7 @@ public:
private:
static void visit(const ASTSelectQuery & node, ASTPtr &, Data & data)
{
ASTPtr array_join_expression_list = node.arrayJoinExpressionList();
auto [array_join_expression_list, _] = node.arrayJoinExpressionList();
if (!array_join_expression_list)
throw Exception("Logical error: no ARRAY JOIN", ErrorCodes::LOGICAL_ERROR);

View File

@ -803,8 +803,7 @@ ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActi
{
const auto * select_query = getSelectQuery();
bool is_array_join_left;
ASTPtr array_join_expression_list = select_query->arrayJoinExpressionList(is_array_join_left);
auto [array_join_expression_list, is_array_join_left] = select_query->arrayJoinExpressionList();
if (!array_join_expression_list)
return nullptr;

View File

@ -854,7 +854,7 @@ static std::pair<UInt64, UInt64> getLimitLengthAndOffset(const ASTSelectQuery &
static UInt64 getLimitForSorting(const ASTSelectQuery & query, ContextPtr context)
{
/// Partial sort can be done if there is LIMIT but no DISTINCT or LIMIT BY, neither ARRAY JOIN.
if (!query.distinct && !query.limitBy() && !query.limit_with_ties && !query.arrayJoinExpressionList() && query.limitLength())
if (!query.distinct && !query.limitBy() && !query.limit_with_ties && !query.arrayJoinExpressionList().first && query.limitLength())
{
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context);
if (limit_length > std::numeric_limits<UInt64>::max() - limit_offset)
@ -1352,17 +1352,15 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
bool apply_prelimit = apply_limit &&
query.limitLength() && !query.limit_with_ties &&
!hasWithTotalsInAnySubqueryInFromClause(query) &&
!query.arrayJoinExpressionList() &&
!query.arrayJoinExpressionList().first &&
!query.distinct &&
!expressions.hasLimitBy() &&
!settings.extremes &&
!has_withfill;
bool apply_offset = options.to_stage != QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit;
bool limit_applied = false;
if (apply_prelimit)
{
executePreLimit(query_plan, /* do_not_skip_offset= */!apply_offset);
limit_applied = true;
}
/** If there was more than one stream,
@ -1384,7 +1382,6 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
if (query.limit_with_ties && apply_offset)
{
executeLimit(query_plan);
limit_applied = true;
}
/// Projection not be done on the shards, since then initiator will not find column in blocks.
@ -1398,6 +1395,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
/// Extremes are calculated before LIMIT, but after LIMIT BY. This is Ok.
executeExtremes(query_plan);
bool limit_applied = apply_prelimit || (query.limit_with_ties && apply_offset);
/// Limit is no longer needed if there is prelimit.
///
/// NOTE: that LIMIT cannot be applied if OFFSET should not be applied,

View File

@ -39,7 +39,7 @@ bool PredicateExpressionsOptimizer::optimize(ASTSelectQuery & select_query)
if (!select_query.tables() || select_query.tables()->children.empty())
return false;
if ((!select_query.where() && !select_query.prewhere()) || select_query.arrayJoinExpressionList())
if ((!select_query.where() && !select_query.prewhere()) || select_query.arrayJoinExpressionList().first)
return false;
const auto & tables_predicates = extractTablesPredicates(select_query.where(), select_query.prewhere());

View File

@ -422,7 +422,7 @@ void executeScalarSubqueries(ASTPtr & query, ContextPtr context, size_t subquery
void getArrayJoinedColumns(ASTPtr & query, TreeRewriterResult & result, const ASTSelectQuery * select_query,
const NamesAndTypesList & source_columns, const NameSet & source_columns_set)
{
if (!select_query->arrayJoinExpressionList())
if (!select_query->arrayJoinExpressionList().first)
return;
ArrayJoinedColumnsVisitor::Data visitor_data{
@ -433,10 +433,10 @@ void getArrayJoinedColumns(ASTPtr & query, TreeRewriterResult & result, const AS
/// to get the correct number of rows.
if (result.array_join_result_to_source.empty())
{
if (select_query->arrayJoinExpressionList()->children.empty())
if (select_query->arrayJoinExpressionList().first->children.empty())
throw DB::Exception("ARRAY JOIN requires an argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
ASTPtr expr = select_query->arrayJoinExpressionList()->children.at(0);
ASTPtr expr = select_query->arrayJoinExpressionList().first->children.at(0);
String source_name = expr->getColumnName();
String result_name = expr->getAliasOrColumnName();

View File

@ -319,24 +319,16 @@ bool ASTSelectQuery::withFill() const
}
ASTPtr ASTSelectQuery::arrayJoinExpressionList(bool & is_left) const
std::pair<ASTPtr, bool> ASTSelectQuery::arrayJoinExpressionList() const
{
const ASTArrayJoin * array_join = getFirstArrayJoin(*this);
if (!array_join)
return {};
is_left = (array_join->kind == ASTArrayJoin::Kind::Left);
return array_join->expression_list;
bool is_left = (array_join->kind == ASTArrayJoin::Kind::Left);
return {array_join->expression_list, is_left};
}
ASTPtr ASTSelectQuery::arrayJoinExpressionList() const
{
bool is_left;
return arrayJoinExpressionList(is_left);
}
const ASTTablesInSelectQueryElement * ASTSelectQuery::join() const
{
return getFirstTableJoin(*this);

View File

@ -123,8 +123,8 @@ public:
/// Compatibility with old parser of tables list. TODO remove
ASTPtr sampleSize() const;
ASTPtr sampleOffset() const;
ASTPtr arrayJoinExpressionList(bool & is_left) const;
ASTPtr arrayJoinExpressionList() const;
std::pair<ASTPtr, bool> arrayJoinExpressionList() const;
const ASTTablesInSelectQueryElement * join() const;
bool final() const;
bool withFill() const;

View File

@ -373,7 +373,7 @@ bool MergeTreeWhereOptimizer::cannotBeMoved(const ASTPtr & ptr, bool is_final) c
void MergeTreeWhereOptimizer::determineArrayJoinedNames(ASTSelectQuery & select)
{
auto array_join_expression_list = select.arrayJoinExpressionList();
auto [array_join_expression_list, _] = select.arrayJoinExpressionList();
/// much simplified code from ExpressionAnalyzer::getArrayJoinedColumns()
if (!array_join_expression_list)