Skip to content

Commit

Permalink
[refactor](nereids) New distribute planner (apache#36531)
Browse files Browse the repository at this point in the history
## Proposed changes
The legacy coordinator act not only scheduler but also distribute planner. The code is so complex to understand, and hard to extend, and exist many limitations.

This pr extract and refine the computation of degree of parallel(dop) to a new DistributePlanner and resolve the limitations.


## How to use this function
This function only use for nereids + pipelinex, and current only support query statement, and non cloud mode.
Open this session variables to use this function:
```sql
set enable_nereids_distribute_planner=true; -- default is false
set enable_nereids_planner=true;  -- default is true
```

## Core process and concepts
```
                                                                                                                              
 ┌──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ 
 │                                                                                                                          │ 
 │             ┌──────────────┐         ┌───────────────┐          ┌───────────────────┐        ┌─────────────────────────┐ │ 
 │  Translate  │              │  Typed  │               │  Assign  │                   │  Wrap  │                         │ │ 
 │ ──────────► │ PlanFragment │ ──────► │ UnassignedJob │ ───────► │ StaticAssignedJob │ ─────► │ PipelineDistributedPlan │ │ 
 │             │              │         │               │          │                   │        │                         │ │ 
 │             └──────────────┘         └───────────────┘          └───────────────────┘        └─────────────────────────┘ │ 
 │                                                                                                                          │ 
 └──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ 
                                                                                                                              
 │                                                                                                                          │ 
 │                                                                                                                          │ 
 └──────────────────────────────────────────────────┐                 ┌─────────────────────────────────────────────────────┘ 
                                                    │                 │                                                       
                                                    │                 │                                                       
                                                                                                                              
                      ┌──────────────┐              ┌─────────────────┐         ┌───────────────────┐                         
                      │              │  Distribute  │                 │  AdHoc  │                   │                         
                      │ PhysicalPlan │ ───────────► │ DistributedPlan │ ──────► │ PipelineScheduler │                         
                      │              │              │                 │         │                   │                         
                      └──────────────┘              └─────────────────┘         └───────────────────┘                         
                                                                                                                              
```

DistributePlanner is a new planner to compute dop and generate instances, it consume PlanFragment and do this tasks
1. Use PlanFragment to generate `UnassignedJob`, it's a **Typed Fragment**, decided how to calculate dop and how to select the datasource, but this fragment not yet assigned some backends and datasources. These are some unassignedJobs: UnassignedScanSingleOlapTableJob, UnassignedScanBucketOlapTableJob, UnassignedShuffleJob, UnassignedQueryConstantJob. Keep UnassignedJob different can decoupling unrelated logic, and easy to extend: just and a new type of UnassignedJob.
2. Use UnassignedJob to select datasource, compute dop, and generate `AssignedJob`, means a instance, which already assigned datasource and backend. There are StaticAssignedJob and LocalShuffleAssignedJob, we will add DynamicAssignedJob when support StageScheduler and adaptive query execution
3. Wrap PlanFragment, UnassignedJob and AssignedJob to `PipelineDistributedPlan`, the coordinator will consume the DistributedPlan and translate to TPlan and schedule instances


## Resolve limitations
**1. left table shuffle to right table**
if right table has distribution which distribute by `storage hash`, and left table has distribution which distribute by `compute hash`, we can shuffle left to right by `storage hash` to do shuffle bucket join, and keep right side not move.

```sql
select *
from
(
  select id2
  from test_shuffle_left
  group by id2
) a
inner join [shuffle]
test_shuffle_left b
on a.id2=b.id;

| PhysicalResultSink[288] ( outputExprs=[id2#1, id#2, id2#3] )                                                                                                                                ...
| +--PhysicalHashJoin[285]@4 ( type=INNER_JOIN, stats=3, hashCondition=[(id2#1 = id#2)], otherCondition=[], markCondition=[], hint=[shuffle] )                                                ...
|    |--PhysicalDistribute[281]@2 ( stats=1.5, distributionSpec=DistributionSpecHash ( orderedShuffledColumns=[1], shuffleType=STORAGE_BUCKETED, tableId=-1, selectedIndexId=-1, partitionIds=...
|    |  +--PhysicalHashAggregate[278]@2 ( aggPhase=GLOBAL, aggMode=BUFFER_TO_RESULT, maybeUseStreaming=false, groupByExpr=[id2#1], outputExpr=[id2#1], partitionExpr=Optional[[id2#1]], requir...
|    |     +--PhysicalDistribute[275]@7 ( stats=1.5, distributionSpec=DistributionSpecHash ( orderedShuffledColumns=[1], shuffleType=EXECUTION_BUCKETED, tableId=-1, selectedIndexId=-1, parti...
|    |        +--PhysicalHashAggregate[272]@7 ( aggPhase=LOCAL, aggMode=INPUT_TO_BUFFER, maybeUseStreaming=true, groupByExpr=[id2#1], outputExpr=[id2#1], partitionExpr=Optional[[id2#1]], req...
|    |           +--PhysicalProject[269]@1 ( stats=3, projects=[id2#1] )                                                                                                                      ...
|    |              +--PhysicalOlapScan[test_shuffle_left]@0 ( stats=3 )                                                                                                                      ...
|    +--PhysicalOlapScan[test_shuffle_left]@3 ( stats=3 )
```

**2. support colocate union numbers function**
support use one instance to union/join numbers, note this plan no any PhysicalDistribute plan:
```sql
explain physical plan
select * from numbers('number'='3')a
union all
select * from numbers('number'='4')b

PhysicalResultSink[98] ( outputExprs=[number#2] )
+--PhysicalUnion@ ( qualifier=ALL, outputs=[number#2], regularChildrenOutputs=[[number#0], [number#1]], constantExprsList=[], stats=7 )
   |--PhysicalTVFRelation ( qualified=NumbersTableValuedFunction, output=[number#0], function=numbers('number' = '3') )
   +--PhysicalTVFRelation ( qualified=NumbersTableValuedFunction, output=[number#1], function=numbers('number' = '4') )
```

**3. support bucket prune with right outer bucket shuffle join**
left table prune some buckets, say [bucket 1, bucket 3]

we should process the right bucket shuffle join like this
```
[
  (left bucket 1) right outer join (exchange right table which should process by bucket 1),
  (empty bucket) right outer join (exchange right table which should process by bucket 2),
  (left bucket 3) right outer join (exchange right table which should process by bucket 3)
]
```
the left bucket 2 is pruned, so right table can not shuffle to left, because the left instance not exists, so bucket 2 will return empty rows and wrong.

new DistributePlanner can fill up this instance.

the case:
```sql
explain physical plan
SELECT * FROM
(select * from test_outer_join1 where c0 =1)a
RIGHT OUTER JOIN
(select * from test_outer_join2)b
ON a.c0 = b.c0
```

### New feature

add an explain statement to show distributed plans
```sql
explain distributed plan select ...
```

for example, you can use this function to check how many instances generated, how many bytes the instance will scan, which backend will process the instance:
```sql
MySQL root@127.0.0.1:test> explain distributed plan select * from test_shuffle_left2 a join [shuffle] test_shuffle_left2 b on a.id2=b.id;
Explain String(Nereids Planner)
-------------------------------------------------------------------------------------------------------
PipelineDistributedPlan(
  id: 0,
  parallel: 2,
  fragmentJob: UnassignedScanSingleOlapTableJob,
  fragment: {
    OUTPUT EXPRS:
      id[apache#8]
      id2[apache#9]
      id[apache#10]
      id2[apache#11]
    PARTITION: HASH_PARTITIONED: id2[#3]

    HAS_COLO_PLAN_NODE: false

    VRESULT SINK
       MYSQL_PROTOCAL

    3:VHASH JOIN(152)
    |  join op: INNER JOIN(PARTITIONED)[]
    |  equal join conjunct: (id2[#3] = id[#0])
    |  cardinality=3
    |  vec output tuple id: 3
    |  output tuple id: 3
    |  vIntermediate tuple ids: 2
    |  hash output slot ids: 0 1 2 3
    |  isMarkJoin: false
    |  final projections: id[#4], id2[#5], id[#6], id2[apache#7]
    |  final project output tuple id: 3
    |  distribute expr lists: id2[#3]
    |  distribute expr lists: id[#0]
    |  tuple ids: 1 0
    |
    |----0:VOlapScanNode(149)
    |       TABLE: test.test_shuffle_left2(test_shuffle_left2), PREAGGREGATION: ON
    |       partitions=1/1 (test_shuffle_left2)
    |       tablets=10/10, tabletList=22038,22040,22042 ...
    |       cardinality=3, avgRowSize=0.0, numNodes=1
    |       pushAggOp=NONE
    |       tuple ids: 0
    |
    2:VEXCHANGE
       offset: 0
       distribute expr lists: id[#2]
       tuple ids: 1
  },
  instanceJobs: [
    LocalShuffleAssignedJob(
      index: 0,
      worker: BackendWorker(id: 10095, address: 192.168.126.1:9050),
      shareScanIndex: 0,
      scanSource: [
        {
          scanNode: OlapScanNode{id=0, tid=0, tblName=test_shuffle_left2, keyRanges=, preds= limit=-1},
          scanRanges: ScanRanges(bytes: 400, ranges: [
            tablet 22038, bytes: 0,
            tablet 22042, bytes: 0,
            tablet 22046, bytes: 0,
            tablet 22050, bytes: 400,
            tablet 22054, bytes: 0
          ])
        }
      ]
    ),
    LocalShuffleAssignedJob(
      index: 1,
      worker: BackendWorker(id: 10096, address: 192.168.126.2:9051),
      shareScanIndex: 1,
      scanSource: [
        {
          scanNode: OlapScanNode{id=0, tid=0, tblName=test_shuffle_left2, keyRanges=, preds= limit=-1},
          scanRanges: ScanRanges(bytes: 796, ranges: [
            tablet 22040, bytes: 397,
            tablet 22044, bytes: 0,
            tablet 22048, bytes: 399,
            tablet 22052, bytes: 0,
            tablet 22056, bytes: 0
          ])
        }
      ]
    )
  ]
)
PipelineDistributedPlan(
  id: 1,
  parallel: 2,
  fragmentJob: UnassignedScanSingleOlapTableJob,
  fragment: {
    PARTITION: HASH_PARTITIONED: id[#2]

    HAS_COLO_PLAN_NODE: false

    STREAM DATA SINK
      EXCHANGE ID: 02
      HASH_PARTITIONED: id2[#3]

    1:VOlapScanNode(145)
       TABLE: test.test_shuffle_left2(test_shuffle_left2), PREAGGREGATION: ON
       partitions=1/1 (test_shuffle_left2)
       tablets=10/10, tabletList=22038,22040,22042 ...
       cardinality=3, avgRowSize=0.0, numNodes=1
       pushAggOp=NONE
       tuple ids: 1
  },
  instanceJobs: [
    LocalShuffleAssignedJob(
      index: 0,
      worker: BackendWorker(id: 10095, address: 192.168.126.1:9050),
      shareScanIndex: 0,
      scanSource: [
        {
          scanNode: OlapScanNode{id=1, tid=1, tblName=test_shuffle_left2, keyRanges=, preds= limit=-1},
          scanRanges: ScanRanges(bytes: 400, ranges: [
            tablet 22038, bytes: 0,
            tablet 22042, bytes: 0,
            tablet 22046, bytes: 0,
            tablet 22050, bytes: 400,
            tablet 22054, bytes: 0
          ])
        }
      ]
    ),
    LocalShuffleAssignedJob(
      index: 1,
      worker: BackendWorker(id: 10096, address: 192.168.126.2:9051),
      shareScanIndex: 1,
      scanSource: [
        {
          scanNode: OlapScanNode{id=1, tid=1, tblName=test_shuffle_left2, keyRanges=, preds= limit=-1},
          scanRanges: ScanRanges(bytes: 796, ranges: [
            tablet 22040, bytes: 397,
            tablet 22044, bytes: 0,
            tablet 22048, bytes: 399,
            tablet 22052, bytes: 0,
            tablet 22056, bytes: 0
          ])
        }
      ]
    )
  ]
)



Hint log:
Used: [shuffle]_2
UnUsed:
SyntaxError:
```

## TODO
1. extract PipelineScheduler from Coordinator
2. move this framework into cascades and compute cost by dop
3. support StageScheduler, adaptive query execution and DynamicAssignedJob
  • Loading branch information
924060929 authored and dataroaring committed Jun 30, 2024
1 parent f04c185 commit 5b68de5
Show file tree
Hide file tree
Showing 76 changed files with 4,438 additions and 77 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -262,6 +262,7 @@ planType
| OPTIMIZED | PHYSICAL // same type
| SHAPE
| MEMO
| DISTRIBUTED
| ALL // default type
;

Expand Down
7 changes: 6 additions & 1 deletion fe/fe-core/src/main/java/org/apache/doris/common/Id.java
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
/**
* Integer ids that cannot accidentally be compared with ints.
*/
public class Id<IdType extends Id<IdType>> {
public class Id<IdType extends Id<IdType>> implements Comparable<Id<IdType>> {
protected final int id;

public Id(int id) {
Expand Down Expand Up @@ -62,4 +62,9 @@ public ArrayList<IdType> asList() {
public String toString() {
return Integer.toString(id);
}

@Override
public int compareTo(Id<IdType> idTypeId) {
return id - idTypeId.id;
}
}
25 changes: 25 additions & 0 deletions fe/fe-core/src/main/java/org/apache/doris/common/TreeNode.java
Original file line number Diff line number Diff line change
Expand Up @@ -249,4 +249,29 @@ public void foreach(ThrowingConsumer<TreeNode<NodeType>> func) throws AnalysisEx
child.foreach(func);
}
}

/** anyMatch */
public boolean anyMatch(Predicate<TreeNode<? extends NodeType>> func) {
if (func.apply(this)) {
return true;
}

for (NodeType child : children) {
if (child.anyMatch(func)) {
return true;
}
}
return false;
}

/** foreachDown */
public void foreachDown(Predicate<TreeNode<NodeType>> visitor) {
if (!visitor.test(this)) {
return;
}

for (TreeNode<NodeType> child : getChildren()) {
child.foreachDown(visitor);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@
import org.apache.doris.common.util.ProfileManager;
import org.apache.doris.common.util.RuntimeProfile;
import org.apache.doris.nereids.NereidsPlanner;
import org.apache.doris.nereids.trees.plans.distribute.DistributedPlan;
import org.apache.doris.nereids.trees.plans.distribute.FragmentIdMapping;
import org.apache.doris.nereids.trees.plans.physical.PhysicalRelation;
import org.apache.doris.planner.Planner;

Expand Down Expand Up @@ -108,6 +110,14 @@ public synchronized void updateSummary(long startTime, Map<String, String> summa
}
summaryInfo.put(SummaryProfile.PHYSICAL_PLAN,
builder.toString().replace("\n", "\n "));

FragmentIdMapping<DistributedPlan> distributedPlans = nereidsPlanner.getDistributedPlans();
if (distributedPlans != null) {
summaryInfo.put(SummaryProfile.DISTRIBUTED_PLAN,
DistributedPlan.toString(Lists.newArrayList(distributedPlans.values()))
.replace("\n", "\n ")
);
}
}
summaryProfile.update(summaryInfo);
for (ExecutionProfile executionProfile : executionProfiles) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@ public class SummaryProfile {
public static final String TRACE_ID = "Trace ID";
public static final String WORKLOAD_GROUP = "Workload Group";
public static final String PHYSICAL_PLAN = "Physical Plan";
public static final String DISTRIBUTED_PLAN = "Distributed Plan";
// Execution Summary
public static final String EXECUTION_SUMMARY_PROFILE_NAME = "Execution Summary";
public static final String ANALYSIS_TIME = "Analysis Time";
Expand Down Expand Up @@ -86,6 +87,7 @@ public class SummaryProfile {
public static final String NEREIDS_REWRITE_TIME = "Nereids Rewrite Time";
public static final String NEREIDS_OPTIMIZE_TIME = "Nereids Optimize Time";
public static final String NEREIDS_TRANSLATE_TIME = "Nereids Translate Time";
public static final String NEREIDS_DISTRIBUTE_TIME = "Nereids Distribute Time";

public static final String FRAGMENT_COMPRESSED_SIZE = "Fragment Compressed Size";
public static final String FRAGMENT_RPC_COUNT = "Fragment RPC Count";
Expand All @@ -109,6 +111,7 @@ public class SummaryProfile {
public static final ImmutableList<String> SUMMARY_KEYS = new ImmutableList.Builder<String>()
.addAll(SUMMARY_CAPTIONS)
.add(PHYSICAL_PLAN)
.add(DISTRIBUTED_PLAN)
.build();

// The display order of execution summary items.
Expand Down Expand Up @@ -199,6 +202,7 @@ public class SummaryProfile {
private long nereidsRewriteFinishTime = -1;
private long nereidsOptimizeFinishTime = -1;
private long nereidsTranslateFinishTime = -1;
private long nereidsDistributeFinishTime = -1;
// timestamp of query begin
private long queryBeginTime = -1;
// Analysis end time
Expand Down Expand Up @@ -315,6 +319,7 @@ private void updateExecutionSummaryProfile() {
executionSummaryProfile.addInfoString(NEREIDS_REWRITE_TIME, getPrettyNereidsRewriteTime());
executionSummaryProfile.addInfoString(NEREIDS_OPTIMIZE_TIME, getPrettyNereidsOptimizeTime());
executionSummaryProfile.addInfoString(NEREIDS_TRANSLATE_TIME, getPrettyNereidsTranslateTime());
executionSummaryProfile.addInfoString(NEREIDS_DISTRIBUTE_TIME, getPrettyNereidsDistributeTime());
executionSummaryProfile.addInfoString(ANALYSIS_TIME,
getPrettyTime(queryAnalysisFinishTime, queryBeginTime, TUnit.TIME_MS));
executionSummaryProfile.addInfoString(PLAN_TIME,
Expand Down Expand Up @@ -419,6 +424,10 @@ public void setNereidsTranslateTime() {
this.nereidsTranslateFinishTime = TimeUtils.getStartTimeMs();
}

public void setNereidsDistributeTime() {
this.nereidsDistributeFinishTime = TimeUtils.getStartTimeMs();
}

public void setQueryBeginTime() {
this.queryBeginTime = TimeUtils.getStartTimeMs();
}
Expand Down Expand Up @@ -654,6 +663,10 @@ public String getPrettyNereidsTranslateTime() {
return getPrettyTime(nereidsTranslateFinishTime, nereidsOptimizeFinishTime, TUnit.TIME_MS);
}

public String getPrettyNereidsDistributeTime() {
return getPrettyTime(nereidsDistributeFinishTime, nereidsTranslateFinishTime, TUnit.TIME_MS);
}

private String getPrettyGetPartitionVersionTime() {
if (getPartitionVersionTime == 0) {
return "N/A";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,9 @@
import org.apache.doris.nereids.trees.expressions.literal.Literal;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel;
import org.apache.doris.nereids.trees.plans.distribute.DistributePlanner;
import org.apache.doris.nereids.trees.plans.distribute.DistributedPlan;
import org.apache.doris.nereids.trees.plans.distribute.FragmentIdMapping;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import org.apache.doris.nereids.trees.plans.logical.LogicalSqlCache;
import org.apache.doris.nereids.trees.plans.physical.PhysicalEmptyRelation;
Expand All @@ -70,6 +73,7 @@
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.qe.ResultSet;
import org.apache.doris.qe.ResultSetMetaData;
import org.apache.doris.qe.SessionVariable;
import org.apache.doris.qe.cache.CacheAnalyzer;

import com.google.common.annotations.VisibleForTesting;
Expand Down Expand Up @@ -102,6 +106,7 @@ public class NereidsPlanner extends Planner {
private Plan rewrittenPlan;
private Plan optimizedPlan;
private PhysicalPlan physicalPlan;
private FragmentIdMapping<DistributedPlan> distributedPlans;
// The cost of optimized plan
private double cost = 0;
private LogicalPlanAdapter logicalPlanAdapter;
Expand Down Expand Up @@ -130,17 +135,18 @@ public void plan(StatementBase queryStmt, org.apache.doris.thrift.TQueryOptions
LogicalPlan parsedPlan = logicalPlanAdapter.getLogicalPlan();
NereidsTracer.logImportantTime("EndParsePlan");
setParsedPlan(parsedPlan);

PhysicalProperties requireProperties = buildInitRequireProperties();
statementContext.getStopwatch().start();
boolean showPlanProcess = showPlanProcess(queryStmt.getExplainOptions());
Plan resultPlan = plan(parsedPlan, requireProperties, explainLevel, showPlanProcess);
statementContext.getStopwatch().stop();
setOptimizedPlan(resultPlan);
if (explainLevel.isPlanLevel) {
return;

if (resultPlan instanceof PhysicalPlan) {
physicalPlan = (PhysicalPlan) resultPlan;
distribute(physicalPlan, explainLevel);
}
physicalPlan = (PhysicalPlan) resultPlan;
translate(physicalPlan);
}

@VisibleForTesting
Expand Down Expand Up @@ -315,7 +321,7 @@ private void optimize() {
}
}

private void translate(PhysicalPlan resultPlan) throws UserException {
private void splitFragments(PhysicalPlan resultPlan) throws UserException {
if (resultPlan instanceof PhysicalSqlCache) {
return;
}
Expand Down Expand Up @@ -360,6 +366,27 @@ private void translate(PhysicalPlan resultPlan) throws UserException {
ScanNode.setVisibleVersionForOlapScanNodes(getScanNodes());
}

private void distribute(PhysicalPlan physicalPlan, ExplainLevel explainLevel) throws UserException {
boolean canUseNereidsDistributePlanner = SessionVariable.canUseNereidsDistributePlanner();
if ((!canUseNereidsDistributePlanner && explainLevel.isPlanLevel)) {
return;
} else if ((canUseNereidsDistributePlanner && explainLevel.isPlanLevel
&& (explainLevel != ExplainLevel.ALL_PLAN && explainLevel != ExplainLevel.DISTRIBUTED_PLAN))) {
return;
}

splitFragments(physicalPlan);

if (!canUseNereidsDistributePlanner) {
return;
}

distributedPlans = new DistributePlanner(fragments).plan();
if (statementContext.getConnectContext().getExecutor() != null) {
statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsDistributeTime();
}
}

private PhysicalPlan postProcess(PhysicalPlan physicalPlan) {
return new PlanPostProcessors(cascadesContext).process(physicalPlan);
}
Expand Down Expand Up @@ -498,6 +525,17 @@ public String getExplainString(ExplainOptions explainOptions) {
+ "\n\n========== MATERIALIZATIONS ==========\n"
+ materializationStringBuilder;
break;
case DISTRIBUTED_PLAN:
StringBuilder distributedPlanStringBuilder = new StringBuilder();

distributedPlanStringBuilder.append("========== DISTRIBUTED PLAN ==========\n");
if (distributedPlans == null || distributedPlans.isEmpty()) {
plan = "Distributed plan not generated, please set enable_nereids_distribute_planner "
+ "and enable_pipeline_x_engine to true";
} else {
plan += DistributedPlan.toString(Lists.newArrayList(distributedPlans.values())) + "\n\n";
}
break;
case ALL_PLAN:
plan = "========== PARSED PLAN "
+ getTimeMetricString(SummaryProfile::getPrettyParseSqlTime) + " ==========\n"
Expand All @@ -510,7 +548,13 @@ public String getExplainString(ExplainOptions explainOptions) {
+ rewrittenPlan.treeString() + "\n\n"
+ "========== OPTIMIZED PLAN "
+ getTimeMetricString(SummaryProfile::getPrettyNereidsOptimizeTime) + " ==========\n"
+ optimizedPlan.treeString();
+ optimizedPlan.treeString() + "\n\n";

if (distributedPlans != null && !distributedPlans.isEmpty()) {
plan += "========== DISTRIBUTED PLAN "
+ getTimeMetricString(SummaryProfile::getPrettyNereidsDistributeTime) + " ==========\n";
plan += DistributedPlan.toString(Lists.newArrayList(distributedPlans.values())) + "\n\n";
}
break;
default:
plan = super.getExplainString(explainOptions)
Expand Down Expand Up @@ -681,6 +725,10 @@ public PhysicalPlan getPhysicalPlan() {
return physicalPlan;
}

public FragmentIdMapping<DistributedPlan> getDistributedPlans() {
return distributedPlans;
}

public LogicalPlanAdapter getLogicalPlanAdapter() {
return logicalPlanAdapter;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3451,6 +3451,9 @@ private ExplainLevel parseExplainPlanType(PlanTypeContext planTypeContext) {
if (planTypeContext.MEMO() != null) {
return ExplainLevel.MEMO_PLAN;
}
if (planTypeContext.DISTRIBUTED() != null) {
return ExplainLevel.DISTRIBUTED_PLAN;
}
return ExplainLevel.ALL_PLAN;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
import org.apache.doris.nereids.util.JoinUtils;
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.qe.SessionVariable;

import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
Expand Down Expand Up @@ -213,12 +214,12 @@ private boolean couldNotRightBucketShuffleJoin(JoinType joinType, DistributionSp
|| joinType == JoinType.FULL_OUTER_JOIN);
boolean isSpecInScope = (leftHashSpec.getShuffleType() == ShuffleType.NATURAL
|| rightHashSpec.getShuffleType() == ShuffleType.NATURAL);
return isJoinTypeInScope && isSpecInScope;
return isJoinTypeInScope && isSpecInScope && !SessionVariable.canUseNereidsDistributePlanner();
}

@Override
public Boolean visitPhysicalHashJoin(PhysicalHashJoin<? extends Plan, ? extends Plan> hashJoin,
Void context) {
public Boolean visitPhysicalHashJoin(
PhysicalHashJoin<? extends Plan, ? extends Plan> hashJoin, Void context) {
Preconditions.checkArgument(children.size() == 2, "children.size() != 2");
Preconditions.checkArgument(childrenProperties.size() == 2);
Preconditions.checkArgument(requiredProperties.size() == 2);
Expand Down Expand Up @@ -303,13 +304,24 @@ public Boolean visitPhysicalHashJoin(PhysicalHashJoin<? extends Plan, ? extends
(DistributionSpecHash) requiredProperties.get(1).getDistributionSpec()));
} else if (leftHashSpec.getShuffleType() == ShuffleType.EXECUTION_BUCKETED
&& rightHashSpec.getShuffleType() == ShuffleType.NATURAL) {
// TODO: we must do shuffle on right because coordinator could not do right be selection in this case,
// since it always to check the left most node whether olap scan node.
// after we fix coordinator problem, we could do right to left bucket shuffle
updatedForRight = Optional.of(calAnotherSideRequired(
ShuffleType.EXECUTION_BUCKETED, leftHashSpec, rightHashSpec,
(DistributionSpecHash) requiredProperties.get(0).getDistributionSpec(),
(DistributionSpecHash) requiredProperties.get(1).getDistributionSpec()));
if (SessionVariable.canUseNereidsDistributePlanner()) {
// nereids coordinator can exchange left side to right side to do bucket shuffle join
// TODO: maybe we should check if left child is PhysicalDistribute.
// If so add storage bucketed shuffle on left side. Other wise,
// add execution bucketed shuffle on right side.
updatedForLeft = Optional.of(calAnotherSideRequired(
ShuffleType.STORAGE_BUCKETED, rightHashSpec, leftHashSpec,
(DistributionSpecHash) requiredProperties.get(1).getDistributionSpec(),
(DistributionSpecHash) requiredProperties.get(0).getDistributionSpec()));
} else {
// legacy coordinator could not do right be selection in this case,
// since it always to check the left most node whether olap scan node.
// so we can only shuffle right to left side to do normal shuffle join
updatedForRight = Optional.of(calAnotherSideRequired(
ShuffleType.EXECUTION_BUCKETED, leftHashSpec, rightHashSpec,
(DistributionSpecHash) requiredProperties.get(0).getDistributionSpec(),
(DistributionSpecHash) requiredProperties.get(1).getDistributionSpec()));
}
} else if (leftHashSpec.getShuffleType() == ShuffleType.EXECUTION_BUCKETED
&& rightHashSpec.getShuffleType() == ShuffleType.EXECUTION_BUCKETED) {
if (bothSideShuffleKeysAreSameOrder(rightHashSpec, leftHashSpec,
Expand Down Expand Up @@ -537,20 +549,20 @@ private List<ExprId> calAnotherSideRequiredShuffleIds(DistributionSpecHash notSh
* calAnotherSideRequiredShuffleIds's comment.
*
* @param shuffleType real output shuffle type
* @param notShuffleSideOutput not shuffle side real output used hash spec
* @param shuffleSideOutput shuffle side real output used hash spec
* @param notShuffleSideRequired not shuffle side required used hash spec
* @param shuffleSideRequired shuffle side required hash spec
* @param notNeedShuffleSideOutput not shuffle side real output used hash spec
* @param needShuffleSideOutput shuffle side real output used hash spec
* @param notNeedShuffleSideRequired not shuffle side required used hash spec
* @param needShuffleSideRequired shuffle side required hash spec
* @return shuffle side new required hash spec
*/
private PhysicalProperties calAnotherSideRequired(ShuffleType shuffleType,
DistributionSpecHash notShuffleSideOutput, DistributionSpecHash shuffleSideOutput,
DistributionSpecHash notShuffleSideRequired, DistributionSpecHash shuffleSideRequired) {
List<ExprId> shuffleSideIds = calAnotherSideRequiredShuffleIds(notShuffleSideOutput,
notShuffleSideRequired, shuffleSideRequired);
DistributionSpecHash notNeedShuffleSideOutput, DistributionSpecHash needShuffleSideOutput,
DistributionSpecHash notNeedShuffleSideRequired, DistributionSpecHash needShuffleSideRequired) {
List<ExprId> shuffleSideIds = calAnotherSideRequiredShuffleIds(notNeedShuffleSideOutput,
notNeedShuffleSideRequired, needShuffleSideRequired);
return new PhysicalProperties(new DistributionSpecHash(shuffleSideIds, shuffleType,
shuffleSideOutput.getTableId(), shuffleSideOutput.getSelectedIndexId(),
shuffleSideOutput.getPartitionIds()));
needShuffleSideOutput.getTableId(), needShuffleSideOutput.getSelectedIndexId(),
needShuffleSideOutput.getPartitionIds()));
}

private void updateChildEnforceAndCost(int index, PhysicalProperties targetProperties) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import org.apache.doris.nereids.types.DataType;
import org.apache.doris.qe.SessionVariable;
import org.apache.doris.statistics.ColumnStatistic;
import org.apache.doris.statistics.Statistics;
import org.apache.doris.tablefunction.TableValuedFunctionIf;
Expand Down Expand Up @@ -113,6 +114,9 @@ public boolean nullable() {
}

public PhysicalProperties getPhysicalProperties() {
if (SessionVariable.canUseNereidsDistributePlanner()) {
return PhysicalProperties.ANY;
}
return PhysicalProperties.STORAGE_ANY;
}

Expand Down
Loading

0 comments on commit 5b68de5

Please sign in to comment.