You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2023/06/30 04:36:28 UTC

[doris] branch runtimefilter_multi_send updated (f94e358886 -> dae170394f)

This is an automated email from the ASF dual-hosted git repository.

morrysnow pushed a change to branch runtimefilter_multi_send
in repository https://gitbox.apache.org/repos/asf/doris.git


 discard f94e358886 [opt](Nereids) support set cte shuffle type for each consumer
 discard c6b3ac6905 [feature](runtime_filter) MultiCastSender supporting runtime filter on be (#21304)
     add f07e0d7686 [typo](docs) Some typo in nereids.md has been fixed (#20475)
     add 9756ff1e25 [feature](Nereids): infer distinct from SetOperator (#21235)
     add a3033bff42 [Fix](s3FileWriter) fix bytes_appended bug for s3_file_writer (#21348)
     add 3fb75c1844 [docs](workload-group) Modify workload group docs (#21349)
     add a3fee40ce5 [bugfix](build script) fix noavx2 package name branch condition #21356
     add 6d63261b71 [docs]<docs>Add file system benchmark tools docs (#21262)
     add 9f44c2d80d [fix](nereids) nest loop join stats estimation (#21275)
     add 33fa5dd1e9 [fix](cast) fix coredump of cast string of invalid datetime (#21350)
     add 2fcb0e090b [Fix](Snapshot) Shoule use false instead of 0 in while loop (#20966)
     new 45c7f137db [feature](runtime_filter) MultiCastSender supporting runtime filter on be (#21304)
     new dae170394f [opt](Nereids) support set cte shuffle type for each consumer

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (f94e358886)
            \
             N -- N -- N   refs/heads/runtimefilter_multi_send (dae170394f)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/io/fs/s3_file_writer.h                      |   3 -
 be/src/olap/snapshot_manager.cpp                   |   2 +-
 be/src/vec/runtime/vdatetime_value.cpp             |   3 +
 build-for-release.sh                               |   2 +-
 docs/en/docs/admin-manual/workload-group.md        |   2 +-
 docs/en/docs/lakehouse/fs_benchmark_tool.md        | 233 +++++++++++++++++++++
 .../Create/CREATE-WORKLOAD-GROUP.md                |   2 +-
 docs/sidebars.json                                 |   3 +-
 docs/zh-CN/docs/admin-manual/workload-group.md     |   6 +-
 docs/zh-CN/docs/lakehouse/fs_benchmark_tool.md     | 229 ++++++++++++++++++++
 docs/zh-CN/docs/query-acceleration/nereids.md      |   6 +-
 .../Create/CREATE-WORKLOAD-GROUP.md                |   6 +-
 .../glue/translator/PhysicalPlanTranslator.java    |   1 +
 .../doris/nereids/jobs/executor/Rewriter.java      |   2 +
 .../org/apache/doris/nereids/rules/RuleType.java   |   1 +
 .../analysis/ProjectWithDistinctToAggregate.java   |  36 ++--
 ...ForUnion.java => InferSetOperatorDistinct.java} |  29 ++-
 .../doris/nereids/stats/FilterEstimation.java      |  19 +-
 .../apache/doris/nereids/stats/JoinEstimation.java |  85 +++++---
 .../trees/plans/logical/LogicalAggregate.java      |   7 +
 .../doris/planner/MultiCastPlanFragment.java       |   5 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query11.out |  20 +-
 .../nereids_tpcds_shape_sf100_p0/shape/query14.out |  86 ++++----
 .../nereids_tpcds_shape_sf100_p0/shape/query18.out |  59 +++---
 .../nereids_tpcds_shape_sf100_p0/shape/query4.out  |  44 ++--
 .../nereids_tpcds_shape_sf100_p0/shape/query74.out |  16 +-
 .../nereids_tpch_shape_sf1000_p0/shape/q10.out     |  35 ++--
 .../data/nereids_tpch_shape_sf1000_p0/shape/q5.out |   5 +-
 .../data/nereids_tpch_shape_sf500_p0/shape/q10.out |  35 ++--
 .../data/nereids_tpch_shape_sf500_p0/shape/q5.out  |   5 +-
 .../cast_function/test_cast_to_datetime.out        |  19 ++
 .../cast_function/test_cast_to_datetime.groovy     |  16 +-
 32 files changed, 769 insertions(+), 253 deletions(-)
 create mode 100644 docs/en/docs/lakehouse/fs_benchmark_tool.md
 create mode 100644 docs/zh-CN/docs/lakehouse/fs_benchmark_tool.md
 copy fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/{BuildAggForUnion.java => InferSetOperatorDistinct.java} (55%)
 create mode 100644 regression-test/data/query_p0/sql_functions/cast_function/test_cast_to_datetime.out
 copy be/src/common/env_config.h.in => regression-test/suites/query_p0/sql_functions/cast_function/test_cast_to_datetime.groovy (58%)


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 02/02: [opt](Nereids) support set cte shuffle type for each consumer

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morrysnow pushed a commit to branch runtimefilter_multi_send
in repository https://gitbox.apache.org/repos/asf/doris.git

commit dae170394fd4cc7a31ef6ea46fdf6230e68a6f62
Author: morrySnow <mo...@126.com>
AuthorDate: Thu Jun 29 21:30:50 2023 +0800

    [opt](Nereids) support set cte shuffle type for each consumer
---
 .../glue/translator/PhysicalPlanTranslator.java    | 134 ++++++++++-----------
 .../properties/ChildOutputPropertyDeriver.java     |   2 +-
 .../properties/ChildrenPropertiesRegulator.java    |  34 ++++++
 .../properties/DistributionSpecMustShuffle.java    |  35 ++++++
 .../nereids/properties/PhysicalProperties.java     |   3 +
 .../org/apache/doris/planner/DataStreamSink.java   |  86 ++++++++++++-
 .../org/apache/doris/planner/ExchangeNode.java     |  10 +-
 .../doris/planner/MultiCastPlanFragment.java       |   5 +-
 .../org/apache/doris/planner/PlanFragment.java     |   2 +-
 .../java/org/apache/doris/planner/PlanNode.java    |   4 +-
 10 files changed, 229 insertions(+), 86 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java
index 8210c1a082..860bb624dd 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java
@@ -247,27 +247,21 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor<PlanFragment, Pla
     @Override
     public PlanFragment visitPhysicalDistribute(PhysicalDistribute<? extends Plan> distribute,
             PlanTranslatorContext context) {
-        PlanFragment childFragment = distribute.child().accept(this, context);
+        PlanFragment inputFragment = distribute.child().accept(this, context);
         // TODO: why need set streaming here? should remove this.
-        if (childFragment.getPlanRoot() instanceof AggregationNode
+        if (inputFragment.getPlanRoot() instanceof AggregationNode
                 && distribute.child() instanceof PhysicalHashAggregate
-                && context.getFirstAggregateInFragment(childFragment) == distribute.child()) {
+                && context.getFirstAggregateInFragment(inputFragment) == distribute.child()) {
             PhysicalHashAggregate<?> hashAggregate = (PhysicalHashAggregate<?>) distribute.child();
             if (hashAggregate.getAggPhase() == AggPhase.LOCAL
                     && hashAggregate.getAggMode() == AggMode.INPUT_TO_BUFFER) {
-                AggregationNode aggregationNode = (AggregationNode) childFragment.getPlanRoot();
+                AggregationNode aggregationNode = (AggregationNode) inputFragment.getPlanRoot();
                 aggregationNode.setUseStreamingPreagg(hashAggregate.isMaybeUsingStream());
             }
         }
 
-        ExchangeNode exchangeNode = new ExchangeNode(context.nextPlanNodeId(), childFragment.getPlanRoot());
+        ExchangeNode exchangeNode = new ExchangeNode(context.nextPlanNodeId(), inputFragment.getPlanRoot());
         updateLegacyPlanIdToPhysicalPlan(exchangeNode, distribute);
-        exchangeNode.setNumInstances(childFragment.getPlanRoot().getNumInstances());
-        if (distribute.getDistributionSpec() instanceof DistributionSpecGather) {
-            // gather to one instance
-            exchangeNode.setNumInstances(1);
-        }
-
         List<ExprId> validOutputIds = distribute.getOutputExprIds();
         if (distribute.child() instanceof PhysicalHashAggregate) {
             // we must add group by keys to output list,
@@ -282,8 +276,28 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor<PlanFragment, Pla
         }
         DataPartition dataPartition = toDataPartition(distribute.getDistributionSpec(), validOutputIds, context);
         PlanFragment parentFragment = new PlanFragment(context.nextFragmentId(), exchangeNode, dataPartition);
-        childFragment.setDestination(exchangeNode);
-        childFragment.setOutputPartition(dataPartition);
+        exchangeNode.setNumInstances(inputFragment.getPlanRoot().getNumInstances());
+        if (distribute.getDistributionSpec() instanceof DistributionSpecGather) {
+            // gather to one instance
+            exchangeNode.setNumInstances(1);
+        }
+
+        // process multicast sink
+        if (inputFragment instanceof MultiCastPlanFragment) {
+            MultiCastDataSink multiCastDataSink = (MultiCastDataSink) inputFragment.getSink();
+            DataStreamSink dataStreamSink = multiCastDataSink.getDataStreamSinks().get(
+                    multiCastDataSink.getDataStreamSinks().size() - 1);
+            TupleDescriptor tupleDescriptor = generateTupleDesc(distribute.getOutput(), null, context);
+            exchangeNode.updateTupleIds(tupleDescriptor);
+            dataStreamSink.setExchNodeId(exchangeNode.getId());
+            dataStreamSink.setOutputPartition(dataPartition);
+            parentFragment.addChild(inputFragment);
+            ((MultiCastPlanFragment) inputFragment).addToDest(exchangeNode);
+        } else {
+            inputFragment.setDestination(exchangeNode);
+            inputFragment.setOutputPartition(dataPartition);
+        }
+
         context.addPlanFragment(parentFragment);
         return parentFragment;
     }
@@ -763,68 +777,21 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor<PlanFragment, Pla
         PhysicalCTEProducer cteProducer = context.getCteProduceMap().get(cteId);
         Preconditions.checkState(cteProducer != null, "invalid cteProducer");
 
-        ExchangeNode exchangeNode = new ExchangeNode(context.nextPlanNodeId(), multiCastFragment.getPlanRoot());
-
-        DataStreamSink streamSink = new DataStreamSink(exchangeNode.getId());
-        streamSink.setPartition(DataPartition.RANDOM);
+        // set datasink to multicast data sink but do not set target now
+        // target will be set when translate distribute
+        DataStreamSink streamSink = new DataStreamSink();
         streamSink.setFragment(multiCastFragment);
-
         multiCastDataSink.getDataStreamSinks().add(streamSink);
         multiCastDataSink.getDestinations().add(Lists.newArrayList());
 
-        exchangeNode.setNumInstances(multiCastFragment.getPlanRoot().getNumInstances());
-
-        PlanFragment consumeFragment = new PlanFragment(context.nextFragmentId(), exchangeNode,
-                multiCastFragment.getDataPartition());
-
-        Map<Slot, Slot> projectMap = Maps.newHashMap();
-        projectMap.putAll(cteConsumer.getProducerToConsumerSlotMap());
-
-        List<NamedExpression> execList = new ArrayList<>();
-        PlanNode inputPlanNode = consumeFragment.getPlanRoot();
-        List<Slot> cteProjects = cteProducer.getProjects();
-        for (Slot slot : cteProjects) {
-            if (projectMap.containsKey(slot)) {
-                execList.add(projectMap.get(slot));
-            } else {
-                throw new RuntimeException("could not find slot in cte producer consumer projectMap");
-            }
+        // update expr to slot mapping
+        for (int i = 0; i < cteConsumer.getOutput().size(); i++) {
+            Slot producerSlot = cteProducer.getOutput().get(i);
+            Slot consumerSlot = cteConsumer.getOutput().get(i);
+            SlotRef slotRef = context.findSlotRef(producerSlot.getExprId());
+            context.addExprIdSlotRefPair(consumerSlot.getExprId(), slotRef);
         }
-
-        List<Slot> slotList = execList
-                .stream()
-                .map(NamedExpression::toSlot)
-                .collect(Collectors.toList());
-
-        TupleDescriptor tupleDescriptor = generateTupleDesc(slotList, null, context);
-
-        // update tuple list and tblTupleList
-        inputPlanNode.getTupleIds().clear();
-        inputPlanNode.getTupleIds().add(tupleDescriptor.getId());
-        inputPlanNode.getTblRefIds().clear();
-        inputPlanNode.getTblRefIds().add(tupleDescriptor.getId());
-        inputPlanNode.getNullableTupleIds().clear();
-        inputPlanNode.getNullableTupleIds().add(tupleDescriptor.getId());
-
-        List<Expr> execExprList = execList
-                .stream()
-                .map(e -> ExpressionTranslator.translate(e, context))
-                .collect(Collectors.toList());
-
-        inputPlanNode.setProjectList(execExprList);
-        inputPlanNode.setOutputTupleDesc(tupleDescriptor);
-
-        // update data partition
-        consumeFragment.setDataPartition(DataPartition.RANDOM);
-
-        SelectNode projectNode = new SelectNode(context.nextPlanNodeId(), inputPlanNode);
-        consumeFragment.setPlanRoot(projectNode);
-
-        multiCastFragment.getDestNodeList().add(exchangeNode);
-        consumeFragment.addChild(multiCastFragment);
-        context.getPlanFragments().add(consumeFragment);
-
-        return consumeFragment;
+        return multiCastFragment;
     }
 
     @Override
@@ -859,6 +826,17 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor<PlanFragment, Pla
         }
         PlanFragment inputFragment = filter.child(0).accept(this, context);
 
+        // process multicast sink
+        if (inputFragment instanceof MultiCastPlanFragment) {
+            MultiCastDataSink multiCastDataSink = (MultiCastDataSink) inputFragment.getSink();
+            DataStreamSink dataStreamSink = multiCastDataSink.getDataStreamSinks().get(
+                    multiCastDataSink.getDataStreamSinks().size() - 1);
+            filter.getConjuncts().stream()
+                    .map(e -> ExpressionTranslator.translate(e, context))
+                    .forEach(dataStreamSink::addConjunct);
+            return inputFragment;
+        }
+
         PlanNode planNode = inputFragment.getPlanRoot();
         if (planNode instanceof ExchangeNode || planNode instanceof SortNode || planNode instanceof UnionNode) {
             // the three nodes don't support conjuncts, need create a SelectNode to filter data
@@ -1397,19 +1375,31 @@ public class PhysicalPlanTranslator extends DefaultPlanVisitor<PlanFragment, Pla
                 ((AbstractPhysicalJoin<?, ?>) project.child(0).child(0)).setShouldTranslateOutput(false);
             }
         }
+
         PlanFragment inputFragment = project.child(0).accept(this, context);
+
         List<Expr> execExprList = project.getProjects()
                 .stream()
                 .map(e -> ExpressionTranslator.translate(e, context))
                 .collect(Collectors.toList());
         // TODO: fix the project alias of an aliased relation.
-
-        PlanNode inputPlanNode = inputFragment.getPlanRoot();
         List<Slot> slotList = project.getProjects()
                 .stream()
                 .map(NamedExpression::toSlot)
                 .collect(Collectors.toList());
 
+        // process multicast sink
+        if (inputFragment instanceof MultiCastPlanFragment) {
+            MultiCastDataSink multiCastDataSink = (MultiCastDataSink) inputFragment.getSink();
+            DataStreamSink dataStreamSink = multiCastDataSink.getDataStreamSinks().get(
+                    multiCastDataSink.getDataStreamSinks().size() - 1);
+            TupleDescriptor tupleDescriptor = generateTupleDesc(slotList, null, context);
+            dataStreamSink.setProjections(execExprList);
+            dataStreamSink.setOutputTupleDesc(tupleDescriptor);
+            return inputFragment;
+        }
+
+        PlanNode inputPlanNode = inputFragment.getPlanRoot();
         List<Expr> predicateList = inputPlanNode.getConjuncts();
         Set<SlotId> requiredSlotIdSet = Sets.newHashSet();
         for (Expr expr : execExprList) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriver.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriver.java
index 2695b67a93..158eaa67f2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriver.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriver.java
@@ -114,7 +114,7 @@ public class ChildOutputPropertyDeriver extends PlanVisitor<PhysicalProperties,
     public PhysicalProperties visitPhysicalCTEConsumer(
             PhysicalCTEConsumer cteConsumer, PlanContext context) {
         Preconditions.checkState(childrenOutputProperties.size() == 0);
-        return PhysicalProperties.ANY;
+        return PhysicalProperties.MUST_SHUFFLE;
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java
index 862dbb5e2b..86a3f650d6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java
@@ -27,9 +27,11 @@ import org.apache.doris.nereids.trees.expressions.ExprId;
 import org.apache.doris.nereids.trees.plans.AggMode;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribute;
+import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter;
 import org.apache.doris.nereids.trees.plans.physical.PhysicalHashAggregate;
 import org.apache.doris.nereids.trees.plans.physical.PhysicalHashJoin;
 import org.apache.doris.nereids.trees.plans.physical.PhysicalNestedLoopJoin;
+import org.apache.doris.nereids.trees.plans.physical.PhysicalProject;
 import org.apache.doris.nereids.trees.plans.physical.PhysicalSetOperation;
 import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
 import org.apache.doris.nereids.util.JoinUtils;
@@ -74,16 +76,33 @@ public class ChildrenPropertiesRegulator extends PlanVisitor<Boolean, Void> {
 
     @Override
     public Boolean visit(Plan plan, Void context) {
+        // process must shuffle
+        for (int i = 0; i < children.size(); i++) {
+            DistributionSpec distributionSpec = childrenProperties.get(i).getDistributionSpec();
+            if (distributionSpec instanceof DistributionSpecMustShuffle) {
+                updateChildEnforceAndCost(i, PhysicalProperties.EXECUTION_ANY);
+            }
+        }
         return true;
     }
 
     @Override
     public Boolean visitPhysicalHashAggregate(PhysicalHashAggregate<? extends Plan> agg, Void context) {
+        // forbid one phase agg on distribute
         if (agg.getAggMode() == AggMode.INPUT_TO_RESULT
                 && children.get(0).getPlan() instanceof PhysicalDistribute) {
             // this means one stage gather agg, usually bad pattern
             return false;
         }
+        // process must shuffle
+        visit(agg, context);
+        // process agg
+        return true;
+    }
+
+    @Override
+    public Boolean visitPhysicalFilter(PhysicalFilter<? extends Plan> filter, Void context) {
+        // do not process must shuffle
         return true;
     }
 
@@ -93,6 +112,9 @@ public class ChildrenPropertiesRegulator extends PlanVisitor<Boolean, Void> {
         Preconditions.checkArgument(children.size() == 2, String.format("children.size() is %d", children.size()));
         Preconditions.checkArgument(childrenProperties.size() == 2);
         Preconditions.checkArgument(requiredProperties.size() == 2);
+        // process must shuffle
+        visit(hashJoin, context);
+        // process hash join
         DistributionSpec leftDistributionSpec = childrenProperties.get(0).getDistributionSpec();
         DistributionSpec rightDistributionSpec = childrenProperties.get(1).getDistributionSpec();
 
@@ -229,6 +251,9 @@ public class ChildrenPropertiesRegulator extends PlanVisitor<Boolean, Void> {
         Preconditions.checkArgument(children.size() == 2, String.format("children.size() is %d", children.size()));
         Preconditions.checkArgument(childrenProperties.size() == 2);
         Preconditions.checkArgument(requiredProperties.size() == 2);
+        // process must shuffle
+        visit(nestedLoopJoin, context);
+        // process nlj
         DistributionSpec rightDistributionSpec = childrenProperties.get(1).getDistributionSpec();
         if (rightDistributionSpec instanceof DistributionSpecStorageGather) {
             updateChildEnforceAndCost(1, PhysicalProperties.GATHER);
@@ -236,8 +261,17 @@ public class ChildrenPropertiesRegulator extends PlanVisitor<Boolean, Void> {
         return true;
     }
 
+    @Override
+    public Boolean visitPhysicalProject(PhysicalProject<? extends Plan> project, Void context) {
+        // do not process must shuffle
+        return true;
+    }
+
     @Override
     public Boolean visitPhysicalSetOperation(PhysicalSetOperation setOperation, Void context) {
+        // process must shuffle
+        visit(setOperation, context);
+        // process set operation
         if (children.isEmpty()) {
             return true;
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecMustShuffle.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecMustShuffle.java
new file mode 100644
index 0000000000..8f718fbb9d
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecMustShuffle.java
@@ -0,0 +1,35 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.nereids.properties;
+
+/**
+ * present data must use after shuffle
+ */
+public class DistributionSpecMustShuffle extends DistributionSpec {
+
+    public static final DistributionSpecMustShuffle INSTANCE = new DistributionSpecMustShuffle();
+
+    public DistributionSpecMustShuffle() {
+        super();
+    }
+
+    @Override
+    public boolean satisfy(DistributionSpec other) {
+        return other instanceof DistributionSpecAny;
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/PhysicalProperties.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/PhysicalProperties.java
index 28bf347977..9596043f67 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/PhysicalProperties.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/PhysicalProperties.java
@@ -44,6 +44,9 @@ public class PhysicalProperties {
 
     public static PhysicalProperties STORAGE_GATHER = new PhysicalProperties(DistributionSpecStorageGather.INSTANCE);
 
+    public static PhysicalProperties MUST_SHUFFLE = new PhysicalProperties(DistributionSpecMustShuffle.INSTANCE);
+
+
     private final OrderSpec orderSpec;
 
     private final DistributionSpec distributionSpec;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/DataStreamSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/DataStreamSink.java
index 0f903d69d1..274cdd86f4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/DataStreamSink.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/DataStreamSink.java
@@ -20,19 +20,38 @@
 
 package org.apache.doris.planner;
 
+import org.apache.doris.analysis.BitmapFilterPredicate;
+import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.TupleDescriptor;
 import org.apache.doris.thrift.TDataSink;
 import org.apache.doris.thrift.TDataSinkType;
 import org.apache.doris.thrift.TDataStreamSink;
 import org.apache.doris.thrift.TExplainLevel;
 
+import com.google.common.collect.Lists;
+import org.springframework.util.CollectionUtils;
+
+import java.util.List;
+
 /**
  * Data sink that forwards data to an exchange node.
  */
 public class DataStreamSink extends DataSink {
-    private final PlanNodeId exchNodeId;
+
+    private PlanNodeId exchNodeId;
 
     private DataPartition outputPartition;
 
+    protected TupleDescriptor outputTupleDesc;
+
+    protected List<Expr> projections;
+
+    protected List<Expr> conjuncts = Lists.newArrayList();
+
+    public DataStreamSink() {
+
+    }
+
     public DataStreamSink(PlanNodeId exchNodeId) {
         this.exchNodeId = exchNodeId;
     }
@@ -42,23 +61,65 @@ public class DataStreamSink extends DataSink {
         return exchNodeId;
     }
 
+    public void setExchNodeId(PlanNodeId exchNodeId) {
+        this.exchNodeId = exchNodeId;
+    }
+
     @Override
     public DataPartition getOutputPartition() {
         return outputPartition;
     }
 
-    public void setPartition(DataPartition partition) {
-        outputPartition = partition;
+    public void setOutputPartition(DataPartition outputPartition) {
+        this.outputPartition = outputPartition;
+    }
+
+    public TupleDescriptor getOutputTupleDesc() {
+        return outputTupleDesc;
+    }
+
+    public void setOutputTupleDesc(TupleDescriptor outputTupleDesc) {
+        this.outputTupleDesc = outputTupleDesc;
+    }
+
+    public List<Expr> getProjections() {
+        return projections;
+    }
+
+    public void setProjections(List<Expr> projections) {
+        this.projections = projections;
+    }
+
+    public List<Expr> getConjuncts() {
+        return conjuncts;
+    }
+
+    public void setConjuncts(List<Expr> conjuncts) {
+        this.conjuncts = conjuncts;
+    }
+
+    public void addConjunct(Expr conjunct) {
+        this.conjuncts.add(conjunct);
     }
 
     @Override
     public String getExplainString(String prefix, TExplainLevel explainLevel) {
         StringBuilder strBuilder = new StringBuilder();
-        strBuilder.append(prefix + "STREAM DATA SINK\n");
-        strBuilder.append(prefix + "  EXCHANGE ID: " + exchNodeId + "\n");
+        strBuilder.append(prefix).append("STREAM DATA SINK\n");
+        strBuilder.append(prefix).append("  EXCHANGE ID: ").append(exchNodeId);
         if (outputPartition != null) {
-            strBuilder.append(prefix + "  " + outputPartition.getExplainString(explainLevel));
+            strBuilder.append("\n").append(prefix).append("  ").append(outputPartition.getExplainString(explainLevel));
+        }
+        if (!conjuncts.isEmpty()) {
+            Expr expr = PlanNode.convertConjunctsToAndCompoundPredicate(conjuncts);
+            strBuilder.append(prefix).append("  CONJUNCTS: ").append(expr.toSql());
+        }
+        if (!CollectionUtils.isEmpty(projections)) {
+            strBuilder.append("\n").append(prefix).append("  PROJECTIONS: ")
+                    .append(PlanNode.getExplainString(projections)).append("\n");
+            strBuilder.append(prefix).append("  PROJECTION TUPLE: ").append(outputTupleDesc.getId());
         }
+        strBuilder.append("\n");
         return strBuilder.toString();
     }
 
@@ -67,6 +128,19 @@ public class DataStreamSink extends DataSink {
         TDataSink result = new TDataSink(TDataSinkType.DATA_STREAM_SINK);
         TDataStreamSink tStreamSink =
                 new TDataStreamSink(exchNodeId.asInt(), outputPartition.toThrift());
+        for (Expr e : conjuncts) {
+            if  (!(e instanceof BitmapFilterPredicate)) {
+                tStreamSink.addToConjuncts(e.treeToThrift());
+            }
+        }
+        if (projections != null) {
+            for (Expr expr : projections) {
+                tStreamSink.addToOutputExprs(expr.treeToThrift());
+            }
+        }
+        if (outputTupleDesc != null) {
+            tStreamSink.setOutputTupleId(outputTupleDesc.getId().asInt());
+        }
         result.setStreamSink(tStreamSink);
         return result;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/ExchangeNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/ExchangeNode.java
index 30bf9eb45d..6694a05219 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/ExchangeNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/ExchangeNode.java
@@ -108,15 +108,21 @@ public class ExchangeNode extends PlanNode {
     public final void computeTupleIds() {
         PlanNode inputNode = getChild(0);
         TupleDescriptor outputTupleDesc = inputNode.getOutputTupleDesc();
+        updateTupleIds(outputTupleDesc);
+    }
+
+    public void updateTupleIds(TupleDescriptor outputTupleDesc) {
         if (outputTupleDesc != null) {
             tupleIds.clear();
             tupleIds.add(outputTupleDesc.getId());
+            tblRefIds.add(outputTupleDesc.getId());
+            nullableTupleIds.add(outputTupleDesc.getId());
         } else {
             clearTupleIds();
             tupleIds.addAll(getChild(0).getTupleIds());
+            tblRefIds.addAll(getChild(0).getTblRefIds());
+            nullableTupleIds.addAll(getChild(0).getNullableTupleIds());
         }
-        tblRefIds.addAll(getChild(0).getTblRefIds());
-        nullableTupleIds.addAll(getChild(0).getNullableTupleIds());
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/MultiCastPlanFragment.java b/fe/fe-core/src/main/java/org/apache/doris/planner/MultiCastPlanFragment.java
index 0d5b54b269..e52bbb0557 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/MultiCastPlanFragment.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/MultiCastPlanFragment.java
@@ -35,10 +35,11 @@ public class MultiCastPlanFragment extends PlanFragment {
         this.children.addAll(planFragment.getChildren());
     }
 
-    public List<ExchangeNode> getDestNodeList() {
-        return destNodeList;
+    public void addToDest(ExchangeNode exchangeNode) {
+        destNodeList.add(exchangeNode);
     }
 
+
     public List<PlanFragment> getDestFragmentList() {
         return destNodeList.stream().map(PlanNode::getFragment).collect(Collectors.toList());
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/PlanFragment.java b/fe/fe-core/src/main/java/org/apache/doris/planner/PlanFragment.java
index 54903beae5..64ac4c3051 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/PlanFragment.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/PlanFragment.java
@@ -256,7 +256,7 @@ public class PlanFragment extends TreeNode<PlanFragment> {
             Preconditions.checkState(sink == null);
             // we're streaming to an exchange node
             DataStreamSink streamSink = new DataStreamSink(destNode.getId());
-            streamSink.setPartition(outputPartition);
+            streamSink.setOutputPartition(outputPartition);
             streamSink.setFragment(this);
             sink = streamSink;
         } else {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/PlanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/PlanNode.java
index ab17c0acec..cf5d14cb5e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/PlanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/PlanNode.java
@@ -415,7 +415,7 @@ public abstract class PlanNode extends TreeNode<PlanNode> implements PlanStats {
         }
     }
 
-    protected Expr convertConjunctsToAndCompoundPredicate(List<Expr> conjuncts) {
+    public static Expr convertConjunctsToAndCompoundPredicate(List<Expr> conjuncts) {
         List<Expr> targetConjuncts = Lists.newArrayList(conjuncts);
         while (targetConjuncts.size() > 1) {
             List<Expr> newTargetConjuncts = Lists.newArrayList();
@@ -824,7 +824,7 @@ public abstract class PlanNode extends TreeNode<PlanNode> implements PlanStats {
         return output.toString();
     }
 
-    protected String getExplainString(List<? extends Expr> exprs) {
+    public static String getExplainString(List<? extends Expr> exprs) {
         if (exprs == null) {
             return "";
         }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 01/02: [feature](runtime_filter) MultiCastSender supporting runtime filter on be (#21304)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morrysnow pushed a commit to branch runtimefilter_multi_send
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 45c7f137db5fc17dd9361ac4a72ed943f7901e63
Author: Jerry Hu <mr...@gmail.com>
AuthorDate: Wed Jun 28 21:09:43 2023 +0800

    [feature](runtime_filter) MultiCastSender supporting runtime filter on be (#21304)
---
 .../exec/multi_cast_data_stream_source.cpp         | 61 +++++++++++++++++++---
 .../pipeline/exec/multi_cast_data_stream_source.h  | 23 ++++++--
 be/src/pipeline/pipeline_fragment_context.cpp      |  3 +-
 ...nsumer_node.cpp => runtime_filter_consumer.cpp} | 44 +++++++++-------
 ...r_consumer_node.h => runtime_filter_consumer.h} | 28 ++++++----
 be/src/vec/exec/scan/vscan_node.cpp                |  4 +-
 be/src/vec/exec/scan/vscan_node.h                  | 10 ++--
 be/src/vec/exec/vselect_node.cpp                   | 20 ++-----
 be/src/vec/exec/vselect_node.h                     |  8 +--
 gensrc/thrift/DataSinks.thrift                     | 12 +++++
 10 files changed, 146 insertions(+), 67 deletions(-)

diff --git a/be/src/pipeline/exec/multi_cast_data_stream_source.cpp b/be/src/pipeline/exec/multi_cast_data_stream_source.cpp
index 06211faf52..18ca62ea15 100644
--- a/be/src/pipeline/exec/multi_cast_data_stream_source.cpp
+++ b/be/src/pipeline/exec/multi_cast_data_stream_source.cpp
@@ -27,14 +27,16 @@
 namespace doris::pipeline {
 
 MultiCastDataStreamerSourceOperatorBuilder::MultiCastDataStreamerSourceOperatorBuilder(
-        int32_t id, const int consumer_id, std::shared_ptr<MultiCastDataStreamer>& data_streamer)
+        int32_t id, const int consumer_id, std::shared_ptr<MultiCastDataStreamer>& data_streamer,
+        const TDataStreamSink& sink)
         : OperatorBuilderBase(id, "MultiCastDataStreamerSourceOperator"),
           _consumer_id(consumer_id),
-          _multi_cast_data_streamer(data_streamer) {};
+          _multi_cast_data_streamer(data_streamer),
+          _t_data_stream_sink(sink) {}
 
 OperatorPtr MultiCastDataStreamerSourceOperatorBuilder::build_operator() {
-    return std::make_shared<MultiCastDataStreamerSourceOperator>(this, _consumer_id,
-                                                                 _multi_cast_data_streamer);
+    return std::make_shared<MultiCastDataStreamerSourceOperator>(
+            this, _consumer_id, _multi_cast_data_streamer, _t_data_stream_sink);
 }
 
 const RowDescriptor& MultiCastDataStreamerSourceOperatorBuilder::row_desc() {
@@ -43,10 +45,44 @@ const RowDescriptor& MultiCastDataStreamerSourceOperatorBuilder::row_desc() {
 
 MultiCastDataStreamerSourceOperator::MultiCastDataStreamerSourceOperator(
         OperatorBuilderBase* operator_builder, const int consumer_id,
-        std::shared_ptr<MultiCastDataStreamer>& data_streamer)
+        std::shared_ptr<MultiCastDataStreamer>& data_streamer, const TDataStreamSink& sink)
         : OperatorBase(operator_builder),
+          vectorized::RuntimeFilterConsumer(sink.dest_node_id, sink.runtime_filters,
+                                            data_streamer->row_desc(), _conjuncts),
           _consumer_id(consumer_id),
-          _multi_cast_data_streamer(data_streamer) {};
+          _multi_cast_data_streamer(data_streamer),
+          _t_data_stream_sink(sink) {}
+
+Status MultiCastDataStreamerSourceOperator::init(const TDataSink& tsink) {
+    RETURN_IF_ERROR(OperatorBase::init(tsink));
+    if (_t_data_stream_sink.__isset.output_exprs) {
+        RETURN_IF_ERROR(vectorized::VExpr::create_expr_trees(_t_data_stream_sink.output_exprs,
+                                                             _output_expr_contexts));
+    }
+
+    if (_t_data_stream_sink.__isset.conjuncts) {
+        RETURN_IF_ERROR(
+                vectorized::VExpr::create_expr_trees(_t_data_stream_sink.conjuncts, _conjuncts));
+    }
+
+    return Status::OK();
+}
+
+Status MultiCastDataStreamerSourceOperator::prepare(doris::RuntimeState* state) {
+    RETURN_IF_ERROR(vectorized::RuntimeFilterConsumer::init(state));
+    _register_runtime_filter();
+    RETURN_IF_ERROR(vectorized::VExpr::prepare(_conjuncts, state, row_desc()));
+    RETURN_IF_ERROR(vectorized::VExpr::prepare(_output_expr_contexts, state, row_desc()));
+    return Status::OK();
+}
+
+Status MultiCastDataStreamerSourceOperator::open(doris::RuntimeState* state) {
+    return _acquire_runtime_filter(state);
+}
+
+bool MultiCastDataStreamerSourceOperator::runtime_filters_are_ready_or_timeout() {
+    return vectorized::RuntimeFilterConsumer::runtime_filters_are_ready_or_timeout();
+}
 
 bool MultiCastDataStreamerSourceOperator::can_read() {
     return _multi_cast_data_streamer->can_read(_consumer_id);
@@ -56,6 +92,19 @@ Status MultiCastDataStreamerSourceOperator::get_block(RuntimeState* state, vecto
                                                       SourceState& source_state) {
     bool eos = false;
     _multi_cast_data_streamer->pull(_consumer_id, block, &eos);
+    if (!_output_expr_contexts.empty()) {
+        vectorized::Block output_block;
+        RETURN_IF_ERROR(vectorized::VExprContext::get_output_block_after_execute_exprs(
+                _output_expr_contexts, *block, &output_block));
+        materialize_block_inplace(output_block);
+        block->swap(output_block);
+    }
+
+    if (!_conjuncts.empty()) {
+        RETURN_IF_ERROR(
+                vectorized::VExprContext::filter_block(_conjuncts, block, block->columns()));
+    }
+
     if (eos) {
         source_state = SourceState::FINISHED;
     }
diff --git a/be/src/pipeline/exec/multi_cast_data_stream_source.h b/be/src/pipeline/exec/multi_cast_data_stream_source.h
index 15bd320b89..b2c7ca9e55 100644
--- a/be/src/pipeline/exec/multi_cast_data_stream_source.h
+++ b/be/src/pipeline/exec/multi_cast_data_stream_source.h
@@ -22,6 +22,7 @@
 
 #include "common/status.h"
 #include "operator.h"
+#include "vec/exec/runtime_filter_consumer.h"
 
 namespace doris {
 class ExecNode;
@@ -37,7 +38,8 @@ class MultiCastDataStreamer;
 class MultiCastDataStreamerSourceOperatorBuilder final : public OperatorBuilderBase {
 public:
     MultiCastDataStreamerSourceOperatorBuilder(int32_t id, const int consumer_id,
-                                               std::shared_ptr<MultiCastDataStreamer>&);
+                                               std::shared_ptr<MultiCastDataStreamer>&,
+                                               const TDataStreamSink&);
 
     bool is_source() const override { return true; }
 
@@ -48,20 +50,27 @@ public:
 private:
     const int _consumer_id;
     std::shared_ptr<MultiCastDataStreamer> _multi_cast_data_streamer;
+    TDataStreamSink _t_data_stream_sink;
 };
 
-class MultiCastDataStreamerSourceOperator final : public OperatorBase {
+class MultiCastDataStreamerSourceOperator final : public OperatorBase,
+                                                  public vectorized::RuntimeFilterConsumer {
 public:
     MultiCastDataStreamerSourceOperator(OperatorBuilderBase* operator_builder,
                                         const int consumer_id,
-                                        std::shared_ptr<MultiCastDataStreamer>& data_streamer);
+                                        std::shared_ptr<MultiCastDataStreamer>& data_streamer,
+                                        const TDataStreamSink& sink);
 
     Status get_block(RuntimeState* state, vectorized::Block* block,
                      SourceState& source_state) override;
 
-    Status prepare(RuntimeState* state) override { return Status::OK(); };
+    Status init(const TDataSink& tsink) override;
 
-    Status open(RuntimeState* state) override { return Status::OK(); };
+    Status prepare(RuntimeState* state) override;
+
+    Status open(RuntimeState* state) override;
+
+    bool runtime_filters_are_ready_or_timeout() override;
 
     Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) override {
         return Status::OK();
@@ -76,6 +85,10 @@ public:
 private:
     const int _consumer_id;
     std::shared_ptr<MultiCastDataStreamer> _multi_cast_data_streamer;
+    TDataStreamSink _t_data_stream_sink;
+
+    vectorized::VExprContextSPtrs _output_expr_contexts;
+    vectorized::VExprContextSPtrs _conjuncts;
 };
 
 } // namespace pipeline
diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp
index 7c24be8ded..a97128ee32 100644
--- a/be/src/pipeline/pipeline_fragment_context.cpp
+++ b/be/src/pipeline/pipeline_fragment_context.cpp
@@ -776,7 +776,8 @@ Status PipelineFragmentContext::_create_sink(int sender_id, const TDataSink& thr
             // 2. create and set the source operator of multi_cast_data_stream_source for new pipeline
             OperatorBuilderPtr source_op =
                     std::make_shared<MultiCastDataStreamerSourceOperatorBuilder>(
-                            next_operator_builder_id(), i, multi_cast_data_streamer);
+                            next_operator_builder_id(), i, multi_cast_data_streamer,
+                            thrift_sink.multi_cast_stream_sink.sinks[i]);
             new_pipeline->add_operator(source_op);
 
             // 3. create and set sink operator of data stream sender for new pipeline
diff --git a/be/src/vec/exec/runtime_filter_consumer_node.cpp b/be/src/vec/exec/runtime_filter_consumer.cpp
similarity index 77%
rename from be/src/vec/exec/runtime_filter_consumer_node.cpp
rename to be/src/vec/exec/runtime_filter_consumer.cpp
index dd631ce66e..ed74d7ecd0 100644
--- a/be/src/vec/exec/runtime_filter_consumer_node.cpp
+++ b/be/src/vec/exec/runtime_filter_consumer.cpp
@@ -15,22 +15,26 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "vec/exec/runtime_filter_consumer_node.h"
+#include "vec/exec/runtime_filter_consumer.h"
 
 namespace doris::vectorized {
 
-RuntimeFilterConsumerNode::RuntimeFilterConsumerNode(ObjectPool* pool, const TPlanNode& tnode,
-                                                     const DescriptorTbl& descs)
-        : ExecNode(pool, tnode, descs), _runtime_filter_descs(tnode.runtime_filters) {}
+RuntimeFilterConsumer::RuntimeFilterConsumer(const int32_t filter_id,
+                                             const std::vector<TRuntimeFilterDesc>& runtime_filters,
+                                             const RowDescriptor& row_descriptor,
+                                             VExprContextSPtrs& conjuncts)
+        : _filter_id(filter_id),
+          _runtime_filter_descs(runtime_filters),
+          _row_descriptor_ref(row_descriptor),
+          _conjuncts_ref(conjuncts) {}
 
-Status RuntimeFilterConsumerNode::init(const TPlanNode& tnode, RuntimeState* state) {
-    RETURN_IF_ERROR(ExecNode::init(tnode, state));
+Status RuntimeFilterConsumer::init(RuntimeState* state) {
     _state = state;
     RETURN_IF_ERROR(_register_runtime_filter());
     return Status::OK();
 }
 
-Status RuntimeFilterConsumerNode::_register_runtime_filter() {
+Status RuntimeFilterConsumer::_register_runtime_filter() {
     int filter_size = _runtime_filter_descs.size();
     _runtime_filter_ctxs.reserve(filter_size);
     _runtime_filter_ready_flag.reserve(filter_size);
@@ -43,14 +47,14 @@ Status RuntimeFilterConsumerNode::_register_runtime_filter() {
             // 1. All BE and FE has been upgraded (e.g. opt_remote_rf)
             // 2. This filter is bloom filter (only bloom filter should be used for merging)
             RETURN_IF_ERROR(_state->get_query_ctx()->runtime_filter_mgr()->register_consumer_filter(
-                    filter_desc, _state->query_options(), id(), false));
+                    filter_desc, _state->query_options(), _filter_id, false));
             RETURN_IF_ERROR(_state->get_query_ctx()->runtime_filter_mgr()->get_consume_filter(
-                    filter_desc.filter_id, id(), &runtime_filter));
+                    filter_desc.filter_id, _filter_id, &runtime_filter));
         } else {
             RETURN_IF_ERROR(_state->runtime_filter_mgr()->register_consumer_filter(
-                    filter_desc, _state->query_options(), id(), false));
+                    filter_desc, _state->query_options(), _filter_id, false));
             RETURN_IF_ERROR(_state->runtime_filter_mgr()->get_consume_filter(
-                    filter_desc.filter_id, id(), &runtime_filter));
+                    filter_desc.filter_id, _filter_id, &runtime_filter));
         }
         _runtime_filter_ctxs.emplace_back(runtime_filter);
         _runtime_filter_ready_flag.emplace_back(false);
@@ -58,7 +62,7 @@ Status RuntimeFilterConsumerNode::_register_runtime_filter() {
     return Status::OK();
 }
 
-bool RuntimeFilterConsumerNode::runtime_filters_are_ready_or_timeout() {
+bool RuntimeFilterConsumer::runtime_filters_are_ready_or_timeout() {
     if (!_blocked_by_rf) {
         return true;
     }
@@ -72,7 +76,7 @@ bool RuntimeFilterConsumerNode::runtime_filters_are_ready_or_timeout() {
     return true;
 }
 
-Status RuntimeFilterConsumerNode::_acquire_runtime_filter(bool wait) {
+Status RuntimeFilterConsumer::_acquire_runtime_filter(bool wait) {
     SCOPED_TIMER(_acquire_runtime_filter_timer);
     VExprSPtrs vexprs;
     for (size_t i = 0; i < _runtime_filter_descs.size(); ++i) {
@@ -101,23 +105,23 @@ Status RuntimeFilterConsumerNode::_acquire_runtime_filter(bool wait) {
     return Status::OK();
 }
 
-Status RuntimeFilterConsumerNode::_append_rf_into_conjuncts(const VExprSPtrs& vexprs) {
+Status RuntimeFilterConsumer::_append_rf_into_conjuncts(const VExprSPtrs& vexprs) {
     if (vexprs.empty()) {
         return Status::OK();
     }
 
     for (auto& expr : vexprs) {
         VExprContextSPtr conjunct = VExprContext::create_shared(expr);
-        RETURN_IF_ERROR(conjunct->prepare(_state, _row_descriptor));
+        RETURN_IF_ERROR(conjunct->prepare(_state, _row_descriptor_ref));
         RETURN_IF_ERROR(conjunct->open(_state));
         _rf_vexpr_set.insert(expr);
-        _conjuncts.emplace_back(conjunct);
+        _conjuncts_ref.emplace_back(conjunct);
     }
 
     return Status::OK();
 }
 
-Status RuntimeFilterConsumerNode::try_append_late_arrival_runtime_filter(int* arrived_rf_num) {
+Status RuntimeFilterConsumer::try_append_late_arrival_runtime_filter(int* arrived_rf_num) {
     if (_is_all_rf_applied) {
         *arrived_rf_num = _runtime_filter_descs.size();
         return Status::OK();
@@ -140,12 +144,12 @@ Status RuntimeFilterConsumerNode::try_append_late_arrival_runtime_filter(int* ar
             continue;
         } else if (_runtime_filter_ctxs[i].runtime_filter->is_ready()) {
             RETURN_IF_ERROR(_runtime_filter_ctxs[i].runtime_filter->get_prepared_exprs(
-                    &exprs, _row_descriptor, _state));
+                    &exprs, _row_descriptor_ref, _state));
             ++current_arrived_rf_num;
             _runtime_filter_ctxs[i].apply_mark = true;
         }
     }
-    // 2. Append unapplied runtime filters to vconjunct_ctx_ptr
+    // 2. Append unapplied runtime filters to _conjuncts
     if (!exprs.empty()) {
         RETURN_IF_ERROR(_append_rf_into_conjuncts(exprs));
     }
@@ -157,7 +161,7 @@ Status RuntimeFilterConsumerNode::try_append_late_arrival_runtime_filter(int* ar
     return Status::OK();
 }
 
-void RuntimeFilterConsumerNode::_prepare_rf_timer(RuntimeProfile* profile) {
+void RuntimeFilterConsumer::_prepare_rf_timer(RuntimeProfile* profile) {
     _acquire_runtime_filter_timer = ADD_TIMER(profile, "AcquireRuntimeFilterTime");
 }
 
diff --git a/be/src/vec/exec/runtime_filter_consumer_node.h b/be/src/vec/exec/runtime_filter_consumer.h
similarity index 81%
rename from be/src/vec/exec/runtime_filter_consumer_node.h
rename to be/src/vec/exec/runtime_filter_consumer.h
index 518e0e865c..18e92abc90 100644
--- a/be/src/vec/exec/runtime_filter_consumer_node.h
+++ b/be/src/vec/exec/runtime_filter_consumer.h
@@ -22,14 +22,16 @@
 
 namespace doris::vectorized {
 
-class RuntimeFilterConsumerNode : public ExecNode {
+class RuntimeFilterConsumer {
 public:
-    RuntimeFilterConsumerNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs);
-    ~RuntimeFilterConsumerNode() override = default;
+    RuntimeFilterConsumer(const int32_t filter_id,
+                          const std::vector<TRuntimeFilterDesc>& runtime_filters,
+                          const RowDescriptor& row_descriptor, VExprContextSPtrs& conjuncts);
+    ~RuntimeFilterConsumer() = default;
 
-    Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override;
+    Status init(RuntimeState* state);
 
-    // Try append late arrived runtime filters.
+    // Try to append late arrived runtime filters.
     // Return num of filters which are applied already.
     Status try_append_late_arrival_runtime_filter(int* arrived_rf_num);
 
@@ -54,15 +56,23 @@ protected:
         IRuntimeFilter* runtime_filter;
     };
 
-    RuntimeState* _state;
-
     std::vector<RuntimeFilterContext> _runtime_filter_ctxs;
-
-    std::vector<TRuntimeFilterDesc> _runtime_filter_descs;
     // Set to true if the runtime filter is ready.
     std::vector<bool> _runtime_filter_ready_flag;
     doris::Mutex _rf_locks;
     phmap::flat_hash_set<VExprSPtr> _rf_vexpr_set;
+
+private:
+    RuntimeState* _state;
+
+    int32_t _filter_id;
+
+    std::vector<TRuntimeFilterDesc> _runtime_filter_descs;
+
+    const RowDescriptor& _row_descriptor_ref;
+
+    VExprContextSPtrs& _conjuncts_ref;
+
     // True means all runtime filters are applied to scanners
     bool _is_all_rf_applied = true;
     bool _blocked_by_rf = false;
diff --git a/be/src/vec/exec/scan/vscan_node.cpp b/be/src/vec/exec/scan/vscan_node.cpp
index 2af6fc87c5..e85775ffa1 100644
--- a/be/src/vec/exec/scan/vscan_node.cpp
+++ b/be/src/vec/exec/scan/vscan_node.cpp
@@ -94,7 +94,9 @@ static bool ignore_cast(SlotDescriptor* slot, VExpr* expr) {
 }
 
 Status VScanNode::init(const TPlanNode& tnode, RuntimeState* state) {
-    RETURN_IF_ERROR(RuntimeFilterConsumerNode::init(tnode, state));
+    RETURN_IF_ERROR(ExecNode::init(tnode, state));
+    RETURN_IF_ERROR(RuntimeFilterConsumer::init(state));
+    _state = state;
     _is_pipeline_scan = state->enable_pipeline_exec();
 
     const TQueryOptions& query_options = state->query_options();
diff --git a/be/src/vec/exec/scan/vscan_node.h b/be/src/vec/exec/scan/vscan_node.h
index 112ca47b54..52284ffe95 100644
--- a/be/src/vec/exec/scan/vscan_node.h
+++ b/be/src/vec/exec/scan/vscan_node.h
@@ -43,7 +43,7 @@
 #include "runtime/runtime_state.h"
 #include "util/lock.h"
 #include "util/runtime_profile.h"
-#include "vec/exec/runtime_filter_consumer_node.h"
+#include "vec/exec/runtime_filter_consumer.h"
 #include "vec/exec/scan/scanner_context.h"
 #include "vec/exec/scan/vscanner.h"
 #include "vec/runtime/shared_scanner_controller.h"
@@ -88,10 +88,12 @@ struct FilterPredicates {
     std::vector<std::pair<std::string, std::shared_ptr<HybridSetBase>>> in_filters;
 };
 
-class VScanNode : public RuntimeFilterConsumerNode {
+class VScanNode : public ExecNode, public RuntimeFilterConsumer {
 public:
     VScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs)
-            : RuntimeFilterConsumerNode(pool, tnode, descs) {
+            : ExecNode(pool, tnode, descs),
+              RuntimeFilterConsumer(id(), tnode.runtime_filters, ExecNode::_row_descriptor,
+                                    ExecNode::_conjuncts) {
         if (!tnode.__isset.conjuncts || tnode.conjuncts.empty()) {
             // Which means the request could be fullfilled in a single segment iterator request.
             if (tnode.limit > 0 && tnode.limit < 1024) {
@@ -304,6 +306,8 @@ protected:
     VExprContextSPtrs _stale_expr_ctxs;
     VExprContextSPtrs _common_expr_ctxs_push_down;
 
+    RuntimeState* _state;
+
     // If sort info is set, push limit to each scanner;
     int64_t _limit_per_scanner = -1;
 
diff --git a/be/src/vec/exec/vselect_node.cpp b/be/src/vec/exec/vselect_node.cpp
index 626fd5ce96..ee1628cd19 100644
--- a/be/src/vec/exec/vselect_node.cpp
+++ b/be/src/vec/exec/vselect_node.cpp
@@ -37,34 +37,22 @@ class TPlanNode;
 namespace vectorized {
 
 VSelectNode::VSelectNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs)
-        : RuntimeFilterConsumerNode(pool, tnode, descs), _child_eos(false) {}
+        : ExecNode(pool, tnode, descs), _child_eos(false) {}
 
 Status VSelectNode::init(const TPlanNode& tnode, RuntimeState* state) {
-    return RuntimeFilterConsumerNode::init(tnode, state);
+    return ExecNode::init(tnode, state);
 }
 
 Status VSelectNode::prepare(RuntimeState* state) {
-    return RuntimeFilterConsumerNode::prepare(state);
+    return ExecNode::prepare(state);
 }
 
 Status VSelectNode::open(RuntimeState* state) {
-    RETURN_IF_ERROR(RuntimeFilterConsumerNode::open(state));
+    RETURN_IF_ERROR(ExecNode::open(state));
     RETURN_IF_ERROR(child(0)->open(state));
     return Status::OK();
 }
 
-Status VSelectNode::alloc_resource(RuntimeState* state) {
-    if (_opened) {
-        return Status::OK();
-    }
-
-    RETURN_IF_ERROR(RuntimeFilterConsumerNode::alloc_resource(state));
-    RETURN_IF_ERROR(_acquire_runtime_filter());
-    RETURN_IF_CANCELLED(state);
-    _opened = true;
-    return Status::OK();
-}
-
 Status VSelectNode::get_next(RuntimeState* state, vectorized::Block* block, bool* eos) {
     SCOPED_TIMER(_runtime_profile->total_time_counter());
     RETURN_IF_CANCELLED(state);
diff --git a/be/src/vec/exec/vselect_node.h b/be/src/vec/exec/vselect_node.h
index 140009e4b3..1b425e80af 100644
--- a/be/src/vec/exec/vselect_node.h
+++ b/be/src/vec/exec/vselect_node.h
@@ -17,7 +17,7 @@
 
 #pragma once
 #include "common/status.h"
-#include "vec/exec/runtime_filter_consumer_node.h"
+#include "exec/exec_node.h"
 
 namespace doris {
 class DescriptorTbl;
@@ -28,7 +28,7 @@ class TPlanNode;
 namespace vectorized {
 class Block;
 
-class VSelectNode final : public RuntimeFilterConsumerNode {
+class VSelectNode final : public ExecNode {
 public:
     VSelectNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs);
     Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override;
@@ -37,13 +37,9 @@ public:
     Status get_next(RuntimeState* state, vectorized::Block* block, bool* eos) override;
     Status close(RuntimeState* state) override;
     Status pull(RuntimeState* state, vectorized::Block* output_block, bool* eos) override;
-
-    Status alloc_resource(RuntimeState* state) override;
-
 private:
     // true if last get_next() call on child signalled eos
     bool _child_eos;
-    bool _opened = false;
 };
 } // namespace vectorized
 } // namespace doris
\ No newline at end of file
diff --git a/gensrc/thrift/DataSinks.thrift b/gensrc/thrift/DataSinks.thrift
index af5d4d26a3..c78a7900a9 100644
--- a/gensrc/thrift/DataSinks.thrift
+++ b/gensrc/thrift/DataSinks.thrift
@@ -153,6 +153,18 @@ struct TDataStreamSink {
   2: required Partitions.TDataPartition output_partition
 
   3: optional bool ignore_not_found
+
+    // per-destination projections
+    4: optional list<Exprs.TExpr> output_exprs
+
+    // project output tuple id
+    5: optional Types.TTupleId output_tuple_id
+
+    // per-destination filters
+    6: optional list<Exprs.TExpr> conjuncts
+
+    // per-destination runtime filters
+    7: optional list<PlanNodes.TRuntimeFilterDesc> runtime_filters
 }
 
 struct TMultiCastDataStreamSink {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org