Fix 03228_pr_subquery_view_order_by

This commit is contained in:
Igor Nikonov 2024-08-30 20:04:43 +00:00
parent d12aac7d12
commit 3c29f27dd2
3 changed files with 25 additions and 1 deletions

View File

@ -50,6 +50,7 @@ size_t tryLiftUpUnion(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes)
expr_node.step = std::make_unique<ExpressionStep>(
expr_node.children.front()->step->getOutputStream(),
expression->getExpression().clone());
expr_node.step->setStepDescription(expression->getStepDescription());
}
/// - Expression - Something

View File

@ -18,6 +18,7 @@
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
#include <Processors/QueryPlan/Optimizations/actionsDAGUtils.h>
#include <Processors/QueryPlan/ReadFromMergeTree.h>
#include <Processors/QueryPlan/ReadFromRemote.h>
#include <Processors/QueryPlan/SortingStep.h>
#include <Processors/QueryPlan/TotalsHavingStep.h>
#include <Processors/QueryPlan/UnionStep.h>
@ -899,6 +900,18 @@ AggregationInputOrder buildInputOrderInfo(AggregatingStep & aggregating, QueryPl
return {};
}
static bool readingFromParallelReplicas(const QueryPlan::Node * node)
{
IQueryPlanStep * step = node->step.get();
while (!node->children.empty())
{
step = node->children.front()->step.get();
node = node->children.front();
}
return typeid_cast<const ReadFromParallelRemoteReplicasStep *>(step);
}
void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes)
{
if (node.children.size() != 1)
@ -924,6 +937,16 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes)
std::vector<InputOrderInfoPtr> infos;
infos.reserve(node.children.size());
for (const auto * child : union_node->children)
{
/// in case of parallel replicas
/// avoid applying read-in-order optimization for local replica
/// since it will lead to different parallel replicas modes
/// between local and remote nodes
if (readingFromParallelReplicas(child))
return;
}
for (auto * child : union_node->children)
{
infos.push_back(buildInputOrderInfo(*sorting, *child, steps_to_update));

View File

@ -12,7 +12,7 @@ FROM
)
ORDER BY number DESC
LIMIT 20
SETTINGS cluster_for_parallel_replicas = 'parallel_replicas', allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree = 1, parallel_replicas_local_plan = 1, query_plan_lift_up_union = 0;
SETTINGS cluster_for_parallel_replicas = 'parallel_replicas', allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree = 1, parallel_replicas_local_plan = 1;
DROP TABLE view1;
DROP TABLE table1;