From 1789ec61c4046523bc81715cddb15e9d0033d0e1 Mon Sep 17 00:00:00 2001 From: 924060929 <924060929@qq.com> Date: Fri, 28 Jun 2024 17:11:24 +0800 Subject: [PATCH] [refactor](nereids) New distribute planner (#36531) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## 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[#8] id2[#9] id[#10] id2[#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[#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 --- .../org/apache/doris/nereids/DorisParser.g4 | 1 + .../main/java/org/apache/doris/common/Id.java | 7 +- .../org/apache/doris/common/TreeNode.java | 25 ++ .../apache/doris/common/profile/Profile.java | 10 + .../doris/common/profile/SummaryProfile.java | 13 + .../apache/doris/nereids/NereidsPlanner.java | 60 +++- .../nereids/parser/LogicalPlanBuilder.java | 3 + .../ChildrenPropertiesRegulator.java | 52 +-- .../functions/table/TableValuedFunction.java | 4 + .../nereids/trees/plans/PlaceholderId.java | 7 +- .../trees/plans/commands/ExplainCommand.java | 1 + .../distribute/BucketSpecifyInstances.java | 30 ++ .../distribute/DefaultSpecifyInstances.java | 30 ++ .../plans/distribute/DistributePlanner.java | 66 ++++ .../plans/distribute/DistributedPlan.java | 57 +++ .../plans/distribute/FragmentIdMapping.java | 71 ++++ .../distribute/NereidsSpecifyInstances.java | 59 +++ .../distribute/PipelineDistributedPlan.java | 72 ++++ .../BackendDistributedPlanWorkerManager.java | 63 ++++ .../distribute/worker/BackendWorker.java | 74 ++++ .../worker/DistributedPlanWorker.java | 40 +++ .../worker/DistributedPlanWorkerManager.java | 25 ++ .../worker/LoadBalanceScanWorkerSelector.java | 336 ++++++++++++++++++ .../distribute/worker/ScanWorkerSelector.java | 62 ++++ .../distribute/worker/WorkerScanRanges.java | 33 ++ .../plans/distribute/worker/Workload.java | 22 ++ .../worker/job/AbstractUnassignedJob.java | 74 ++++ .../worker/job/AbstractUnassignedScanJob.java | 202 +++++++++++ .../distribute/worker/job/AssignedJob.java | 39 ++ .../worker/job/AssignedJobBuilder.java | 64 ++++ .../worker/job/BucketScanSource.java | 148 ++++++++ .../worker/job/CustomAssignmentJob.java | 29 ++ .../worker/job/DefaultScanSource.java | 111 ++++++ .../worker/job/LocalShuffleAssignedJob.java | 43 +++ .../distribute/worker/job/ScanRange.java | 23 ++ .../distribute/worker/job/ScanRanges.java | 109 ++++++ .../distribute/worker/job/ScanSource.java | 41 +++ .../distribute/worker/job/Splittable.java | 69 ++++ .../worker/job/StaticAssignedJob.java | 106 ++++++ ...ssignedGatherScanMultiRemoteTablesJob.java | 87 +++++ .../distribute/worker/job/UnassignedJob.java | 52 +++ .../worker/job/UnassignedJobBuilder.java | 266 ++++++++++++++ .../job/UnassignedQueryConstantJob.java | 50 +++ .../job/UnassignedScanBucketOlapTableJob.java | 310 ++++++++++++++++ .../job/UnassignedScanSingleOlapTableJob.java | 82 +++++ .../UnassignedScanSingleRemoteTableJob.java | 53 +++ .../worker/job/UnassignedShuffleJob.java | 124 +++++++ .../job/UnassignedSpecifyInstancesJob.java | 49 +++ .../worker/job/UninstancedScanSource.java | 36 ++ .../worker/job/WorkerScanSource.java | 31 ++ .../apache/doris/nereids/util/JoinUtils.java | 22 +- .../org/apache/doris/nereids/util/Utils.java | 14 + .../apache/doris/planner/DataGenScanNode.java | 4 + .../apache/doris/planner/OlapScanNode.java | 37 +- .../apache/doris/planner/PlanFragment.java | 25 ++ .../org/apache/doris/planner/PlanNode.java | 25 ++ .../org/apache/doris/planner/ScanNode.java | 9 +- .../org/apache/doris/qe/ConnectContext.java | 6 + .../java/org/apache/doris/qe/Coordinator.java | 51 ++- .../apache/doris/qe/NereidsCoordinator.java | 194 ++++++++++ .../org/apache/doris/qe/SessionVariable.java | 54 ++- .../org/apache/doris/qe/StmtExecutor.java | 12 +- .../apache/doris/qe/HmsQueryCacheTest.java | 14 +- .../distribute/colocate_union_numbers.out | 10 + .../distribute/local_shuffle.out | 36 ++ .../prune_bucket_with_bucket_shuffle_join.out | 5 + .../distribute/query_constant.out | 12 + .../nereids_syntax_p0/distribute/shuffle.out | 8 + .../distribute/shuffle_left_join.out | 9 + .../test_forbid_unknown_col_stats.groovy | 2 +- .../distribute/colocate_union_numbers.groovy | 70 ++++ .../distribute/local_shuffle.groovy | 187 ++++++++++ ...une_bucket_with_bucket_shuffle_join.groovy | 98 +++++ .../distribute/query_constant.groovy | 39 ++ .../distribute/shuffle.groovy | 34 ++ .../distribute/shuffle_left_join.groovy | 117 ++++++ 76 files changed, 4438 insertions(+), 77 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/BucketSpecifyInstances.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DefaultSpecifyInstances.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DistributePlanner.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DistributedPlan.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/FragmentIdMapping.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/NereidsSpecifyInstances.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/PipelineDistributedPlan.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/BackendDistributedPlanWorkerManager.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/BackendWorker.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/DistributedPlanWorker.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/DistributedPlanWorkerManager.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/LoadBalanceScanWorkerSelector.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/ScanWorkerSelector.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/WorkerScanRanges.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/Workload.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AbstractUnassignedJob.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AbstractUnassignedScanJob.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AssignedJob.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AssignedJobBuilder.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/BucketScanSource.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/CustomAssignmentJob.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/DefaultScanSource.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/LocalShuffleAssignedJob.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/ScanRange.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/ScanRanges.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/ScanSource.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/Splittable.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/StaticAssignedJob.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedGatherScanMultiRemoteTablesJob.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJob.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJobBuilder.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedQueryConstantJob.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanBucketOlapTableJob.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanSingleOlapTableJob.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanSingleRemoteTableJob.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedShuffleJob.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedSpecifyInstancesJob.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UninstancedScanSource.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/WorkerScanSource.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java create mode 100644 regression-test/data/nereids_syntax_p0/distribute/colocate_union_numbers.out create mode 100644 regression-test/data/nereids_syntax_p0/distribute/local_shuffle.out create mode 100644 regression-test/data/nereids_syntax_p0/distribute/prune_bucket_with_bucket_shuffle_join.out create mode 100644 regression-test/data/nereids_syntax_p0/distribute/query_constant.out create mode 100644 regression-test/data/nereids_syntax_p0/distribute/shuffle.out create mode 100644 regression-test/data/nereids_syntax_p0/distribute/shuffle_left_join.out create mode 100644 regression-test/suites/nereids_syntax_p0/distribute/colocate_union_numbers.groovy create mode 100644 regression-test/suites/nereids_syntax_p0/distribute/local_shuffle.groovy create mode 100644 regression-test/suites/nereids_syntax_p0/distribute/prune_bucket_with_bucket_shuffle_join.groovy create mode 100644 regression-test/suites/nereids_syntax_p0/distribute/query_constant.groovy create mode 100644 regression-test/suites/nereids_syntax_p0/distribute/shuffle.groovy create mode 100644 regression-test/suites/nereids_syntax_p0/distribute/shuffle_left_join.groovy diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index ee9b91fdf6f9c7..05a675997fc0f8 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -262,6 +262,7 @@ planType | OPTIMIZED | PHYSICAL // same type | SHAPE | MEMO + | DISTRIBUTED | ALL // default type ; diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/Id.java b/fe/fe-core/src/main/java/org/apache/doris/common/Id.java index 9d6dad50a46238..a6bd3896708058 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/Id.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/Id.java @@ -25,7 +25,7 @@ /** * Integer ids that cannot accidentally be compared with ints. */ -public class Id> { +public class Id> implements Comparable> { protected final int id; public Id(int id) { @@ -62,4 +62,9 @@ public ArrayList asList() { public String toString() { return Integer.toString(id); } + + @Override + public int compareTo(Id idTypeId) { + return id - idTypeId.id; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/TreeNode.java b/fe/fe-core/src/main/java/org/apache/doris/common/TreeNode.java index 8070505798b98c..b254dd9a1d0aa1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/TreeNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/TreeNode.java @@ -249,4 +249,29 @@ public void foreach(ThrowingConsumer> func) throws AnalysisEx child.foreach(func); } } + + /** anyMatch */ + public boolean anyMatch(Predicate> func) { + if (func.apply(this)) { + return true; + } + + for (NodeType child : children) { + if (child.anyMatch(func)) { + return true; + } + } + return false; + } + + /** foreachDown */ + public void foreachDown(Predicate> visitor) { + if (!visitor.test(this)) { + return; + } + + for (TreeNode child : getChildren()) { + child.foreachDown(visitor); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/profile/Profile.java b/fe/fe-core/src/main/java/org/apache/doris/common/profile/Profile.java index 96cecd19ec62b3..b45527f60bd596 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/profile/Profile.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/profile/Profile.java @@ -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; @@ -108,6 +110,14 @@ public synchronized void updateSummary(long startTime, Map summa } summaryInfo.put(SummaryProfile.PHYSICAL_PLAN, builder.toString().replace("\n", "\n ")); + + FragmentIdMapping 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) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java b/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java index 8a7119cb2cef39..2e686171238e1e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java @@ -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"; @@ -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"; @@ -109,6 +111,7 @@ public class SummaryProfile { public static final ImmutableList SUMMARY_KEYS = new ImmutableList.Builder() .addAll(SUMMARY_CAPTIONS) .add(PHYSICAL_PLAN) + .add(DISTRIBUTED_PLAN) .build(); // The display order of execution summary items. @@ -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 @@ -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, @@ -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(); } @@ -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"; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java index 829cf6512d6d67..e0d6f2f7589b13 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java @@ -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; @@ -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; @@ -102,6 +106,7 @@ public class NereidsPlanner extends Planner { private Plan rewrittenPlan; private Plan optimizedPlan; private PhysicalPlan physicalPlan; + private FragmentIdMapping distributedPlans; // The cost of optimized plan private double cost = 0; private LogicalPlanAdapter logicalPlanAdapter; @@ -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 @@ -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; } @@ -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); } @@ -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" @@ -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) @@ -681,6 +725,10 @@ public PhysicalPlan getPhysicalPlan() { return physicalPlan; } + public FragmentIdMapping getDistributedPlans() { + return distributedPlans; + } + public LogicalPlanAdapter getLogicalPlanAdapter() { return logicalPlanAdapter; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index ac4f25a7ab70cd..b6f679da5e4752 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -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; } 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 3beed014aac910..f4029b3e7d58e2 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 @@ -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; @@ -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 hashJoin, - Void context) { + public Boolean visitPhysicalHashJoin( + PhysicalHashJoin hashJoin, Void context) { Preconditions.checkArgument(children.size() == 2, "children.size() != 2"); Preconditions.checkArgument(childrenProperties.size() == 2); Preconditions.checkArgument(requiredProperties.size() == 2); @@ -303,13 +304,24 @@ public Boolean visitPhysicalHashJoin(PhysicalHashJoin 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 shuffleSideIds = calAnotherSideRequiredShuffleIds(notShuffleSideOutput, - notShuffleSideRequired, shuffleSideRequired); + DistributionSpecHash notNeedShuffleSideOutput, DistributionSpecHash needShuffleSideOutput, + DistributionSpecHash notNeedShuffleSideRequired, DistributionSpecHash needShuffleSideRequired) { + List 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) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java index 26602435651fab..fda5302059831a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java @@ -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; @@ -113,6 +114,9 @@ public boolean nullable() { } public PhysicalProperties getPhysicalProperties() { + if (SessionVariable.canUseNereidsDistributePlanner()) { + return PhysicalProperties.ANY; + } return PhysicalProperties.STORAGE_ANY; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlaceholderId.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlaceholderId.java index be3cb645fe6233..f1d410100e16fa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlaceholderId.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlaceholderId.java @@ -23,7 +23,7 @@ /** * placeholder id for prepared statement parameters */ -public class PlaceholderId extends Id implements Comparable { +public class PlaceholderId extends Id { public PlaceholderId(int id) { super(id); @@ -55,9 +55,4 @@ public boolean equals(Object obj) { public int hashCode() { return super.hashCode(); } - - @Override - public int compareTo(PlaceholderId o) { - return this.id - o.id; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExplainCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExplainCommand.java index 500f1d4a09a478..03278723d344ab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExplainCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExplainCommand.java @@ -49,6 +49,7 @@ public enum ExplainLevel { OPTIMIZED_PLAN(true), SHAPE_PLAN(true), MEMO_PLAN(true), + DISTRIBUTED_PLAN(true), ALL_PLAN(true) ; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/BucketSpecifyInstances.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/BucketSpecifyInstances.java new file mode 100644 index 00000000000000..24c0ddf5509b60 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/BucketSpecifyInstances.java @@ -0,0 +1,30 @@ +// 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.trees.plans.distribute; + +import org.apache.doris.nereids.trees.plans.distribute.worker.job.BucketScanSource; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.WorkerScanSource; + +import java.util.List; + +/** DefaultNereidsSpecifyInstances */ +public class BucketSpecifyInstances extends NereidsSpecifyInstances { + public BucketSpecifyInstances(List> workerScanSources) { + super(workerScanSources); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DefaultSpecifyInstances.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DefaultSpecifyInstances.java new file mode 100644 index 00000000000000..75d4b24c2d4480 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DefaultSpecifyInstances.java @@ -0,0 +1,30 @@ +// 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.trees.plans.distribute; + +import org.apache.doris.nereids.trees.plans.distribute.worker.job.DefaultScanSource; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.WorkerScanSource; + +import java.util.List; + +/** DefaultSpecifyInstances */ +public class DefaultSpecifyInstances extends NereidsSpecifyInstances { + public DefaultSpecifyInstances(List> workerToScanSources) { + super(workerToScanSources); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DistributePlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DistributePlanner.java new file mode 100644 index 00000000000000..ceef281c0fc534 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DistributePlanner.java @@ -0,0 +1,66 @@ +// 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.trees.plans.distribute; + +import org.apache.doris.nereids.trees.plans.distribute.worker.job.AssignedJob; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.AssignedJobBuilder; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.UnassignedJob; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.UnassignedJobBuilder; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.planner.PlanFragmentId; + +import com.google.common.collect.ListMultimap; + +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; + +/** DistributePlanner */ +public class DistributePlanner { + private final List fragments; + private final FragmentIdMapping idToFragments; + + public DistributePlanner(List fragments) { + this.fragments = Objects.requireNonNull(fragments, "fragments can not be null"); + this.idToFragments = FragmentIdMapping.buildFragmentMapping(fragments); + } + + public FragmentIdMapping plan() { + FragmentIdMapping fragmentJobs = UnassignedJobBuilder.buildJobs(idToFragments); + ListMultimap instanceJobs = AssignedJobBuilder.buildJobs(fragmentJobs); + return buildDistributePlans(fragmentJobs, instanceJobs); + } + + private FragmentIdMapping buildDistributePlans( + Map idToUnassignedJobs, + ListMultimap idToAssignedJobs) { + FragmentIdMapping idToDistributedPlans = new FragmentIdMapping<>(); + for (Entry kv : idToFragments.entrySet()) { + PlanFragmentId fragmentId = kv.getKey(); + PlanFragment fragment = kv.getValue(); + + UnassignedJob fragmentJob = idToUnassignedJobs.get(fragmentId); + List instanceJobs = idToAssignedJobs.get(fragmentId); + + List childrenPlans = idToDistributedPlans.getByChildrenFragments(fragment); + idToDistributedPlans.put(fragmentId, new PipelineDistributedPlan(fragmentJob, instanceJobs, childrenPlans)); + } + return (FragmentIdMapping) idToDistributedPlans; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DistributedPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DistributedPlan.java new file mode 100644 index 00000000000000..8f176e3caccc7c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/DistributedPlan.java @@ -0,0 +1,57 @@ +// 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.trees.plans.distribute; + +import org.apache.doris.nereids.trees.AbstractTreeNode; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.UnassignedJob; +import org.apache.doris.nereids.util.Utils; + +import java.util.List; +import java.util.Objects; + +/** DistributedPlan */ +@lombok.Getter +public abstract class DistributedPlan extends AbstractTreeNode { + protected final UnassignedJob fragmentJob; + protected final List inputs; + + public DistributedPlan(UnassignedJob fragmentJob, List inputs) { + this.fragmentJob = Objects.requireNonNull(fragmentJob, "fragmentJob can not be null"); + this.inputs = Utils.fastToImmutableList(Objects.requireNonNull(inputs, "inputs can not be null")); + } + + @Override + public DistributedPlan withChildren(List children) { + throw new UnsupportedOperationException(); + } + + public abstract String toString(int displayFragmentId); + + /** toString */ + public static String toString(List distributedPlansBottomToTop) { + StringBuilder distributedPlanStringBuilder = new StringBuilder(); + int fragmentDisplayId = 0; + for (int i = distributedPlansBottomToTop.size() - 1; i >= 0; i--) { + DistributedPlan distributedPlan = distributedPlansBottomToTop.get(i); + distributedPlanStringBuilder + .append(distributedPlan.toString(fragmentDisplayId++)) + .append("\n"); + } + return distributedPlanStringBuilder.toString(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/FragmentIdMapping.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/FragmentIdMapping.java new file mode 100644 index 00000000000000..95bf36051d2033 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/FragmentIdMapping.java @@ -0,0 +1,71 @@ +// 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.trees.plans.distribute; + +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.planner.PlanFragmentId; + +import com.google.common.collect.ImmutableList; + +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; + +/** + * FragmentIdMapping: + * key: PlanFragmentId + * value: T + * + * NOTE: this map should order by PlanFragmentId asc + */ +public class FragmentIdMapping extends TreeMap { + public FragmentIdMapping() { + } + + public FragmentIdMapping(Comparator comparator) { + super(comparator); + } + + public FragmentIdMapping(Map m) { + super(m); + } + + public FragmentIdMapping(SortedMap m) { + super(m); + } + + /** getByChildrenFragments */ + public List getByChildrenFragments(PlanFragment fragment) { + List children = fragment.getChildren(); + ImmutableList.Builder values = ImmutableList.builderWithExpectedSize(children.size()); + for (PlanFragment child : children) { + values.add(get(child.getFragmentId())); + } + return values.build(); + } + + public static FragmentIdMapping buildFragmentMapping(List fragments) { + FragmentIdMapping idToFragments = new FragmentIdMapping<>(); + for (PlanFragment fragment : fragments) { + idToFragments.put(fragment.getFragmentId(), fragment); + } + return idToFragments; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/NereidsSpecifyInstances.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/NereidsSpecifyInstances.java new file mode 100644 index 00000000000000..58a564d3f873d2 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/NereidsSpecifyInstances.java @@ -0,0 +1,59 @@ +// 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.trees.plans.distribute; + +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.AssignedJob; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.ScanSource; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.StaticAssignedJob; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.UnassignedJob; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.WorkerScanSource; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.collect.Lists; + +import java.util.List; +import java.util.Objects; + +/** NereidsSpecifyInstances */ +public abstract class NereidsSpecifyInstances { + public final List> workerScanSources; + + public NereidsSpecifyInstances(List> workerScanSources) { + this.workerScanSources = Objects.requireNonNull(workerScanSources, + "workerScanSources can not be null"); + } + + /** buildAssignedJobs */ + public List buildAssignedJobs(UnassignedJob unassignedJob) { + List instances = Lists.newArrayListWithCapacity(workerScanSources.size()); + int instanceNum = 0; + ConnectContext context = ConnectContext.get(); + for (WorkerScanSource workerToScanSource : workerScanSources) { + TUniqueId instanceId = context.nextInstanceId(); + DistributedPlanWorker worker = workerToScanSource.worker; + ScanSource scanSource = workerToScanSource.scanSource; + StaticAssignedJob assignedJob = new StaticAssignedJob( + instanceNum++, instanceId, unassignedJob, worker, scanSource + ); + instances.add(assignedJob); + } + return instances; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/PipelineDistributedPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/PipelineDistributedPlan.java new file mode 100644 index 00000000000000..13f903d8a2cf05 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/PipelineDistributedPlan.java @@ -0,0 +1,72 @@ +// 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.trees.plans.distribute; + +import org.apache.doris.nereids.trees.plans.distribute.worker.job.AssignedJob; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.UnassignedJob; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.thrift.TExplainLevel; + +import java.util.List; +import java.util.Objects; + +/** PipelineDistributedPlan */ +public class PipelineDistributedPlan extends DistributedPlan { + protected final List instanceJobs; + + public PipelineDistributedPlan( + UnassignedJob fragmentJob, + List instanceJobs, + List inputs) { + super(fragmentJob, inputs); + this.instanceJobs = Utils.fastToImmutableList( + Objects.requireNonNull(instanceJobs, "instanceJobs can not be null") + ); + } + + public List getInstanceJobs() { + return instanceJobs; + } + + @Override + public String toString(int displayFragmentId) { + StringBuilder instancesStr = new StringBuilder(); + for (int i = 0; i < instanceJobs.size(); i++) { + instancesStr.append(instanceJobs.get(i).toString(false)); + if (i + 1 < instanceJobs.size()) { + instancesStr.append(",\n"); + } + } + String instancesStrWithIndent = Utils.addLinePrefix(instancesStr.toString(), " "); + + String explainString = Utils.addLinePrefix( + fragmentJob.getFragment().getExplainString(TExplainLevel.VERBOSE).trim(), " " + ); + + return "PipelineDistributedPlan(\n" + + " id: " + displayFragmentId + ",\n" + + " parallel: " + instanceJobs.size() + ",\n" + + " fragmentJob: " + fragmentJob + ",\n" + + " fragment: {\n" + + " " + explainString + "\n" + + " },\n" + + " instanceJobs: [\n" + instancesStrWithIndent + "\n" + + " ]\n" + + ")"; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/BackendDistributedPlanWorkerManager.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/BackendDistributedPlanWorkerManager.java new file mode 100644 index 00000000000000..190d6d898a9922 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/BackendDistributedPlanWorkerManager.java @@ -0,0 +1,63 @@ +// 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.trees.plans.distribute.worker; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.NereidsException; +import org.apache.doris.common.Reference; +import org.apache.doris.qe.SimpleScheduler; +import org.apache.doris.system.Backend; + +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableMap; + +import java.util.function.Supplier; + +/** BackendWorkerManager */ +public class BackendDistributedPlanWorkerManager implements DistributedPlanWorkerManager { + private final Supplier> backends = Suppliers.memoize(() -> { + try { + return Env.getCurrentSystemInfo().getBackendsWithIdByCurrentCluster(); + } catch (Exception t) { + throw new NereidsException("Can not get backends: " + t, t); + } + }); + + @Override + public DistributedPlanWorker getWorker(long backendId) { + ImmutableMap backends = this.backends.get(); + Backend backend = backends.get(backendId); + if (backend == null) { + throw new IllegalStateException("Backend " + backendId + " is not exist"); + } + return new BackendWorker(backend); + } + + @Override + public DistributedPlanWorker randomAvailableWorker() { + try { + Reference selectedBackendId = new Reference<>(); + ImmutableMap backends = this.backends.get(); + SimpleScheduler.getHost(backends, selectedBackendId); + Backend selctedBackend = backends.get(selectedBackendId.getRef()); + return new BackendWorker(selctedBackend); + } catch (Exception t) { + throw new NereidsException("Can not get backends: " + t, t); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/BackendWorker.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/BackendWorker.java new file mode 100644 index 00000000000000..702a00dd358d29 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/BackendWorker.java @@ -0,0 +1,74 @@ +// 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.trees.plans.distribute.worker; + +import org.apache.doris.system.Backend; + +import java.util.Objects; + +/** BackendWorker */ +public class BackendWorker implements DistributedPlanWorker { + private final Backend backend; + + public BackendWorker(Backend backend) { + this.backend = backend; + } + + @Override + public long id() { + return backend.getId(); + } + + @Override + public String address() { + return backend.getAddress(); + } + + @Override + public String host() { + return backend.getHost(); + } + + @Override + public int port() { + return backend.getBePort(); + } + + @Override + public boolean available() { + return backend.isQueryAvailable(); + } + + @Override + public int hashCode() { + return Objects.hash(backend.getId()); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof BackendWorker)) { + return false; + } + return backend.getId() == ((BackendWorker) obj).backend.getId(); + } + + @Override + public String toString() { + return "BackendWorker(id: " + backend.getId() + ", address: " + backend.getAddress() + ")"; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/DistributedPlanWorker.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/DistributedPlanWorker.java new file mode 100644 index 00000000000000..c86675a6dab27c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/DistributedPlanWorker.java @@ -0,0 +1,40 @@ +// 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.trees.plans.distribute.worker; + +/** + * DistributedPlanWorker: a worker who can execute the assigned job(instance) of the DistributedPlan + */ +public interface DistributedPlanWorker extends Comparable { + long id(); + + // ipv4/ipv6 address + String address(); + + String host(); + + int port(); + + // whether is this worker alive? + boolean available(); + + @Override + default int compareTo(DistributedPlanWorker worker) { + return address().compareTo(worker.address()); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/DistributedPlanWorkerManager.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/DistributedPlanWorkerManager.java new file mode 100644 index 00000000000000..f0604d13e346fb --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/DistributedPlanWorkerManager.java @@ -0,0 +1,25 @@ +// 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.trees.plans.distribute.worker; + +/** DistributedPlanWorkerManager */ +public interface DistributedPlanWorkerManager { + DistributedPlanWorker getWorker(long backendId); + + DistributedPlanWorker randomAvailableWorker(); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/LoadBalanceScanWorkerSelector.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/LoadBalanceScanWorkerSelector.java new file mode 100644 index 00000000000000..60021281181f82 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/LoadBalanceScanWorkerSelector.java @@ -0,0 +1,336 @@ +// 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.trees.plans.distribute.worker; + +import org.apache.doris.common.Pair; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.BucketScanSource; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.DefaultScanSource; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.ScanRanges; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.UnassignedJob; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.UnassignedScanBucketOlapTableJob; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.UninstancedScanSource; +import org.apache.doris.planner.DataPartition; +import org.apache.doris.planner.OlapScanNode; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.planner.ScanNode; +import org.apache.doris.thrift.TExternalScanRange; +import org.apache.doris.thrift.TFileRangeDesc; +import org.apache.doris.thrift.TFileScanRange; +import org.apache.doris.thrift.TPaloScanRange; +import org.apache.doris.thrift.TScanRange; +import org.apache.doris.thrift.TScanRangeLocation; +import org.apache.doris.thrift.TScanRangeLocations; +import org.apache.doris.thrift.TScanRangeParams; + +import com.google.common.base.Function; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.function.BiFunction; + +/** LoadBalanceScanWorkerSelector */ +public class LoadBalanceScanWorkerSelector implements ScanWorkerSelector { + private final BackendDistributedPlanWorkerManager workerManager = new BackendDistributedPlanWorkerManager(); + private final Map workloads = Maps.newLinkedHashMap(); + + @Override + public DistributedPlanWorkerManager getWorkerManager() { + return workerManager; + } + + @Override + public DistributedPlanWorker selectMinWorkloadWorker(List workers) { + DistributedPlanWorker minWorkloadWorker = null; + WorkerWorkload minWorkload = new WorkerWorkload(Integer.MAX_VALUE, Long.MAX_VALUE); + for (DistributedPlanWorker worker : workers) { + WorkerWorkload workload = getWorkload(worker); + if (minWorkload.compareTo(workload) > 0) { + minWorkloadWorker = worker; + minWorkload = workload; + } + } + minWorkload.recordOneScanTask(1); + return minWorkloadWorker; + } + + @Override + public Map selectReplicaAndWorkerWithoutBucket(ScanNode scanNode) { + Map workerScanRanges = Maps.newLinkedHashMap(); + // allScanRangesLocations is all scan ranges in all partition which need to scan + List allScanRangesLocations = scanNode.getScanRangeLocations(0); + for (TScanRangeLocations onePartitionOneScanRangeLocation : allScanRangesLocations) { + // usually, the onePartitionOneScanRangeLocation is a tablet in one partition + long bytes = getScanRangeSize(scanNode, onePartitionOneScanRangeLocation); + + WorkerScanRanges assigned = selectScanReplicaAndMinWorkloadWorker( + onePartitionOneScanRangeLocation, bytes); + UninstancedScanSource scanRanges = workerScanRanges.computeIfAbsent( + assigned.worker, + w -> new UninstancedScanSource( + new DefaultScanSource(ImmutableMap.of(scanNode, new ScanRanges())) + ) + ); + DefaultScanSource scanSource = (DefaultScanSource) scanRanges.scanSource; + scanSource.scanNodeToScanRanges.get(scanNode).addScanRanges(assigned.scanRanges); + } + return workerScanRanges; + } + + @Override + public Map selectReplicaAndWorkerWithBucket( + UnassignedScanBucketOlapTableJob unassignedJob) { + PlanFragment fragment = unassignedJob.getFragment(); + List scanNodes = unassignedJob.getScanNodes(); + List olapScanNodes = unassignedJob.getOlapScanNodes(); + + BiFunction> bucketScanRangeSupplier = bucketScanRangeSupplier(); + Function> bucketBytesSupplier = bucketBytesSupplier(); + // all are olap scan nodes + if (!scanNodes.isEmpty() && scanNodes.size() == olapScanNodes.size()) { + if (olapScanNodes.size() == 1 && fragment.hasBucketShuffleJoin()) { + return selectForBucket(unassignedJob, scanNodes, bucketScanRangeSupplier, bucketBytesSupplier); + } else if (fragment.hasColocatePlanNode()) { + return selectForBucket(unassignedJob, scanNodes, bucketScanRangeSupplier, bucketBytesSupplier); + } + } else if (olapScanNodes.isEmpty() && fragment.getDataPartition() == DataPartition.UNPARTITIONED) { + return selectForBucket(unassignedJob, scanNodes, bucketScanRangeSupplier, bucketBytesSupplier); + } + throw new IllegalStateException( + "Illegal bucket shuffle join or colocate join in fragment: " + fragment.getFragmentId() + ); + } + + private BiFunction> bucketScanRangeSupplier() { + return (scanNode, bucketIndex) -> { + if (scanNode instanceof OlapScanNode) { + return (List) ((OlapScanNode) scanNode).bucketSeq2locations.get(bucketIndex); + } else { + // the backend is selected by XxxScanNode.createScanRangeLocations() + return scanNode.getScanRangeLocations(0); + } + }; + } + + private Function> bucketBytesSupplier() { + return scanNode -> { + if (scanNode instanceof OlapScanNode) { + return ((OlapScanNode) scanNode).bucketSeq2Bytes; + } else { + // not supported yet + return ImmutableMap.of(0, 0L); + } + }; + } + + private Map selectForBucket( + UnassignedJob unassignedJob, List scanNodes, + BiFunction> bucketScanRangeSupplier, + Function> bucketBytesSupplier) { + Map assignment = Maps.newLinkedHashMap(); + + Map bucketIndexToBytes = computeEachBucketScanBytes(scanNodes, bucketBytesSupplier); + + for (Entry kv : bucketIndexToBytes.entrySet()) { + Integer bucketIndex = kv.getKey(); + long allScanNodeScanBytesInOneBucket = kv.getValue(); + + DistributedPlanWorker selectedWorker = null; + for (ScanNode scanNode : scanNodes) { + List allPartitionTabletsInOneBucketInOneTable + = bucketScanRangeSupplier.apply(scanNode, bucketIndex); + if (!allPartitionTabletsInOneBucketInOneTable.isEmpty()) { + WorkerScanRanges replicaAndWorker = selectScanReplicaAndMinWorkloadWorker( + allPartitionTabletsInOneBucketInOneTable.get(0), allScanNodeScanBytesInOneBucket); + selectedWorker = replicaAndWorker.worker; + break; + } + // else: the bucket is pruned, we should use another ScanNode to select worker for this bucket + } + if (selectedWorker == null) { + throw new IllegalStateException("Can not assign worker for bucket: " + bucketIndex); + } + + long workerId = selectedWorker.id(); + for (ScanNode scanNode : scanNodes) { + List allPartitionTabletsInOneBucket + = bucketScanRangeSupplier.apply(scanNode, bucketIndex); + List> selectedReplicasInOneBucket = filterReplicaByWorkerInBucket( + scanNode, workerId, bucketIndex, allPartitionTabletsInOneBucket + ); + UninstancedScanSource bucketIndexToScanNodeToTablets + = assignment.computeIfAbsent( + selectedWorker, + worker -> new UninstancedScanSource(new BucketScanSource(Maps.newLinkedHashMap())) + ); + BucketScanSource scanSource = (BucketScanSource) bucketIndexToScanNodeToTablets.scanSource; + Map scanNodeToScanRanges = scanSource.bucketIndexToScanNodeToTablets + .computeIfAbsent(bucketIndex, bucket -> Maps.newLinkedHashMap()); + ScanRanges scanRanges = scanNodeToScanRanges.computeIfAbsent(scanNode, node -> new ScanRanges()); + for (Pair replica : selectedReplicasInOneBucket) { + TScanRangeParams replicaParam = replica.first; + Long scanBytes = replica.second; + scanRanges.addScanRange(replicaParam, scanBytes); + } + } + } + return assignment; + } + + private WorkerScanRanges selectScanReplicaAndMinWorkloadWorker( + TScanRangeLocations tabletLocation, long tabletBytes) { + List replicaLocations = tabletLocation.getLocations(); + int replicaNum = replicaLocations.size(); + WorkerWorkload minWorkload = new WorkerWorkload(Integer.MAX_VALUE, Long.MAX_VALUE); + DistributedPlanWorker minWorkLoadWorker = null; + TScanRangeLocation selectedReplicaLocation = null; + + for (int i = 0; i < replicaNum; i++) { + TScanRangeLocation replicaLocation = replicaLocations.get(i); + DistributedPlanWorker worker = workerManager.getWorker(replicaLocation.getBackendId()); + if (!worker.available()) { + continue; + } + + WorkerWorkload workload = getWorkload(worker); + if (workload.compareTo(minWorkload) < 0) { + minWorkLoadWorker = worker; + minWorkload = workload; + selectedReplicaLocation = replicaLocation; + } + } + if (minWorkLoadWorker == null) { + throw new AnalysisException("No available workers"); + } else { + minWorkload.recordOneScanTask(tabletBytes); + ScanRanges scanRanges = new ScanRanges(); + TScanRangeParams scanReplicaParams = + ScanWorkerSelector.buildScanReplicaParams(tabletLocation, selectedReplicaLocation); + scanRanges.addScanRange(scanReplicaParams, tabletBytes); + return new WorkerScanRanges(minWorkLoadWorker, scanRanges); + } + } + + private List> filterReplicaByWorkerInBucket( + ScanNode scanNode, long filterWorkerId, int bucketIndex, + List allPartitionTabletsInOneBucket) { + List> selectedReplicasInOneBucket = Lists.newArrayList(); + for (TScanRangeLocations onePartitionOneTabletLocation : allPartitionTabletsInOneBucket) { + TScanRange scanRange = onePartitionOneTabletLocation.getScanRange(); + if (scanRange.getPaloScanRange() != null) { + long tabletId = scanRange.getPaloScanRange().getTabletId(); + boolean foundTabletInThisWorker = false; + for (TScanRangeLocation replicaLocation : onePartitionOneTabletLocation.getLocations()) { + if (replicaLocation.getBackendId() == filterWorkerId) { + TScanRangeParams scanReplicaParams = ScanWorkerSelector.buildScanReplicaParams( + onePartitionOneTabletLocation, replicaLocation); + Long replicaSize = ((OlapScanNode) scanNode).getTabletSingleReplicaSize(tabletId); + selectedReplicasInOneBucket.add(Pair.of(scanReplicaParams, replicaSize)); + foundTabletInThisWorker = true; + break; + } + } + if (!foundTabletInThisWorker) { + throw new IllegalStateException( + "Can not find tablet " + tabletId + " in the bucket: " + bucketIndex); + } + } else if (onePartitionOneTabletLocation.getLocations().size() == 1) { + TScanRangeLocation replicaLocation = onePartitionOneTabletLocation.getLocations().get(0); + TScanRangeParams scanReplicaParams = ScanWorkerSelector.buildScanReplicaParams( + onePartitionOneTabletLocation, replicaLocation); + Long replicaSize = 0L; + selectedReplicasInOneBucket.add(Pair.of(scanReplicaParams, replicaSize)); + } else { + throw new IllegalStateException("Unsupported"); + } + } + return selectedReplicasInOneBucket; + } + + private Map computeEachBucketScanBytes( + List scanNodes, Function> bucketBytesSupplier) { + Map bucketIndexToBytes = Maps.newLinkedHashMap(); + for (ScanNode scanNode : scanNodes) { + Map bucketSeq2Bytes = bucketBytesSupplier.apply(scanNode); + for (Entry bucketSeq2Byte : bucketSeq2Bytes.entrySet()) { + Integer bucketIndex = bucketSeq2Byte.getKey(); + Long scanBytes = bucketSeq2Byte.getValue(); + bucketIndexToBytes.merge(bucketIndex, scanBytes, Long::sum); + } + } + return bucketIndexToBytes; + } + + private WorkerWorkload getWorkload(DistributedPlanWorker worker) { + return workloads.computeIfAbsent(worker, w -> new WorkerWorkload()); + } + + private long getScanRangeSize(ScanNode scanNode, TScanRangeLocations scanRangeLocations) { + TScanRange scanRange = scanRangeLocations.getScanRange(); + TPaloScanRange paloScanRange = scanRange.getPaloScanRange(); + if (paloScanRange != null) { + long tabletId = paloScanRange.getTabletId(); + Long tabletBytes = ((OlapScanNode) scanNode).getTabletSingleReplicaSize(tabletId); + return tabletBytes == null ? 0L : tabletBytes; + } + + TExternalScanRange extScanRange = scanRange.getExtScanRange(); + if (extScanRange != null) { + TFileScanRange fileScanRange = extScanRange.getFileScanRange(); + long size = 0; + for (TFileRangeDesc range : fileScanRange.getRanges()) { + size += range.getSize(); + } + return size; + } + + return 0L; + } + + private static class WorkerWorkload implements Comparable { + private int taskNum; + private long scanBytes; + + public WorkerWorkload() { + this(0, 0); + } + + public WorkerWorkload(int taskNum, long scanBytes) { + this.taskNum = taskNum; + this.scanBytes = scanBytes; + } + + public void recordOneScanTask(long scanBytes) { + this.scanBytes += scanBytes; + } + + // order by scanBytes asc, taskNum asc + @Override + public int compareTo(WorkerWorkload workerWorkload) { + int compareScanBytes = Long.compare(this.scanBytes, workerWorkload.scanBytes); + if (compareScanBytes != 0) { + return compareScanBytes; + } + return taskNum - workerWorkload.taskNum; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/ScanWorkerSelector.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/ScanWorkerSelector.java new file mode 100644 index 00000000000000..40876a09e44301 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/ScanWorkerSelector.java @@ -0,0 +1,62 @@ +// 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.trees.plans.distribute.worker; + +import org.apache.doris.nereids.trees.plans.distribute.worker.job.UnassignedScanBucketOlapTableJob; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.UninstancedScanSource; +import org.apache.doris.planner.ScanNode; +import org.apache.doris.thrift.TScanRangeLocation; +import org.apache.doris.thrift.TScanRangeLocations; +import org.apache.doris.thrift.TScanRangeParams; + +import java.util.List; +import java.util.Map; + +/** ScanWorkerSelector */ +public interface ScanWorkerSelector { + DistributedPlanWorkerManager getWorkerManager(); + + DistributedPlanWorker selectMinWorkloadWorker(List workers); + + // for a scan node, select replica for each scan range(denote tablet if the ScanNode is OlapScanNode), + // use the replica location to build a worker execute the instance + Map selectReplicaAndWorkerWithoutBucket(ScanNode scanNode); + + // return + // key: Worker, the backend which will process this fragment + // value.key: Integer, the bucket index, from 0 to (bucket_num - 1) + // for example, create table statement contains: distributed by hash(id) buckets 10, + // the bucket index will from 0 to 9 + // value.value.key: ScanNode, which ScanNode the worker will process scan task + // value.value.value: ScanRanges, the tablets in current bucket, + // for example, colocate table `tbl` has 2 range partitions: + // p1 values[(1), (10)) and p2 values[(10), 11) with integer partition column part, + // and distributed by hash(id) buckets 10. And, so, there has 10 buckets from bucket 0 to + // bucket 9, and every bucket contains two tablets, because there are two partitions. + Map selectReplicaAndWorkerWithBucket( + UnassignedScanBucketOlapTableJob unassignedJob); + + static TScanRangeParams buildScanReplicaParams( + TScanRangeLocations tabletLocation, TScanRangeLocation replicaLocation) { + TScanRangeParams replicaParam = new TScanRangeParams(); + replicaParam.scan_range = tabletLocation.scan_range; + // Volume is optional, so we need to set the value and the is-set bit + replicaParam.setVolumeId(replicaLocation.volume_id); + return replicaParam; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/WorkerScanRanges.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/WorkerScanRanges.java new file mode 100644 index 00000000000000..25daa29bd870d1 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/WorkerScanRanges.java @@ -0,0 +1,33 @@ +// 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.trees.plans.distribute.worker; + +import org.apache.doris.nereids.trees.plans.distribute.worker.job.ScanRanges; + +import java.util.Objects; + +/** WorkerScanRange */ +public class WorkerScanRanges { + public final DistributedPlanWorker worker; + public final ScanRanges scanRanges; + + public WorkerScanRanges(DistributedPlanWorker worker, ScanRanges scanRanges) { + this.worker = Objects.requireNonNull(worker, "scanRangeParams can not be null"); + this.scanRanges = Objects.requireNonNull(scanRanges, "scanRanges can not be null"); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/Workload.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/Workload.java new file mode 100644 index 00000000000000..faf6ec1c5e824b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/Workload.java @@ -0,0 +1,22 @@ +// 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.trees.plans.distribute.worker; + +/** Workload */ +public interface Workload extends Comparable { +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AbstractUnassignedJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AbstractUnassignedJob.java new file mode 100644 index 00000000000000..f53ee614523379 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AbstractUnassignedJob.java @@ -0,0 +1,74 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.nereids.trees.AbstractTreeNode; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.planner.ExchangeNode; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.planner.ScanNode; + +import com.google.common.collect.ListMultimap; + +import java.util.List; +import java.util.Objects; + +/** AbstractUnassignedJob */ +public abstract class AbstractUnassignedJob + extends AbstractTreeNode implements UnassignedJob { + protected final PlanFragment fragment; + protected final List scanNodes; + protected final ListMultimap exchangeToChildJob; + + /** AbstractUnassignedJob */ + public AbstractUnassignedJob(PlanFragment fragment, List scanNodes, + ListMultimap exchangeToChildJob) { + super(Utils.fastToImmutableList(exchangeToChildJob.values())); + this.fragment = Objects.requireNonNull(fragment, "fragment can not be null"); + this.scanNodes = Utils.fastToImmutableList( + Objects.requireNonNull(scanNodes, "scanNodes can not be null") + ); + this.exchangeToChildJob + = Objects.requireNonNull(exchangeToChildJob, "exchangeToChildJob can not be null"); + } + + @Override + public PlanFragment getFragment() { + return fragment; + } + + @Override + public List getScanNodes() { + return scanNodes; + } + + @Override + public ListMultimap getExchangeToChildJob() { + return exchangeToChildJob; + } + + @Override + public String toString() { + return getClass().getSimpleName(); + } + + @Override + public UnassignedJob withChildren(List children) { + throw new UnsupportedOperationException(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AbstractUnassignedScanJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AbstractUnassignedScanJob.java new file mode 100644 index 00000000000000..92bb059122f9af --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AbstractUnassignedScanJob.java @@ -0,0 +1,202 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; +import org.apache.doris.planner.ExchangeNode; +import org.apache.doris.planner.OlapScanNode; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.planner.ScanNode; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ListMultimap; +import com.google.common.collect.Lists; + +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.atomic.AtomicInteger; + +/** AbstractUnassignedScanJob */ +public abstract class AbstractUnassignedScanJob extends AbstractUnassignedJob { + protected final AtomicInteger shareScanIdGenerator = new AtomicInteger(); + + public AbstractUnassignedScanJob(PlanFragment fragment, + List scanNodes, ListMultimap exchangeToChildJob) { + super(fragment, scanNodes, exchangeToChildJob); + } + + @Override + public List computeAssignedJobs(DistributedPlanWorkerManager workerManager, + ListMultimap inputJobs) { + + Map workerToScanSource = multipleMachinesParallelization( + workerManager, inputJobs); + + return insideMachineParallelization(workerToScanSource, inputJobs); + } + + protected abstract Map multipleMachinesParallelization( + DistributedPlanWorkerManager workerManager, ListMultimap inputJobs); + + protected List insideMachineParallelization( + Map workerToScanRanges, + ListMultimap inputJobs) { + + ConnectContext context = ConnectContext.get(); + boolean useLocalShuffleToAddParallel = useLocalShuffleToAddParallel(workerToScanRanges); + int instanceIndexInFragment = 0; + List instances = Lists.newArrayList(); + for (Entry entry : workerToScanRanges.entrySet()) { + DistributedPlanWorker worker = entry.getKey(); + + // the scanRanges which this worker should scan, + // for example: + // { + // scan tbl1: [tablet_10001, tablet_10002, tablet_10003, tablet_10004] // no instances + // } + ScanSource scanSource = entry.getValue().scanSource; + + // usually, its tablets num, or buckets num + int scanSourceMaxParallel = scanSource.maxParallel(scanNodes); + + // now we should compute how many instances to process the data, + // for example: two instances + int instanceNum = degreeOfParallelism(scanSourceMaxParallel); + + List instanceToScanRanges; + if (useLocalShuffleToAddParallel) { + // only generate one instance to scan all data, in this step + instanceToScanRanges = scanSource.parallelize( + scanNodes, 1 + ); + + // Some tablets too big, we need add parallel to process these tablets after scan, + // for example, use one OlapScanNode to scan data, and use some local instances + // to process Aggregation parallel. We call it `share scan`. Backend will know this + // instances share the same ScanSource, and will not scan same data multiple times. + // + // +-------------------------------- same fragment in one host -------------------------------------+ + // | instance1 instance2 instance3 instance4 | + // | \ \ / / | + // | | + // | OlapScanNode | + // |(share scan node, and local shuffle data to other local instances to parallel compute this data)| + // +------------------------------------------------------------------------------------------------+ + ScanSource shareScanSource = instanceToScanRanges.get(0); + + // one scan range generate multiple instances, + // different instances reference the same scan source + int shareScanId = shareScanIdGenerator.getAndIncrement(); + for (int i = 0; i < instanceNum; i++) { + LocalShuffleAssignedJob instance = new LocalShuffleAssignedJob( + instanceIndexInFragment++, shareScanId, context.nextInstanceId(), + this, worker, shareScanSource); + instances.add(instance); + } + } else { + // split the scanRanges to some partitions, one partition for one instance + // for example: + // [ + // scan tbl1: [tablet_10001, tablet_10003], // instance 1 + // scan tbl1: [tablet_10002, tablet_10004] // instance 2 + // ] + instanceToScanRanges = scanSource.parallelize( + scanNodes, instanceNum + ); + + for (ScanSource instanceToScanRange : instanceToScanRanges) { + instances.add( + assignWorkerAndDataSources( + instanceIndexInFragment++, context.nextInstanceId(), worker, instanceToScanRange + ) + ); + } + } + } + + return instances; + } + + protected boolean useLocalShuffleToAddParallel( + Map workerToScanRanges) { + if (ConnectContext.get() != null && ConnectContext.get().getSessionVariable().isForceToLocalShuffle()) { + return true; + } + return parallelTooLittle(workerToScanRanges); + } + + protected boolean parallelTooLittle(Map workerToScanRanges) { + if (this instanceof UnassignedScanBucketOlapTableJob) { + return scanRangesToLittle(workerToScanRanges) && bucketsTooLittle(workerToScanRanges); + } else if (this instanceof UnassignedScanSingleOlapTableJob + || this instanceof UnassignedScanSingleRemoteTableJob) { + return scanRangesToLittle(workerToScanRanges); + } else { + return false; + } + } + + protected boolean scanRangesToLittle( + Map workerToScanRanges) { + ConnectContext context = ConnectContext.get(); + int backendNum = workerToScanRanges.size(); + for (ScanNode scanNode : scanNodes) { + if (!scanNode.ignoreStorageDataDistribution(context, backendNum)) { + return false; + } + } + return true; + } + + protected int degreeOfParallelism(int maxParallel) { + Preconditions.checkArgument(maxParallel > 0, "maxParallel must be positive"); + if (!fragment.getDataPartition().isPartitioned()) { + return 1; + } + if (scanNodes.size() == 1 && scanNodes.get(0) instanceof OlapScanNode) { + OlapScanNode olapScanNode = (OlapScanNode) scanNodes.get(0); + // if the scan node have limit and no conjuncts, only need 1 instance to save cpu and mem resource, + // e.g. select * from tbl limit 10 + ConnectContext connectContext = ConnectContext.get(); + if (connectContext != null && olapScanNode.shouldUseOneInstance(connectContext)) { + return 1; + } + } + + // the scan instance num should not larger than the tablets num + return Math.min(maxParallel, Math.max(fragment.getParallelExecNum(), 1)); + } + + protected boolean bucketsTooLittle(Map workerToScanRanges) { + int parallelExecNum = fragment.getParallelExecNum(); + for (UninstancedScanSource uninstancedScanSource : workerToScanRanges.values()) { + ScanSource scanSource = uninstancedScanSource.scanSource; + if (scanSource instanceof BucketScanSource) { + BucketScanSource bucketScanSource = (BucketScanSource) scanSource; + int bucketNum = bucketScanSource.bucketIndexToScanNodeToTablets.size(); + if (bucketNum >= parallelExecNum) { + return false; + } + } + } + return true; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AssignedJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AssignedJob.java new file mode 100644 index 00000000000000..f9f6b9dea1451b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AssignedJob.java @@ -0,0 +1,39 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.thrift.TUniqueId; + +/** + * AssignedJob. + * for example: an instance job in a fragment job, which already assign to a worker and some data sources + */ +public interface AssignedJob { + int indexInUnassignedJob(); + + TUniqueId instanceId(); + + UnassignedJob unassignedJob(); + + DistributedPlanWorker getAssignedWorker(); + + ScanSource getScanSource(); + + String toString(boolean showUnassignedJob); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AssignedJobBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AssignedJobBuilder.java new file mode 100644 index 00000000000000..aab6316a3ba164 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/AssignedJobBuilder.java @@ -0,0 +1,64 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.nereids.trees.plans.distribute.worker.BackendDistributedPlanWorkerManager; +import org.apache.doris.planner.ExchangeNode; +import org.apache.doris.planner.PlanFragmentId; + +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ListMultimap; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +/** AssignedJobBuilder */ +public class AssignedJobBuilder { + /** buildJobs */ + public static ListMultimap buildJobs( + Map unassignedJobs) { + BackendDistributedPlanWorkerManager workerManager = new BackendDistributedPlanWorkerManager(); + ListMultimap allAssignedJobs = ArrayListMultimap.create(); + for (Entry kv : unassignedJobs.entrySet()) { + PlanFragmentId fragmentId = kv.getKey(); + UnassignedJob unassignedJob = kv.getValue(); + ListMultimap inputAssignedJobs + = getInputAssignedJobs(unassignedJob, allAssignedJobs); + List fragmentAssignedJobs = + unassignedJob.computeAssignedJobs(workerManager, inputAssignedJobs); + allAssignedJobs.putAll(fragmentId, fragmentAssignedJobs); + } + return allAssignedJobs; + } + + private static ListMultimap getInputAssignedJobs( + UnassignedJob unassignedJob, ListMultimap assignedJobs) { + ListMultimap inputJobs = ArrayListMultimap.create(); + for (Entry> exchangeNodeToChildJobs + : unassignedJob.getExchangeToChildJob().asMap().entrySet()) { + ExchangeNode exchangeNode = exchangeNodeToChildJobs.getKey(); + Collection childJobs = exchangeNodeToChildJobs.getValue(); + for (UnassignedJob childJob : childJobs) { + inputJobs.putAll(exchangeNode, assignedJobs.get(childJob.getFragment().getFragmentId())); + } + } + return inputJobs; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/BucketScanSource.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/BucketScanSource.java new file mode 100644 index 00000000000000..33d066a02b9fcb --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/BucketScanSource.java @@ -0,0 +1,148 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.common.util.ListUtil; +import org.apache.doris.planner.ScanNode; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +/** BucketScanSource */ +public class BucketScanSource extends ScanSource { + // for example: + // 1. bucket 0 use OlapScanNode(tableName=`tbl`) to scan with tablet: [tablet 10001, tablet 10003] + // 2. bucket 1 use OlapScanNode(tableName=`tbl`) to scan with tablet: [tablet 10002, tablet 10004] + public final Map> bucketIndexToScanNodeToTablets; + + public BucketScanSource(Map> bucketIndexToScanNodeToTablets) { + this.bucketIndexToScanNodeToTablets = bucketIndexToScanNodeToTablets; + } + + @Override + public int maxParallel(List scanNodes) { + // maxParallel is buckets num + return bucketIndexToScanNodeToTablets.size(); + } + + @Override + public List parallelize(List scanNodes, int instanceNum) { + // current state, no any instance, we only known how many buckets + // this worker should process, and the data that this buckets should process: + // + // [ + // bucket 0: { + // scanNode1: ScanRanges([tablet_10001, tablet_10004, tablet_10007]), + // scanNode2: ScanRanges([tablet_10010, tablet_10013, tablet_10016]) + // }, + // bucket 1: { + // scanNode1: ScanRanges([tablet_10002, tablet_10005, tablet_10008]), + // scanNode2: ScanRanges([tablet_10011, tablet_10014, tablet_10017]) + // }, + // bucket 3: { + // scanNode1: ScanRanges([tablet_10003, tablet_10006, tablet_10009]), + // scanNode2: ScanRanges([tablet_10012, tablet_10015, tablet_10018]) + // } + // ] + List>> bucketIndexToScanRanges + = Lists.newArrayList(bucketIndexToScanNodeToTablets.entrySet()); + + // separate buckets to instanceNum groups. + // for example: + // [ + // // instance 1 process two buckets + // [ + // bucket 0: { + // scanNode1: ScanRanges([tablet_10001, tablet_10004, tablet_10007]), + // scanNode2: ScanRanges([tablet_10010, tablet_10013, tablet_10016]) + // }, + // bucket 3: { + // scanNode1: ScanRanges([tablet_10003, tablet_10006, tablet_10009]), + // scanNode2: ScanRanges([tablet_10012, tablet_10015, tablet_10018]) + // } + // ], + // // instance 2 process one bucket + // [ + // bucket 1: { + // scanNode1: ScanRanges([tablet_10002, tablet_10005, tablet_10008]), + // scanNode2: ScanRanges([tablet_10011, tablet_10014, tablet_10017]) + // } + // ] + // ] + List>>> scanBucketsPerInstance + = ListUtil.splitBySize(bucketIndexToScanRanges, instanceNum); + + // rebuild BucketScanSource for each instance + ImmutableList.Builder instancesScanSource = ImmutableList.builder(); + for (List>> oneInstanceScanBuckets : scanBucketsPerInstance) { + Map> bucketsScanSources = Maps.newLinkedHashMap(); + for (Entry> bucketIndexToScanNodeToScanRange : oneInstanceScanBuckets) { + Integer bucketIndex = bucketIndexToScanNodeToScanRange.getKey(); + Map scanNodeToScanRanges = bucketIndexToScanNodeToScanRange.getValue(); + bucketsScanSources.put(bucketIndex, scanNodeToScanRanges); + } + + instancesScanSource.add(new BucketScanSource( + bucketsScanSources + )); + } + return instancesScanSource.build(); + } + + /** getBucketIndexToScanRanges */ + public Map getBucketIndexToScanRanges(ScanNode scanNode) { + Map bucketIndexToScanRanges = Maps.newLinkedHashMap(); + for (Entry> entry : bucketIndexToScanNodeToTablets.entrySet()) { + Integer bucketIndex = entry.getKey(); + Map scanNodeToScanRanges = entry.getValue(); + ScanRanges scanRanges = scanNodeToScanRanges.get(scanNode); + if (scanRanges != null) { + bucketIndexToScanRanges.put(bucketIndex, scanRanges); + } + } + + return bucketIndexToScanRanges; + } + + /** toString */ + public void toString(StringBuilder str, String prefix) { + int i = 0; + String nextIndent = prefix + " "; + str.append("[\n"); + for (Entry> entry : bucketIndexToScanNodeToTablets.entrySet()) { + Integer bucketId = entry.getKey(); + Map scanNodeToScanRanges = entry.getValue(); + str.append(prefix).append(" bucket ").append(bucketId).append(": "); + DefaultScanSource.toString(scanNodeToScanRanges, str, nextIndent); + if (++i < bucketIndexToScanNodeToTablets.size()) { + str.append(",\n"); + } + } + str.append("\n").append(prefix).append("]"); + } + + @Override + public boolean isEmpty() { + return bucketIndexToScanNodeToTablets.isEmpty(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/CustomAssignmentJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/CustomAssignmentJob.java new file mode 100644 index 00000000000000..dc009dba09acef --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/CustomAssignmentJob.java @@ -0,0 +1,29 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.planner.ExchangeNode; + +import com.google.common.collect.ListMultimap; + +import java.util.List; + +/** CustomAssignmentJob */ +public interface CustomAssignmentJob { + List customAssignment(ListMultimap inputJobs); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/DefaultScanSource.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/DefaultScanSource.java new file mode 100644 index 00000000000000..89e5270801dcb4 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/DefaultScanSource.java @@ -0,0 +1,111 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.planner.ScanNode; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +/** DefaultScanSource */ +public class DefaultScanSource extends ScanSource { + // for example: + // 1. use OlapScanNode(tableName=`tbl1`) to scan with tablet: [tablet 10001, tablet 10002] + // 2. use OlapScanNode(tableName=`tbl2`) to scan with tablet: [tablet 10003, tablet 10004] + public final Map scanNodeToScanRanges; + + public DefaultScanSource(Map scanNodeToScanRanges) { + this.scanNodeToScanRanges = scanNodeToScanRanges; + } + + @Override + public int maxParallel(List scanNodes) { + if (scanNodes.size() == 1) { + ScanRanges scanRanges = scanNodeToScanRanges.get(scanNodes.get(0)); + if (scanRanges != null) { + // max parallel is the scan ranges(tablets) num + return Math.max(scanRanges.params.size(), 1); + } + } + return 1; + } + + @Override + public List parallelize(List scanNodes, int instanceNum) { + Preconditions.checkArgument(scanNodes.size() == 1, + "Only support parallelize one ScanNode, but meet " + scanNodes.size() + " ScanNodes"); + + ScanNode scanNode = scanNodes.get(0); + ScanRanges scanRanges = scanNodeToScanRanges.get(scanNode); + if (scanRanges == null) { + return ImmutableList.of(); + } + + List scanRangesPerInstance = scanRanges.split(instanceNum); + + ImmutableList.Builder instancesSource + = ImmutableList.builderWithExpectedSize(scanRangesPerInstance.size()); + for (ScanRanges oneInstanceScanRanges : scanRangesPerInstance) { + DefaultScanSource oneInstanceScanSource + = new DefaultScanSource(ImmutableMap.of(scanNode, oneInstanceScanRanges)); + instancesSource.add(oneInstanceScanSource); + } + return instancesSource.build(); + } + + @Override + public boolean isEmpty() { + return scanNodeToScanRanges.isEmpty(); + } + + @Override + public void toString(StringBuilder str, String prefix) { + toString(scanNodeToScanRanges, str, prefix); + } + + /** toString */ + public static void toString(Map scanNodeToScanRanges, StringBuilder str, String prefix) { + if (scanNodeToScanRanges.isEmpty()) { + str.append("[]"); + return; + } + int i = 0; + String nextIndent = prefix + " "; + str.append("[\n"); + for (Entry entry : scanNodeToScanRanges.entrySet()) { + ScanNode scanNode = entry.getKey(); + ScanRanges scanRanges = entry.getValue(); + str.append(prefix).append(" {\n") + .append(prefix).append(" scanNode: ").append(scanNode).append(",\n") + .append(prefix).append(" scanRanges: "); + + scanRanges.toString(str, nextIndent); + str.append("\n").append(prefix).append(" }"); + + if (++i < scanNodeToScanRanges.size()) { + str.append(",\n"); + } + } + str.append("\n").append(prefix).append("]"); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/LocalShuffleAssignedJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/LocalShuffleAssignedJob.java new file mode 100644 index 00000000000000..50e43fc0282755 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/LocalShuffleAssignedJob.java @@ -0,0 +1,43 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.collect.ImmutableMap; + +import java.util.Map; + +/** LocalShuffleAssignedJob */ +public class LocalShuffleAssignedJob extends StaticAssignedJob { + public final int shareScanId; + + public LocalShuffleAssignedJob( + int indexInUnassignedJob, int shareScanId, TUniqueId instanceId, + UnassignedJob unassignedJob, + DistributedPlanWorker worker, ScanSource scanSource) { + super(indexInUnassignedJob, instanceId, unassignedJob, worker, scanSource); + this.shareScanId = shareScanId; + } + + @Override + protected Map extraInfo() { + return ImmutableMap.of("shareScanIndex", String.valueOf(shareScanId)); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/ScanRange.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/ScanRange.java new file mode 100644 index 00000000000000..24bf7a6d910a5f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/ScanRange.java @@ -0,0 +1,23 @@ +// 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.trees.plans.distribute.worker.job; + +/** ScanRange */ +public class ScanRange { + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/ScanRanges.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/ScanRanges.java new file mode 100644 index 00000000000000..2539b8416dea08 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/ScanRanges.java @@ -0,0 +1,109 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.thrift.TPaloScanRange; +import org.apache.doris.thrift.TScanRangeParams; + +import com.google.common.collect.Lists; + +import java.util.List; + +/**ScanRanges */ +public class ScanRanges implements Splittable { + // usually, it's tablets + public final List params; + // size corresponding to tablets one by one + public final List bytes; + public long totalBytes; + + public ScanRanges() { + this(Lists.newArrayList(), Lists.newArrayList()); + } + + /** ScanRanges */ + public ScanRanges(List params, List bytes) { + this.params = params; + this.bytes = bytes; + long totalBytes = 0; + for (Long size : bytes) { + totalBytes += size; + } + this.totalBytes = totalBytes; + } + + public void addScanRanges(ScanRanges scanRanges) { + this.params.addAll(scanRanges.params); + this.bytes.addAll(scanRanges.bytes); + this.totalBytes += scanRanges.totalBytes; + } + + public void addScanRange(TScanRangeParams params, long bytes) { + this.params.add(params); + this.bytes.add(bytes); + this.totalBytes += bytes; + } + + @Override + public int itemSize() { + return params.size(); + } + + @Override + public void addItem(ScanRanges other, int index) { + addScanRange(other.params.get(index), other.bytes.get(index)); + } + + @Override + public ScanRanges newSplittable() { + return new ScanRanges(); + } + + @Override + public String toString() { + StringBuilder str = new StringBuilder(); + toString(str, ""); + return str.toString(); + } + + /** toString */ + public void toString(StringBuilder str, String prefix) { + str.append("ScanRanges(bytes: " + totalBytes + ", ranges: ["); + if (params.isEmpty()) { + str.append("])"); + return; + } + str.append("\n"); + for (int i = 0; i < params.size(); i++) { + str.append(prefix).append(" " + toString(params.get(i)) + ", bytes: " + bytes.get(i)); + if (i + 1 < params.size()) { + str.append(",\n"); + } + } + str.append("\n").append(prefix).append("])"); + } + + private String toString(TScanRangeParams scanRange) { + TPaloScanRange paloScanRange = scanRange.getScanRange().getPaloScanRange(); + if (paloScanRange != null) { + return "tablet " + paloScanRange.getTabletId(); + } else { + return scanRange.toString(); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/ScanSource.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/ScanSource.java new file mode 100644 index 00000000000000..b124e14bd73c77 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/ScanSource.java @@ -0,0 +1,41 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.planner.ScanNode; + +import java.util.List; + +/** ScanSource */ +public abstract class ScanSource { + + public abstract int maxParallel(List scanNodes); + + public abstract List parallelize(List scanNodes, int instanceNum); + + public abstract boolean isEmpty(); + + public abstract void toString(StringBuilder str, String prefix); + + @Override + public String toString() { + StringBuilder str = new StringBuilder(); + toString(str, ""); + return str.toString(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/Splittable.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/Splittable.java new file mode 100644 index 00000000000000..269dcedfb868f9 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/Splittable.java @@ -0,0 +1,69 @@ +// 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.trees.plans.distribute.worker.job; + +import com.google.common.base.Preconditions; + +import java.util.ArrayList; +import java.util.List; + +/** Splittable */ +public interface Splittable> { + + int itemSize(); + + void addItem(S other, int index); + + S newSplittable(); + + default List split(int splitSize) { + return Splittable.split(this, splitSize); + } + + /** + * split a list to multi expected number sublist + * for example: + * + * list is : [1, 2, 3, 4, 5, 6, 7] + * expectedSize is : 3 + * + * return : + * [1, 4, 7] + * [2, 5] + * [3, 6] + */ + static > List split(Splittable splittable, int splitSize) { + Preconditions.checkNotNull(splittable, "splittable must not be null"); + Preconditions.checkArgument(splitSize > 0, "splitSize must larger than 0"); + + int itemSize = splittable.itemSize(); + splitSize = Math.min(splitSize, itemSize); + + List result = new ArrayList<>(splitSize); + for (int i = 0; i < splitSize; i++) { + result.add(splittable.newSplittable()); + } + + int index = 0; + for (int i = 0; i < itemSize; i++) { + result.get(index).addItem((S) splittable, i); + index = (index + 1) % splitSize; + } + return result; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/StaticAssignedJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/StaticAssignedJob.java new file mode 100644 index 00000000000000..1a92cf71019e66 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/StaticAssignedJob.java @@ -0,0 +1,106 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.common.util.DebugUtil; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.collect.ImmutableMap; + +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; + +/** StaticAssignedJob */ +public class StaticAssignedJob implements AssignedJob { + private final int indexInUnassignedJob; + private final UnassignedJob unassignedJob; + private final TUniqueId instanceId; + private final DistributedPlanWorker worker; + private final ScanSource scanSource; + + public StaticAssignedJob( + int indexInUnassignedJob, TUniqueId instanceId, UnassignedJob unassignedJob, DistributedPlanWorker worker, + ScanSource scanSource) { + this.indexInUnassignedJob = indexInUnassignedJob; + this.instanceId = Objects.requireNonNull(instanceId, "instanceId can not be null"); + this.unassignedJob = Objects.requireNonNull(unassignedJob, "unassignedJob can not be null"); + this.worker = worker; + this.scanSource = Objects.requireNonNull(scanSource, "scanSource can not be null"); + } + + @Override + public int indexInUnassignedJob() { + return indexInUnassignedJob; + } + + @Override + public TUniqueId instanceId() { + return instanceId; + } + + @Override + public UnassignedJob unassignedJob() { + return unassignedJob; + } + + @Override + public DistributedPlanWorker getAssignedWorker() { + return worker; + } + + @Override + public ScanSource getScanSource() { + return scanSource; + } + + @Override + public String toString() { + return toString(true); + } + + @Override + public String toString(boolean showUnassignedJob) { + StringBuilder scanSourceString = new StringBuilder(); + if (!scanSource.isEmpty()) { + scanSource.toString(scanSourceString, " "); + } else { + scanSourceString = new StringBuilder("[]"); + } + StringBuilder str = new StringBuilder(getClass().getSimpleName()).append("("); + if (showUnassignedJob) { + str.append("\n unassignedJob: ").append(unassignedJob).append(","); + } + str.append("\n index: " + indexInUnassignedJob) + .append(",\n instanceId: " + DebugUtil.printId(instanceId)) + .append(",\n worker: " + worker); + for (Entry kv : extraInfo().entrySet()) { + str.append(",\n ").append(kv.getKey()).append(": ").append(kv.getValue()); + } + + return str + .append(",\n scanSource: " + scanSourceString) + .append("\n)") + .toString(); + } + + protected Map extraInfo() { + return ImmutableMap.of(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedGatherScanMultiRemoteTablesJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedGatherScanMultiRemoteTablesJob.java new file mode 100644 index 00000000000000..558af8844974fc --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedGatherScanMultiRemoteTablesJob.java @@ -0,0 +1,87 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; +import org.apache.doris.nereids.trees.plans.distribute.worker.ScanWorkerSelector; +import org.apache.doris.planner.DataGenScanNode; +import org.apache.doris.planner.ExchangeNode; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.planner.ScanNode; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.thrift.TScanRangeLocations; +import org.apache.doris.thrift.TScanRangeParams; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ListMultimap; +import com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; + +/** UnassignedGatherScanMultiRemoteTablesJob */ +public class UnassignedGatherScanMultiRemoteTablesJob extends AbstractUnassignedJob { + + public UnassignedGatherScanMultiRemoteTablesJob(PlanFragment fragment, + List scanNodes, ListMultimap exchangeToChildJob) { + super(fragment, scanNodes, exchangeToChildJob); + } + + /** canApply */ + public static boolean canApply(List scanNodes) { + if (scanNodes.size() <= 1) { + return false; + } + for (ScanNode scanNode : scanNodes) { + if (!(scanNode instanceof DataGenScanNode)) { + return false; + } + DataGenScanNode dataGenScanNode = (DataGenScanNode) scanNode; + if (dataGenScanNode.getScanRangeLocations(0).size() != 1) { + return false; + } + } + return true; + } + + @Override + public List computeAssignedJobs(DistributedPlanWorkerManager workerManager, + ListMultimap inputJobs) { + ConnectContext context = ConnectContext.get(); + Map scanNodeToScanRanges = Maps.newLinkedHashMap(); + for (ScanNode scanNode : scanNodes) { + List scanRangeLocations = scanNode.getScanRangeLocations(0); + ScanRanges scanRanges = new ScanRanges(); + for (TScanRangeLocations scanRangeLocation : scanRangeLocations) { + TScanRangeParams replica = ScanWorkerSelector.buildScanReplicaParams( + scanRangeLocation, scanRangeLocation.locations.get(0)); + scanRanges.addScanRange(replica, 0); + } + + scanNodeToScanRanges.put(scanNode, scanRanges); + } + + DistributedPlanWorker randomWorker = workerManager.randomAvailableWorker(); + return ImmutableList.of( + assignWorkerAndDataSources(0, context.nextInstanceId(), + randomWorker, new DefaultScanSource(scanNodeToScanRanges) + ) + ); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJob.java new file mode 100644 index 00000000000000..d142460ea2db02 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJob.java @@ -0,0 +1,52 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.nereids.trees.TreeNode; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; +import org.apache.doris.planner.ExchangeNode; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.planner.ScanNode; +import org.apache.doris.thrift.TUniqueId; + +import com.google.common.collect.ListMultimap; + +import java.util.List; + +/** + * WorkerJob. + * for example: a fragment job, which doesn't parallelization to some instance jobs and also no worker to invoke it + */ +public interface UnassignedJob extends TreeNode { + PlanFragment getFragment(); + + List getScanNodes(); + + ListMultimap getExchangeToChildJob(); + + List computeAssignedJobs( + DistributedPlanWorkerManager workerManager, ListMultimap inputJobs); + + // generate an instance job + // e.g. build an instance job by a backends and the replica ids it contains + default AssignedJob assignWorkerAndDataSources( + int instanceIndexInFragment, TUniqueId instanceId, DistributedPlanWorker worker, ScanSource scanSource) { + return new StaticAssignedJob(instanceIndexInFragment, instanceId, this, worker, scanSource); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJobBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJobBuilder.java new file mode 100644 index 00000000000000..c5fadc8fb8df57 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJobBuilder.java @@ -0,0 +1,266 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.nereids.trees.plans.distribute.FragmentIdMapping; +import org.apache.doris.nereids.trees.plans.distribute.worker.LoadBalanceScanWorkerSelector; +import org.apache.doris.nereids.trees.plans.distribute.worker.ScanWorkerSelector; +import org.apache.doris.planner.DataSink; +import org.apache.doris.planner.DataStreamSink; +import org.apache.doris.planner.ExchangeNode; +import org.apache.doris.planner.MultiCastDataSink; +import org.apache.doris.planner.OlapScanNode; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.planner.PlanFragmentId; +import org.apache.doris.planner.PlanNodeId; +import org.apache.doris.planner.ScanNode; +import org.apache.doris.qe.ConnectContext; +import org.apache.doris.qe.SessionVariable; + +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ListMultimap; +import com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +/** + * UnassignedJobBuilder. + * build UnassignedJob by fragment + */ +public class UnassignedJobBuilder { + private final ScanWorkerSelector scanWorkerSelector = new LoadBalanceScanWorkerSelector(); + + /** + * build job from fragment. + */ + public static FragmentIdMapping buildJobs(FragmentIdMapping fragments) { + UnassignedJobBuilder builder = new UnassignedJobBuilder(); + + FragmentLineage fragmentLineage = buildFragmentLineage(fragments); + FragmentIdMapping unassignedJobs = new FragmentIdMapping<>(); + + // build from leaf to parent + for (Entry kv : fragments.entrySet()) { + PlanFragmentId fragmentId = kv.getKey(); + PlanFragment fragment = kv.getValue(); + + ListMultimap inputJobs = findInputJobs( + fragmentLineage, fragmentId, unassignedJobs); + UnassignedJob unassignedJob = builder.buildJob(fragment, inputJobs); + unassignedJobs.put(fragmentId, unassignedJob); + } + return unassignedJobs; + } + + private UnassignedJob buildJob( + PlanFragment planFragment, ListMultimap inputJobs) { + List scanNodes = collectScanNodesInThisFragment(planFragment); + if (planFragment.specifyInstances.isPresent()) { + return buildSpecifyInstancesJob(planFragment, scanNodes, inputJobs); + } else if (!scanNodes.isEmpty() || isLeafFragment(planFragment)) { + return buildLeafOrScanJob(planFragment, scanNodes, inputJobs); + } else { + return buildShuffleJob(planFragment, inputJobs); + } + } + + private UnassignedJob buildLeafOrScanJob( + PlanFragment planFragment, List scanNodes, + ListMultimap inputJobs) { + int olapScanNodeNum = olapScanNodeNum(scanNodes); + + UnassignedJob unassignedJob = null; + if (!scanNodes.isEmpty() && olapScanNodeNum == scanNodes.size()) { + // we need assign a backend which contains the data, + // so that the OlapScanNode can find the data in the backend + // e.g. select * from olap_table + unassignedJob = buildScanOlapTableJob(planFragment, (List) scanNodes, inputJobs, scanWorkerSelector); + } else if (scanNodes.isEmpty()) { + // select constant without table, + // e.g. select 100 union select 200 + unassignedJob = buildQueryConstantJob(planFragment); + } else if (olapScanNodeNum == 0) { + // only scan external tables or cloud tables or table valued functions + // e,g. select * from numbers('number'='100') + unassignedJob = buildScanRemoteTableJob(planFragment, scanNodes, inputJobs, scanWorkerSelector); + } + + if (unassignedJob != null) { + return unassignedJob; + } + + throw new IllegalStateException( + "Unsupported fragment which contains multiple scan nodes and some of them are not OlapScanNode" + ); + } + + private UnassignedJob buildSpecifyInstancesJob( + PlanFragment planFragment, List scanNodes, ListMultimap inputJobs) { + return new UnassignedSpecifyInstancesJob(planFragment, scanNodes, inputJobs); + } + + private UnassignedJob buildScanOlapTableJob( + PlanFragment planFragment, List olapScanNodes, + ListMultimap inputJobs, + ScanWorkerSelector scanWorkerSelector) { + if (shouldAssignByBucket(planFragment)) { + return new UnassignedScanBucketOlapTableJob( + planFragment, olapScanNodes, inputJobs, scanWorkerSelector); + } else if (olapScanNodes.size() == 1) { + return new UnassignedScanSingleOlapTableJob( + planFragment, olapScanNodes.get(0), inputJobs, scanWorkerSelector); + } else { + throw new IllegalStateException("Not supported multiple scan multiple " + + "OlapTable but not contains colocate join or bucket shuffle join"); + } + } + + private List collectScanNodesInThisFragment(PlanFragment planFragment) { + return planFragment.getPlanRoot().collectInCurrentFragment(ScanNode.class::isInstance); + } + + private int olapScanNodeNum(List scanNodes) { + int olapScanNodeNum = 0; + for (ScanNode scanNode : scanNodes) { + if (scanNode instanceof OlapScanNode) { + olapScanNodeNum++; + } + } + return olapScanNodeNum; + } + + private boolean isLeafFragment(PlanFragment planFragment) { + return planFragment.getChildren().isEmpty(); + } + + private UnassignedQueryConstantJob buildQueryConstantJob(PlanFragment planFragment) { + return new UnassignedQueryConstantJob(planFragment); + } + + private UnassignedJob buildScanRemoteTableJob( + PlanFragment planFragment, List scanNodes, + ListMultimap inputJobs, + ScanWorkerSelector scanWorkerSelector) { + if (scanNodes.size() == 1) { + return new UnassignedScanSingleRemoteTableJob( + planFragment, scanNodes.get(0), inputJobs, scanWorkerSelector); + } else if (UnassignedGatherScanMultiRemoteTablesJob.canApply(scanNodes)) { + // select * from numbers("number" = "10") a union all select * from numbers("number" = "20") b; + // use an instance to scan table a and table b + return new UnassignedGatherScanMultiRemoteTablesJob(planFragment, scanNodes, inputJobs); + } else { + return null; + } + } + + private UnassignedShuffleJob buildShuffleJob( + PlanFragment planFragment, ListMultimap inputJobs) { + return new UnassignedShuffleJob(planFragment, inputJobs); + } + + private static ListMultimap findInputJobs( + FragmentLineage lineage, PlanFragmentId fragmentId, FragmentIdMapping unassignedJobs) { + ListMultimap inputJobs = ArrayListMultimap.create(); + Map exchangeNodes = lineage.parentFragmentToExchangeNode.get(fragmentId); + if (exchangeNodes != null) { + for (Entry idToExchange : exchangeNodes.entrySet()) { + PlanNodeId exchangeId = idToExchange.getKey(); + ExchangeNode exchangeNode = idToExchange.getValue(); + List childFragmentIds = lineage.exchangeToChildFragment.get(exchangeId); + for (PlanFragmentId childFragmentId : childFragmentIds) { + inputJobs.put(exchangeNode, unassignedJobs.get(childFragmentId)); + } + } + } + return inputJobs; + } + + private static List collectExchangeNodesInThisFragment(PlanFragment planFragment) { + return planFragment + .getPlanRoot() + .collectInCurrentFragment(ExchangeNode.class::isInstance); + } + + private static FragmentLineage buildFragmentLineage( + FragmentIdMapping fragments) { + ListMultimap exchangeToChildFragment = ArrayListMultimap.create(); + FragmentIdMapping> parentFragmentToExchangeNode = new FragmentIdMapping<>(); + + for (PlanFragment fragment : fragments.values()) { + PlanFragmentId fragmentId = fragment.getFragmentId(); + + // 1. link child fragment to exchange node + DataSink sink = fragment.getSink(); + if (sink instanceof DataStreamSink) { + PlanNodeId exchangeNodeId = sink.getExchNodeId(); + exchangeToChildFragment.put(exchangeNodeId, fragmentId); + } else if (sink instanceof MultiCastDataSink) { + MultiCastDataSink multiCastDataSink = (MultiCastDataSink) sink; + for (DataStreamSink dataStreamSink : multiCastDataSink.getDataStreamSinks()) { + PlanNodeId exchangeNodeId = dataStreamSink.getExchNodeId(); + exchangeToChildFragment.put(exchangeNodeId, fragmentId); + } + } + + // 2. link parent fragment to exchange node + List exchangeNodes = collectExchangeNodesInThisFragment(fragment); + Map exchangeNodesInFragment = Maps.newLinkedHashMap(); + for (ExchangeNode exchangeNode : exchangeNodes) { + exchangeNodesInFragment.put(exchangeNode.getId(), exchangeNode); + } + parentFragmentToExchangeNode.put(fragmentId, exchangeNodesInFragment); + } + + return new FragmentLineage(parentFragmentToExchangeNode, exchangeToChildFragment); + } + + private static boolean shouldAssignByBucket(PlanFragment fragment) { + if (fragment.hasColocatePlanNode()) { + return true; + } + if (enableBucketShuffleJoin() && fragment.hasBucketShuffleJoin()) { + return true; + } + return false; + } + + private static boolean enableBucketShuffleJoin() { + if (ConnectContext.get() != null) { + SessionVariable sessionVariable = ConnectContext.get().getSessionVariable(); + if (!sessionVariable.isEnableBucketShuffleJoin() && !sessionVariable.isEnableNereidsPlanner()) { + return false; + } + } + return true; + } + + // the class support find exchange nodes in the fragment, and find child fragment by exchange node id + private static class FragmentLineage { + private final FragmentIdMapping> parentFragmentToExchangeNode; + private final ListMultimap exchangeToChildFragment; + + public FragmentLineage( + FragmentIdMapping> parentFragmentToExchangeNode, + ListMultimap exchangeToChildFragment) { + this.parentFragmentToExchangeNode = parentFragmentToExchangeNode; + this.exchangeToChildFragment = exchangeToChildFragment; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedQueryConstantJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedQueryConstantJob.java new file mode 100644 index 00000000000000..bfbafd0739065c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedQueryConstantJob.java @@ -0,0 +1,50 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; +import org.apache.doris.planner.ExchangeNode; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ListMultimap; + +import java.util.List; + +/** UnassignedQueryConstantJob */ +public class UnassignedQueryConstantJob extends AbstractUnassignedJob { + public UnassignedQueryConstantJob(PlanFragment fragment) { + super(fragment, ImmutableList.of(), ArrayListMultimap.create()); + } + + @Override + public List computeAssignedJobs(DistributedPlanWorkerManager workerManager, + ListMultimap inputJobs) { + DistributedPlanWorker randomWorker = workerManager.randomAvailableWorker(); + ConnectContext context = ConnectContext.get(); + return ImmutableList.of( + new StaticAssignedJob(0, context.nextInstanceId(), this, + randomWorker, new DefaultScanSource(ImmutableMap.of()) + ) + ); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanBucketOlapTableJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanBucketOlapTableJob.java new file mode 100644 index 00000000000000..7a14d5a2f1cafa --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanBucketOlapTableJob.java @@ -0,0 +1,310 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.analysis.JoinOperator; +import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.Partition; +import org.apache.doris.catalog.Replica; +import org.apache.doris.catalog.Tablet; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; +import org.apache.doris.nereids.trees.plans.distribute.worker.ScanWorkerSelector; +import org.apache.doris.planner.ExchangeNode; +import org.apache.doris.planner.HashJoinNode; +import org.apache.doris.planner.OlapScanNode; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.planner.ScanNode; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ListMultimap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.Set; +import java.util.TreeSet; + +/** + * UnassignedScanBucketOlapTableJob. + * bucket shuffle join olap table, or colocate join olap table + */ +public class UnassignedScanBucketOlapTableJob extends AbstractUnassignedScanJob { + private final ScanWorkerSelector scanWorkerSelector; + private final List olapScanNodes; + + /** UnassignedScanNativeTableJob */ + public UnassignedScanBucketOlapTableJob( + PlanFragment fragment, List olapScanNodes, + ListMultimap exchangeToChildJob, + ScanWorkerSelector scanWorkerSelector) { + super(fragment, (List) olapScanNodes, exchangeToChildJob); + this.scanWorkerSelector = Objects.requireNonNull( + scanWorkerSelector, "scanWorkerSelector cat not be null"); + + Preconditions.checkArgument(!olapScanNodes.isEmpty(), "OlapScanNode is empty"); + this.olapScanNodes = olapScanNodes; + } + + public List getOlapScanNodes() { + return olapScanNodes; + } + + @Override + protected Map multipleMachinesParallelization( + DistributedPlanWorkerManager workerManager, ListMultimap inputJobs) { + // for every bucket tablet, select its replica and worker. + // for example, colocate join: + // { + // BackendWorker("172.0.0.1"): { + // bucket 0: { + // olapScanNode1: ScanRanges([tablet_10001, tablet_10002, tablet_10003, tablet_10004]), + // olapScanNode2: ScanRanges([tablet_10009, tablet_10010, tablet_10011, tablet_10012]) + // }, + // bucket 1: { + // olapScanNode1: ScanRanges([tablet_10005, tablet_10006, tablet_10007, tablet_10008]) + // olapScanNode2: ScanRanges([tablet_10013, tablet_10014, tablet_10015, tablet_10016]) + // }, + // ... + // }, + // BackendWorker("172.0.0.2"): { + // ... + // } + // } + return scanWorkerSelector.selectReplicaAndWorkerWithBucket(this); + } + + @Override + protected List insideMachineParallelization( + Map workerToScanRanges, + ListMultimap inputJobs) { + // separate buckets to instanceNum groups, let one instance process some buckets. + // for example, colocate join: + // { + // // 172.0.0.1 has two instances + // BackendWorker("172.0.0.1"): [ + // // instance 1 process two buckets + // { + // bucket 0: { + // olapScanNode1: ScanRanges([tablet_10001, tablet_10002, tablet_10003, tablet_10004]), + // olapScanNode2: ScanRanges([tablet_10009, tablet_10010, tablet_10011, tablet_10012]) + // }, + // bucket 1: { + // olapScanNode1: ScanRanges([tablet_10005, tablet_10006, tablet_10007, tablet_10008]) + // olapScanNode2: ScanRanges([tablet_10013, tablet_10014, tablet_10015, tablet_10016]) + // } + // }, + // // instance 1 process one bucket + // { + // bucket 3: ... + // } + // ] + // // instance 4... in "172.0.0.1" + // BackendWorker("172.0.0.2"): [ + // ... + // ], + // ... + // } + List assignedJobs = super.insideMachineParallelization(workerToScanRanges, inputJobs); + + // the case: + // ```sql + // SELECT * FROM + // (select * from tbl1 where c0 =1)a + // RIGHT OUTER JOIN + // (select * from tbl2)b + // ON a.id = b.id; + // ``` + // contains right outer join and missing instance in left side because of tablet pruner, for example + // left: [bucket 1] + // right: [bucket 1, bucket 2] + // + // we should join buckets corresponding: + // [ + // (left bucket 1) right outer join (right bucket 1) + // (no any machine) right outer join (right bucket 2) + // ] + // if missing the left bucket 2, it will compute an empty result + // because right bucket 2 doesn't exist destination instance, + // so we should fill up this instance + List hashJoinNodes = fragment.getPlanRoot() + .collectInCurrentFragment(HashJoinNode.class::isInstance); + if (shouldFillUpInstances(hashJoinNodes)) { + return fillUpInstances(assignedJobs); + } + + return assignedJobs; + } + + private boolean shouldFillUpInstances(List hashJoinNodes) { + for (HashJoinNode hashJoinNode : hashJoinNodes) { + if (!hashJoinNode.isBucketShuffle()) { + continue; + } + JoinOperator joinOp = hashJoinNode.getJoinOp(); + switch (joinOp) { + case INNER_JOIN: + case CROSS_JOIN: + break; + default: + return true; + } + } + return false; + } + + private List fillUpInstances(List instances) { + Set missingBucketIndexes = missingBuckets(instances); + if (missingBucketIndexes.isEmpty()) { + return instances; + } + + ConnectContext context = ConnectContext.get(); + + OlapScanNode olapScanNode = (OlapScanNode) scanNodes.get(0); + MaterializedIndex randomPartition = randomPartition(olapScanNode); + ListMultimap missingBuckets = selectWorkerForMissingBuckets( + olapScanNode, randomPartition, missingBucketIndexes); + + boolean useLocalShuffle = instances.stream().anyMatch(LocalShuffleAssignedJob.class::isInstance); + List newInstances = new ArrayList<>(instances); + for (Entry> workerToBuckets : missingBuckets.asMap().entrySet()) { + Map> scanEmptyBuckets = Maps.newLinkedHashMap(); + for (Integer bucketIndex : workerToBuckets.getValue()) { + Map scanTableWithEmptyData = Maps.newLinkedHashMap(); + for (ScanNode scanNode : scanNodes) { + scanTableWithEmptyData.put(scanNode, new ScanRanges()); + } + scanEmptyBuckets.put(bucketIndex, scanTableWithEmptyData); + } + + AssignedJob fillUpInstance = null; + DistributedPlanWorker worker = workerToBuckets.getKey(); + BucketScanSource scanSource = new BucketScanSource(scanEmptyBuckets); + if (useLocalShuffle) { + // when use local shuffle, we should ensure every backend only process one instance! + // so here we should try to merge the missing buckets into exist instances + boolean mergedBucketsInSameWorkerInstance = false; + for (AssignedJob newInstance : newInstances) { + if (newInstance.getAssignedWorker().equals(worker)) { + BucketScanSource bucketScanSource = (BucketScanSource) newInstance.getScanSource(); + bucketScanSource.bucketIndexToScanNodeToTablets.putAll(scanEmptyBuckets); + mergedBucketsInSameWorkerInstance = true; + } + } + if (!mergedBucketsInSameWorkerInstance) { + fillUpInstance = new LocalShuffleAssignedJob( + newInstances.size(), shareScanIdGenerator.getAndIncrement(), + context.nextInstanceId(), this, worker, scanSource + ); + } + } else { + fillUpInstance = assignWorkerAndDataSources( + newInstances.size(), context.nextInstanceId(), worker, scanSource + ); + } + if (fillUpInstance != null) { + newInstances.add(fillUpInstance); + } + } + return newInstances; + } + + private int fullBucketNum() { + for (ScanNode scanNode : scanNodes) { + if (scanNode instanceof OlapScanNode) { + return ((OlapScanNode) scanNode).getBucketNum(); + } + } + throw new IllegalStateException("Not support bucket shuffle join with non OlapTable"); + } + + private Set missingBuckets(List instances) { + Set usedBuckets = usedBuckets(instances); + int bucketNum = fullBucketNum(); + Set missingBuckets = new TreeSet<>(); + for (int i = 0; i < bucketNum; i++) { + if (!usedBuckets.contains(i)) { + missingBuckets.add(i); + } + } + return missingBuckets; + } + + private Set usedBuckets(List instances) { + Set usedBuckets = new TreeSet<>(); + for (AssignedJob instance : instances) { + ScanSource scanSource = instance.getScanSource(); + if (scanSource instanceof BucketScanSource) { + BucketScanSource bucketScanSource = (BucketScanSource) scanSource; + usedBuckets.addAll(bucketScanSource.bucketIndexToScanNodeToTablets.keySet()); + } + } + return usedBuckets; + } + + private MaterializedIndex randomPartition(OlapScanNode olapScanNode) { + List selectedPartitionIds = ImmutableList.copyOf(olapScanNode.getSelectedPartitionIds()); + if (selectedPartitionIds.isEmpty()) { + throw new IllegalStateException("Missing selected partitions in " + olapScanNode); + } + + Long randomSelectPartitionId = selectedPartitionIds.get((int) (Math.random() * selectedPartitionIds.size())); + Partition partition = olapScanNode.getOlapTable().getPartition(randomSelectPartitionId); + return partition.getBaseIndex(); + } + + private ListMultimap selectWorkerForMissingBuckets( + OlapScanNode olapScanNode, MaterializedIndex partition, Set selectBucketIndexes) { + List tabletIdsInOrder = partition.getTabletIdsInOrder(); + ListMultimap fillUpWorkerToBuckets = ArrayListMultimap.create(); + for (Integer bucketIndex : selectBucketIndexes) { + Long tabletIdInBucket = tabletIdsInOrder.get(bucketIndex); + Tablet tabletInBucket = partition.getTablet(tabletIdInBucket); + List workers = getWorkersByReplicas(tabletInBucket); + if (workers.isEmpty()) { + throw new IllegalStateException("Can not found available replica for bucket " + bucketIndex + + ", table: " + olapScanNode); + } + DistributedPlanWorker worker = scanWorkerSelector.selectMinWorkloadWorker(workers); + fillUpWorkerToBuckets.put(worker, bucketIndex); + } + return fillUpWorkerToBuckets; + } + + private List getWorkersByReplicas(Tablet tablet) { + DistributedPlanWorkerManager workerManager = scanWorkerSelector.getWorkerManager(); + List replicas = tablet.getReplicas(); + List workers = Lists.newArrayListWithCapacity(replicas.size()); + for (Replica replica : replicas) { + DistributedPlanWorker worker = workerManager.getWorker(replica.getBackendId()); + if (worker.available()) { + workers.add(worker); + } + } + return workers; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanSingleOlapTableJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanSingleOlapTableJob.java new file mode 100644 index 00000000000000..94b545f9f9ed67 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanSingleOlapTableJob.java @@ -0,0 +1,82 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; +import org.apache.doris.nereids.trees.plans.distribute.worker.ScanWorkerSelector; +import org.apache.doris.planner.ExchangeNode; +import org.apache.doris.planner.OlapScanNode; +import org.apache.doris.planner.PlanFragment; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ListMultimap; + +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** UnassignedScanSingleOlapTableJob */ +public class UnassignedScanSingleOlapTableJob extends AbstractUnassignedScanJob { + private OlapScanNode olapScanNode; + private final ScanWorkerSelector scanWorkerSelector; + + public UnassignedScanSingleOlapTableJob( + PlanFragment fragment, OlapScanNode olapScanNode, + ListMultimap exchangeToChildJob, + ScanWorkerSelector scanWorkerSelector) { + super(fragment, ImmutableList.of(olapScanNode), exchangeToChildJob); + this.scanWorkerSelector = Objects.requireNonNull( + scanWorkerSelector, "scanWorkerSelector cat not be null"); + this.olapScanNode = olapScanNode; + } + + @Override + protected Map multipleMachinesParallelization( + DistributedPlanWorkerManager workerManager, ListMultimap inputJobs) { + // for every tablet, select its replica and worker. + // for example: + // { + // BackendWorker("172.0.0.1"): + // olapScanNode1: ScanRanges([tablet_10001, tablet_10002, tablet_10003, tablet_10004]), + // BackendWorker("172.0.0.2"): + // olapScanNode1: ScanRanges([tablet_10005, tablet_10006, tablet_10007, tablet_10008, tablet_10009]) + // } + return scanWorkerSelector.selectReplicaAndWorkerWithoutBucket(olapScanNode); + } + + @Override + protected List insideMachineParallelization( + Map workerToScanRanges, + ListMultimap inputJobs) { + // for each worker, compute how many instances should be generated, and which data should be scanned. + // for example: + // { + // BackendWorker("172.0.0.1"): [ + // instance 1: olapScanNode1: ScanRanges([tablet_10001, tablet_10003]) + // instance 2: olapScanNode1: ScanRanges([tablet_10002, tablet_10004]) + // ], + // BackendWorker("172.0.0.2"): [ + // instance 3: olapScanNode1: ScanRanges([tablet_10005, tablet_10008]) + // instance 4: olapScanNode1: ScanRanges([tablet_10006, tablet_10009]) + // instance 5: olapScanNode1: ScanRanges([tablet_10007]) + // ], + // } + return super.insideMachineParallelization(workerToScanRanges, inputJobs); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanSingleRemoteTableJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanSingleRemoteTableJob.java new file mode 100644 index 00000000000000..8205f0d67d2c84 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedScanSingleRemoteTableJob.java @@ -0,0 +1,53 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; +import org.apache.doris.nereids.trees.plans.distribute.worker.ScanWorkerSelector; +import org.apache.doris.planner.ExchangeNode; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.planner.ScanNode; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ListMultimap; + +import java.util.Map; +import java.util.Objects; + +/** + * UnassignedScanSingleRemoteTableJob + * it should be a leaf job which not contains scan native olap table node, + * for example, select literal without table, or scan an external table + */ +public class UnassignedScanSingleRemoteTableJob extends AbstractUnassignedScanJob { + private final ScanWorkerSelector scanWorkerSelector; + + public UnassignedScanSingleRemoteTableJob( + PlanFragment fragment, ScanNode scanNode, ListMultimap exchangeToChildJob, + ScanWorkerSelector scanWorkerSelector) { + super(fragment, ImmutableList.of(scanNode), exchangeToChildJob); + this.scanWorkerSelector = Objects.requireNonNull(scanWorkerSelector, "scanWorkerSelector is not null"); + } + + @Override + protected Map multipleMachinesParallelization( + DistributedPlanWorkerManager workerManager, ListMultimap inputJobs) { + return scanWorkerSelector.selectReplicaAndWorkerWithoutBucket(scanNodes.get(0)); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedShuffleJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedShuffleJob.java new file mode 100644 index 00000000000000..5b1cb1f9878fb1 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedShuffleJob.java @@ -0,0 +1,124 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; +import org.apache.doris.planner.ExchangeNode; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ListMultimap; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map.Entry; +import java.util.Set; +import java.util.function.Function; + +/** UnassignedShuffleJob */ +public class UnassignedShuffleJob extends AbstractUnassignedJob { + public UnassignedShuffleJob(PlanFragment fragment, ListMultimap exchangeToChildJob) { + super(fragment, ImmutableList.of(), exchangeToChildJob); + } + + @Override + public List computeAssignedJobs( + DistributedPlanWorkerManager workerManager, ListMultimap inputJobs) { + int expectInstanceNum = degreeOfParallelism(); + List biggestParallelChildFragment = getInstancesOfBiggestParallelChildFragment(inputJobs); + + if (expectInstanceNum > 0 && expectInstanceNum < biggestParallelChildFragment.size()) { + // When group by cardinality is smaller than number of backend, only some backends always + // process while other has no data to process. + // So we shuffle instances to make different backends handle different queries. + List shuffleWorkersInBiggestParallelChildFragment + = distinctShuffleWorkers(biggestParallelChildFragment); + Function workerSelector = instanceIndex -> { + int selectIndex = instanceIndex % shuffleWorkersInBiggestParallelChildFragment.size(); + return shuffleWorkersInBiggestParallelChildFragment.get(selectIndex); + }; + return buildInstances(expectInstanceNum, workerSelector); + } else { + // keep same instance num like child fragment + Function workerSelector = instanceIndex -> { + int selectIndex = instanceIndex % biggestParallelChildFragment.size(); + return biggestParallelChildFragment.get(selectIndex).getAssignedWorker(); + }; + return buildInstances(biggestParallelChildFragment.size(), workerSelector); + } + } + + protected int degreeOfParallelism() { + if (!fragment.getDataPartition().isPartitioned()) { + return 1; + } + + // TODO: check we use nested loop join do right outer / semi / anti join, + // we should add an exchange node with gather distribute under the nested loop join + + int expectInstanceNum = -1; + if (ConnectContext.get() != null && ConnectContext.get().getSessionVariable() != null) { + expectInstanceNum = ConnectContext.get().getSessionVariable().getExchangeInstanceParallel(); + } + return expectInstanceNum; + } + + private List getInstancesOfBiggestParallelChildFragment( + ListMultimap inputJobs) { + int maxInstanceNum = -1; + List biggestParallelChildFragment = ImmutableList.of(); + // skip broadcast exchange + for (Entry> exchangeToChildInstances : inputJobs.asMap().entrySet()) { + List instances = (List) exchangeToChildInstances.getValue(); + if (instances.size() > maxInstanceNum) { + biggestParallelChildFragment = instances; + maxInstanceNum = instances.size(); + } + } + return biggestParallelChildFragment; + } + + private List buildInstances(int instanceNum, Function workerSelector) { + ImmutableList.Builder instances = ImmutableList.builderWithExpectedSize(instanceNum); + ConnectContext context = ConnectContext.get(); + for (int i = 0; i < instanceNum; i++) { + DistributedPlanWorker selectedWorker = workerSelector.apply(i); + AssignedJob assignedJob = assignWorkerAndDataSources( + i, context.nextInstanceId(), selectedWorker, new DefaultScanSource(ImmutableMap.of()) + ); + instances.add(assignedJob); + } + return instances.build(); + } + + private List distinctShuffleWorkers(List instances) { + Set candidateWorkerSet = Sets.newLinkedHashSet(); + for (AssignedJob instance : instances) { + candidateWorkerSet.add(instance.getAssignedWorker()); + } + List candidateWorkers = Lists.newArrayList(candidateWorkerSet); + Collections.shuffle(candidateWorkers); + return candidateWorkers; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedSpecifyInstancesJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedSpecifyInstancesJob.java new file mode 100644 index 00000000000000..c450161ae41905 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedSpecifyInstancesJob.java @@ -0,0 +1,49 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.nereids.trees.plans.distribute.NereidsSpecifyInstances; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; +import org.apache.doris.planner.ExchangeNode; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.planner.ScanNode; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ListMultimap; + +import java.util.List; + +/** UnassignedSpecifyInstancesJob */ +public class UnassignedSpecifyInstancesJob extends AbstractUnassignedJob { + private final NereidsSpecifyInstances specifyInstances; + + public UnassignedSpecifyInstancesJob( + PlanFragment fragment, List scanNodes, + ListMultimap exchangeToChildJob) { + super(fragment, scanNodes, exchangeToChildJob); + Preconditions.checkArgument(fragment.specifyInstances.isPresent(), + "Missing fragment specifyInstances"); + this.specifyInstances = fragment.specifyInstances.get(); + } + + @Override + public List computeAssignedJobs(DistributedPlanWorkerManager workerManager, + ListMultimap inputJobs) { + return specifyInstances.buildAssignedJobs(this); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UninstancedScanSource.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UninstancedScanSource.java new file mode 100644 index 00000000000000..cce4c2f25f966a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UninstancedScanSource.java @@ -0,0 +1,36 @@ +// 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.trees.plans.distribute.worker.job; + +import com.google.common.collect.ImmutableMap; + +/** + * UninstancedScanSource: + * a ScanSource which doesn't parallelize/split to instances + */ +public class UninstancedScanSource { + public final ScanSource scanSource; + + public UninstancedScanSource(ScanSource scanSource) { + this.scanSource = scanSource; + } + + public static UninstancedScanSource emptyDefaultScanSource() { + return new UninstancedScanSource(new DefaultScanSource(ImmutableMap.of())); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/WorkerScanSource.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/WorkerScanSource.java new file mode 100644 index 00000000000000..f9c5993a5b5eb3 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/WorkerScanSource.java @@ -0,0 +1,31 @@ +// 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.trees.plans.distribute.worker.job; + +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; + +/** WorkerScanSource */ +public class WorkerScanSource { + public final DistributedPlanWorker worker; + public final S scanSource; + + public WorkerScanSource(DistributedPlanWorker worker, S scanSource) { + this.worker = worker; + this.scanSource = scanSource; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/JoinUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/JoinUtils.java index c9067e7cc4643d..986aef4fc17666 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/JoinUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/JoinUtils.java @@ -25,6 +25,7 @@ import org.apache.doris.nereids.properties.DistributionSpecHash; import org.apache.doris.nereids.properties.DistributionSpecHash.ShuffleType; import org.apache.doris.nereids.properties.DistributionSpecReplicated; +import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.rules.rewrite.ForeignKeyContext; import org.apache.doris.nereids.trees.expressions.EqualPredicate; import org.apache.doris.nereids.trees.expressions.ExprId; @@ -224,12 +225,25 @@ public static EqualPredicate swapEqualToForChildrenOrder(EqualPredicate equalTo, * return true if we should do bucket shuffle join when translate plan. */ public static boolean shouldBucketShuffleJoin(AbstractPhysicalJoin join) { - DistributionSpec rightDistributionSpec = join.right().getPhysicalProperties().getDistributionSpec(); - if (!(rightDistributionSpec instanceof DistributionSpecHash)) { + if (isStorageBucketed(join.right().getPhysicalProperties())) { + return true; + } else if (SessionVariable.canUseNereidsDistributePlanner() + && isStorageBucketed(join.left().getPhysicalProperties())) { + return true; + } + return false; + } + + private static boolean isStorageBucketed(PhysicalProperties physicalProperties) { + DistributionSpec distributionSpec = physicalProperties.getDistributionSpec(); + if (!(distributionSpec instanceof DistributionSpecHash)) { return false; } - DistributionSpecHash rightHash = (DistributionSpecHash) rightDistributionSpec; - return rightHash.getShuffleType() == ShuffleType.STORAGE_BUCKETED; + DistributionSpecHash rightHash = (DistributionSpecHash) distributionSpec; + if (rightHash.getShuffleType() == ShuffleType.STORAGE_BUCKETED) { + return true; + } + return false; } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/Utils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/Utils.java index 8525d2629e34ab..289119dc0d0bd9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/Utils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/Utils.java @@ -466,4 +466,18 @@ public static String convertFirstChar(String input) { } return firstChar + input.substring(1); } + + /** addLinePrefix */ + public static String addLinePrefix(String str, String prefix) { + StringBuilder newStr = new StringBuilder((int) (str.length() * 1.2)); + String[] lines = str.split("\n"); + for (int i = 0; i < lines.length; i++) { + String line = lines[i]; + newStr.append(prefix).append(line); + if (i + 1 < lines.length) { + newStr.append("\n"); + } + } + return newStr.toString(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/DataGenScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/DataGenScanNode.java index 1c760adb94aa5b..60fce4df14848f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/DataGenScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/DataGenScanNode.java @@ -59,6 +59,10 @@ public void init(Analyzer analyzer) throws UserException { super.init(analyzer); } + public DataGenTableValuedFunction getTvf() { + return tvf; + } + @Override public List getScanRangeLocations(long maxScanRangeLength) { return scanRangeLocations; diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java index 5849aa71d6aeff..0e403123364e22 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java @@ -169,6 +169,8 @@ public class OlapScanNode extends ScanNode { private int selectedPartitionNum = 0; private Collection selectedPartitionIds = Lists.newArrayList(); private long totalBytes = 0; + // tablet id to single replica bytes + private Map tabletBytes = Maps.newLinkedHashMap(); private SortInfo sortInfo = null; private Set outputColumnUniqueIds = new HashSet<>(); @@ -191,6 +193,7 @@ public class OlapScanNode extends ScanNode { // a bucket seq may map to many tablets, and each tablet has a // TScanRangeLocations. public ArrayListMultimap bucketSeq2locations = ArrayListMultimap.create(); + public Map bucketSeq2Bytes = Maps.newLinkedHashMap(); boolean isFromPrepareStmt = false; // For point query @@ -748,6 +751,10 @@ public void updateScanRangeVersions(Map visibleVersionMap) { } } + public Long getTabletSingleReplicaSize(Long tabletId) { + return tabletBytes.get(tabletId); + } + private void addScanRangeLocations(Partition partition, List tablets) throws UserException { long visibleVersion = Partition.PARTITION_INIT_VERSION; @@ -877,6 +884,9 @@ private void addScanRangeLocations(Partition partition, boolean tabletIsNull = true; boolean collectedStat = false; List errs = Lists.newArrayList(); + + int replicaInTablet = 0; + long oneReplicaBytes = 0; for (Replica replica : replicas) { Backend backend = Env.getCurrentSystemInfo().getBackend(replica.getBackendId()); if (backend == null || !backend.isAlive()) { @@ -916,7 +926,13 @@ private void addScanRangeLocations(Partition partition, // for CBO if (!collectedStat && replica.getRowCount() != -1) { - totalBytes += replica.getDataSize(); + long dataSize = replica.getDataSize(); + if (replicaInTablet == 0) { + oneReplicaBytes = dataSize; + tabletBytes.put(tabletId, dataSize); + } + replicaInTablet++; + totalBytes += dataSize; collectedStat = true; } scanBackendIds.add(backend.getId()); @@ -934,8 +950,9 @@ private void addScanRangeLocations(Partition partition, scanRange.setPaloScanRange(paloRange); locations.setScanRange(scanRange); - bucketSeq2locations.put(tabletId2BucketSeq.get(tabletId), locations); - + Integer bucketSeq = tabletId2BucketSeq.get(tabletId); + bucketSeq2locations.put(bucketSeq, locations); + bucketSeq2Bytes.merge(bucketSeq, oneReplicaBytes, Long::sum); scanRangeLocations.add(locations); } @@ -1403,6 +1420,20 @@ public boolean getShouldColoScan() { return shouldColoScan; } + public int getBucketNum() { + // In bucket shuffle join, we have 2 situation. + // 1. Only one partition: in this case, we use scanNode.getTotalTabletsNum() to get the right bucket num + // because when table turn on dynamic partition, the bucket number in default distribution info + // is not correct. + // 2. Table is colocated: in this case, table could have more than one partition, but all partition's + // bucket number must be same, so we use default bucket num is ok. + if (olapTable.isColocateTable()) { + return olapTable.getDefaultDistributionInfo().getBucketNum(); + } else { + return (int) totalTabletsNum; + } + } + @Override // If scan is key search, should not enable the shared scan opt to prevent the performance problem // 1. where contain the eq or in expr of key column slot 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 7418e15bdc8f57..8a26ec4af66ac1 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 @@ -27,6 +27,8 @@ import org.apache.doris.analysis.StatementBase; import org.apache.doris.analysis.TupleDescriptor; import org.apache.doris.common.TreeNode; +import org.apache.doris.nereids.trees.plans.distribute.NereidsSpecifyInstances; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.ScanSource; import org.apache.doris.qe.ConnectContext; import org.apache.doris.thrift.TExplainLevel; import org.apache.doris.thrift.TPartitionType; @@ -34,6 +36,7 @@ import org.apache.doris.thrift.TResultSinkType; import com.google.common.base.Preconditions; +import com.google.common.base.Suppliers; import com.google.common.collect.Lists; import org.apache.commons.collections.CollectionUtils; import org.apache.logging.log4j.LogManager; @@ -43,7 +46,9 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; +import java.util.function.Supplier; import java.util.stream.Collectors; /** @@ -148,9 +153,12 @@ public class PlanFragment extends TreeNode { // has colocate plan node protected boolean hasColocatePlanNode = false; + protected final Supplier hasBucketShuffleJoin; private TResultSinkType resultSinkType = TResultSinkType.MYSQL_PROTOCAL; + public Optional> specifyInstances = Optional.empty(); + /** * C'tor for fragment with specific partition; the output is by default broadcast. */ @@ -162,6 +170,7 @@ public PlanFragment(PlanFragmentId id, PlanNode root, DataPartition partition) { this.transferQueryStatisticsWithEveryBatch = false; this.builderRuntimeFilterIds = new HashSet<>(); this.targetRuntimeFilterIds = new HashSet<>(); + this.hasBucketShuffleJoin = buildHasBucketShuffleJoin(); setParallelExecNumIfExists(); setFragmentInPlanTree(planRoot); } @@ -178,6 +187,18 @@ public PlanFragment(PlanFragmentId id, PlanNode root, DataPartition partition, this.targetRuntimeFilterIds = new HashSet<>(targetRuntimeFilterIds); } + private Supplier buildHasBucketShuffleJoin() { + return Suppliers.memoize(() -> { + List hashJoinNodes = getPlanRoot().collectInCurrentFragment(HashJoinNode.class::isInstance); + for (HashJoinNode hashJoinNode : hashJoinNodes) { + if (hashJoinNode.isBucketShuffle()) { + return true; + } + } + return false; + }); + } + /** * Assigns 'this' as fragment of all PlanNodes in the plan tree rooted at node. * Does not traverse the children of ExchangeNodes because those must belong to a @@ -240,6 +261,10 @@ public void setHasColocatePlanNode(boolean hasColocatePlanNode) { this.hasColocatePlanNode = hasColocatePlanNode; } + public boolean hasBucketShuffleJoin() { + return hasBucketShuffleJoin.get(); + } + public void setResultSinkType(TResultSinkType resultSinkType) { this.resultSinkType = resultSinkType; } 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 198d5171e26b46..37f121a5c23d98 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 @@ -59,6 +59,8 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Consumer; +import java.util.function.Predicate; import java.util.stream.Collectors; /** @@ -1256,4 +1258,27 @@ public void addIntermediateOutputTupleDescList(TupleDescriptor tupleDescriptor) public void addIntermediateProjectList(List exprs) { intermediateProjectListList.add(exprs); } + + public List collectInCurrentFragment(Predicate predicate) { + List result = Lists.newArrayList(); + foreachDownInCurrentFragment(child -> { + if (predicate.test(child)) { + result.add(child); + } + }); + return (List) result; + } + + /** foreachDownInCurrentFragment */ + public void foreachDownInCurrentFragment(Consumer visitor) { + int currentFragmentId = getFragmentId().asInt(); + foreachDown(child -> { + PlanNode childNode = (PlanNode) child; + if (childNode.getFragmentId().asInt() != currentFragmentId) { + return false; + } + visitor.accept(childNode); + return true; + }); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java index 342ec3c91490a1..f53f69505da5bd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java @@ -570,9 +570,12 @@ public static ColumnRanges create(List> ranges) { @Override public String toString() { - return MoreObjects.toStringHelper(this).add("tid", desc.getId().asInt()).add("tblName", - desc.getTable().getName()).add("keyRanges", "").addValue( - super.debugString()).toString(); + return MoreObjects.toStringHelper(this) + .add("id", getId().asInt()) + .add("tid", desc.getId().asInt()) + .add("tblName", desc.getTable().getName()) + .add("keyRanges", "") + .addValue(super.debugString()).toString(); } // Some of scan node(eg, DataGenScanNode) does not need to check column priv diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java index 6b6278db7d69e2..71a10d2117165d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java @@ -89,6 +89,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import javax.annotation.Nullable; @@ -118,6 +119,7 @@ public enum ConnectType { protected volatile LoadTaskInfo streamLoadInfo; protected volatile TUniqueId queryId = null; + protected volatile AtomicInteger instanceIdGenerator = new AtomicInteger(); protected volatile String traceId; // id for this connection protected volatile int connectionId; @@ -870,6 +872,10 @@ public TUniqueId queryId() { return queryId; } + public TUniqueId nextInstanceId() { + return new TUniqueId(queryId.hi, queryId.lo + instanceIdGenerator.incrementAndGet()); + } + public String getSqlHash() { return sqlHash; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index a29f679303e226..2c2d4437441c3d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -41,6 +41,8 @@ import org.apache.doris.mysql.MysqlCommand; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.stats.StatsErrorEstimator; +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.TopnFilter; import org.apache.doris.planner.DataPartition; import org.apache.doris.planner.DataSink; @@ -186,9 +188,10 @@ public class Coordinator implements CoordInterface { // copied from TQueryExecRequest; constant across all fragments private final TDescriptorTable descTable; + private FragmentIdMapping distributedPlans; // scan node id -> TFileScanRangeParams - private Map fileScanRangeParamsMap = Maps.newHashMap(); + protected Map fileScanRangeParamsMap = Maps.newHashMap(); // Why do we use query global? // When `NOW()` function is in sql, we need only one now(), @@ -212,7 +215,7 @@ public class Coordinator implements CoordInterface { private boolean returnedAllResults = false; // populated in computeFragmentExecParams() - private final Map fragmentExecParamsMap = Maps.newHashMap(); + protected final Map fragmentExecParamsMap = Maps.newHashMap(); private final List fragments; @@ -336,6 +339,8 @@ public Coordinator(ConnectContext context, Analyzer analyzer, Planner planner) { if (!useNereids) { // Enable local shuffle on pipelineX engine only if Nereids planner is applied. queryOptions.setEnableLocalShuffle(false); + } else { + distributedPlans = ((NereidsPlanner) planner).getDistributedPlans(); } setFromUserProperty(context); @@ -1287,7 +1292,7 @@ private void cancelRemoteFragmentsAsync(Status cancelReason) { } } - private void computeFragmentExecParams() throws Exception { + protected void computeFragmentExecParams() throws Exception { // fill hosts field in fragmentExecParams computeFragmentHosts(); @@ -1303,10 +1308,16 @@ private void computeFragmentExecParams() throws Exception { for (int j = 0; j < params.instanceExecParams.size(); ++j) { // we add instance_num to query_id.lo to create a // globally-unique instance id + FInstanceExecParam instanceExecParam = params.instanceExecParams.get(j); + + // already set by nereids coordinator? + if (instanceExecParam.instanceId != null) { + continue; + } TUniqueId instanceId = new TUniqueId(); instanceId.setHi(queryId.hi); instanceId.setLo(queryId.lo + instanceIds.size() + 1); - params.instanceExecParams.get(j).instanceId = instanceId; + instanceExecParam.instanceId = instanceId; instanceIds.add(instanceId); } } @@ -1679,7 +1690,7 @@ private boolean containsSetOperationNode(PlanNode node) { // For each fragment in fragments, computes hosts on which to run the instances // and stores result in fragmentExecParams.hosts. - private void computeFragmentHosts() throws Exception { + protected void computeFragmentHosts() throws Exception { // compute hosts of producer fragment before those of consumer fragment(s), // the latter might inherit the set of hosts from the former // compute hosts *bottom up*. @@ -2102,12 +2113,7 @@ private void computeScanRangeAssignmentByColocate( fragmentIdTobucketSeqToScanRangeMap.put(scanNode.getFragmentId(), new BucketSeqToScanRange()); // Same as bucket shuffle. - int bucketNum = 0; - if (scanNode.getOlapTable().isColocateTable()) { - bucketNum = scanNode.getOlapTable().getDefaultDistributionInfo().getBucketNum(); - } else { - bucketNum = (int) (scanNode.getTotalTabletsNum()); - } + int bucketNum = scanNode.getBucketNum(); scanNode.getFragment().setBucketNum(bucketNum); } Map bucketSeqToAddress = fragmentIdToSeqToAddressMap.get(scanNode.getFragmentId()); @@ -2460,14 +2466,14 @@ static class BucketSeqToScanRange extends HashMap < bucket_seq -> < scannode_id -> scan_range_params >> - private final Map fragmentIdBucketSeqToScanRangeMap = Maps.newHashMap(); + protected final Map fragmentIdBucketSeqToScanRangeMap = Maps.newHashMap(); // fragment_id -> < bucket_seq -> be_addresss > private final Map> fragmentIdToSeqToAddressMap = Maps.newHashMap(); // fragment_id -> < be_id -> bucket_count > private final Map> fragmentIdToBuckendIdBucketCountMap = Maps.newHashMap(); // fragment_id -> bucket_num - private final Map fragmentIdToBucketNumMap = Maps.newHashMap(); + protected final Map fragmentIdToBucketNumMap = Maps.newHashMap(); // cache the bucketShuffleFragmentIds private final Set bucketShuffleFragmentIds = new HashSet<>(); @@ -2480,7 +2486,7 @@ public BucketShuffleJoinController(Map> fragmentIdT } // check whether the node fragment is bucket shuffle join fragment - private boolean isBucketShuffleJoin(int fragmentId, PlanNode node) { + protected boolean isBucketShuffleJoin(int fragmentId, PlanNode node) { if (ConnectContext.get() != null) { if (!ConnectContext.get().getSessionVariable().isEnableBucketShuffleJoin() && !ConnectContext.get().getSessionVariable().isEnableNereidsPlanner()) { @@ -2565,18 +2571,7 @@ private void computeScanRangeAssignmentByBucket( Map addressToBackendID, Map replicaNumPerHost) throws Exception { if (!fragmentIdToSeqToAddressMap.containsKey(scanNode.getFragmentId())) { - // In bucket shuffle join, we have 2 situation. - // 1. Only one partition: in this case, we use scanNode.getTotalTabletsNum() to get the right bucket num - // because when table turn on dynamic partition, the bucket number in default distribution info - // is not correct. - // 2. Table is colocated: in this case, table could have more than one partition, but all partition's - // bucket number must be same, so we use default bucket num is ok. - int bucketNum = 0; - if (scanNode.getOlapTable().isColocateTable()) { - bucketNum = scanNode.getOlapTable().getDefaultDistributionInfo().getBucketNum(); - } else { - bucketNum = (int) (scanNode.getTotalTabletsNum()); - } + int bucketNum = scanNode.getBucketNum(); fragmentIdToBucketNumMap.put(scanNode.getFragmentId(), bucketNum); fragmentIdToSeqToAddressMap.put(scanNode.getFragmentId(), new HashMap<>()); fragmentIdBucketSeqToScanRangeMap.put(scanNode.getFragmentId(), new BucketSeqToScanRange()); @@ -2731,11 +2726,11 @@ private void assignScanRanges(PlanFragmentId fragmentId, int parallelExecInstanc } private final Map fragmentIdTobucketSeqToScanRangeMap = Maps.newHashMap(); - private final Map> fragmentIdToSeqToAddressMap = Maps.newHashMap(); + protected final Map> fragmentIdToSeqToAddressMap = Maps.newHashMap(); // cache the fragment id to its scan node ids. Used for colocate join. private final Map> fragmentIdToScanNodeIds = Maps.newHashMap(); private final Set colocateFragmentIds = new HashSet<>(); - private final BucketShuffleJoinController bucketShuffleJoinController + protected final BucketShuffleJoinController bucketShuffleJoinController = new BucketShuffleJoinController(fragmentIdToScanNodeIds); public class PipelineExecContext { diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java new file mode 100644 index 00000000000000..4f5af3762c51af --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/NereidsCoordinator.java @@ -0,0 +1,194 @@ +// 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.qe; + +import org.apache.doris.analysis.Analyzer; +import org.apache.doris.datasource.FileQueryScanNode; +import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.stats.StatsErrorEstimator; +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.distribute.PipelineDistributedPlan; +import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.AssignedJob; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.BucketScanSource; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.DefaultScanSource; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.LocalShuffleAssignedJob; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.ScanRanges; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.ScanSource; +import org.apache.doris.nereids.trees.plans.distribute.worker.job.UnassignedJob; +import org.apache.doris.planner.OlapScanNode; +import org.apache.doris.planner.PlanFragment; +import org.apache.doris.planner.Planner; +import org.apache.doris.planner.ScanNode; +import org.apache.doris.thrift.TNetworkAddress; +import org.apache.doris.thrift.TScanRangeParams; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; + +/** NereidsCoordinator */ +public class NereidsCoordinator extends Coordinator { + private NereidsPlanner nereidsPlanner; + private FragmentIdMapping distributedPlans; + + public NereidsCoordinator(ConnectContext context, Analyzer analyzer, + Planner planner, StatsErrorEstimator statsErrorEstimator, NereidsPlanner nereidsPlanner) { + super(context, analyzer, planner, statsErrorEstimator); + this.nereidsPlanner = Objects.requireNonNull(nereidsPlanner, "nereidsPlanner can not be null"); + this.distributedPlans = Objects.requireNonNull( + nereidsPlanner.getDistributedPlans(), "distributedPlans can not be null" + ); + } + + @Override + protected void processFragmentAssignmentAndParams() throws Exception { + // prepare information + prepare(); + + computeFragmentExecParams(); + } + + @Override + protected void computeFragmentHosts() { + // translate distributed plan to params + for (DistributedPlan distributedPlan : distributedPlans.values()) { + UnassignedJob fragmentJob = distributedPlan.getFragmentJob(); + PlanFragment fragment = fragmentJob.getFragment(); + + bucketShuffleJoinController + .isBucketShuffleJoin(fragment.getFragmentId().asInt(), fragment.getPlanRoot()); + + setFileScanParams(distributedPlan); + + FragmentExecParams fragmentExecParams = fragmentExecParamsMap.computeIfAbsent( + fragment.getFragmentId(), id -> new FragmentExecParams(fragment) + ); + List instanceJobs = ((PipelineDistributedPlan) distributedPlan).getInstanceJobs(); + boolean useLocalShuffle = useLocalShuffle(distributedPlan); + if (useLocalShuffle) { + fragmentExecParams.ignoreDataDistribution = true; + fragmentExecParams.parallelTasksNum = 1; + } else { + fragmentExecParams.parallelTasksNum = instanceJobs.size(); + } + + for (AssignedJob instanceJob : instanceJobs) { + DistributedPlanWorker worker = instanceJob.getAssignedWorker(); + TNetworkAddress address = new TNetworkAddress(worker.host(), worker.port()); + FInstanceExecParam instanceExecParam = new FInstanceExecParam( + null, address, 0, fragmentExecParams); + instanceExecParam.instanceId = instanceJob.instanceId(); + fragmentExecParams.instanceExecParams.add(instanceExecParam); + addressToBackendID.put(address, worker.id()); + ScanSource scanSource = instanceJob.getScanSource(); + if (scanSource instanceof BucketScanSource) { + setForBucketScanSource(instanceExecParam, (BucketScanSource) scanSource, useLocalShuffle); + } else { + setForDefaultScanSource(instanceExecParam, (DefaultScanSource) scanSource, useLocalShuffle); + } + } + } + } + + private void setFileScanParams(DistributedPlan distributedPlan) { + for (ScanNode scanNode : distributedPlan.getFragmentJob().getScanNodes()) { + if (scanNode instanceof FileQueryScanNode) { + fileScanRangeParamsMap.put( + scanNode.getId().asInt(), + ((FileQueryScanNode) scanNode).getFileScanRangeParams() + ); + } + } + } + + private boolean useLocalShuffle(DistributedPlan distributedPlan) { + List instanceJobs = ((PipelineDistributedPlan) distributedPlan).getInstanceJobs(); + for (AssignedJob instanceJob : instanceJobs) { + if (instanceJob instanceof LocalShuffleAssignedJob) { + return true; + } + } + return false; + } + + private void setForDefaultScanSource( + FInstanceExecParam instanceExecParam, DefaultScanSource scanSource, boolean isShareScan) { + for (Entry scanNodeIdToReplicaIds : scanSource.scanNodeToScanRanges.entrySet()) { + ScanNode scanNode = scanNodeIdToReplicaIds.getKey(); + ScanRanges scanReplicas = scanNodeIdToReplicaIds.getValue(); + instanceExecParam.perNodeScanRanges.put(scanNode.getId().asInt(), scanReplicas.params); + instanceExecParam.perNodeSharedScans.put(scanNode.getId().asInt(), isShareScan); + } + } + + private void setForBucketScanSource(FInstanceExecParam instanceExecParam, + BucketScanSource bucketScanSource, boolean isShareScan) { + for (Entry> bucketIndexToScanTablets : + bucketScanSource.bucketIndexToScanNodeToTablets.entrySet()) { + Integer bucketIndex = bucketIndexToScanTablets.getKey(); + instanceExecParam.addBucketSeq(bucketIndex); + Map scanNodeToRangeMap = bucketIndexToScanTablets.getValue(); + for (Entry scanNodeToRange : scanNodeToRangeMap.entrySet()) { + ScanNode scanNode = scanNodeToRange.getKey(); + ScanRanges scanRanges = scanNodeToRange.getValue(); + List scanBucketTablets = instanceExecParam.perNodeScanRanges.computeIfAbsent( + scanNode.getId().asInt(), id -> Lists.newArrayList()); + scanBucketTablets.addAll(scanRanges.params); + instanceExecParam.perNodeSharedScans.put(scanNode.getId().asInt(), isShareScan); + + if (scanNode instanceof OlapScanNode) { + OlapScanNode olapScanNode = (OlapScanNode) scanNode; + if (!fragmentIdToSeqToAddressMap.containsKey(scanNode.getFragmentId())) { + int bucketNum = olapScanNode.getBucketNum(); + fragmentIdToSeqToAddressMap.put(olapScanNode.getFragmentId(), new HashMap<>()); + bucketShuffleJoinController.fragmentIdBucketSeqToScanRangeMap + .put(scanNode.getFragmentId(), new BucketSeqToScanRange()); + bucketShuffleJoinController.fragmentIdToBucketNumMap + .put(scanNode.getFragmentId(), bucketNum); + olapScanNode.getFragment().setBucketNum(bucketNum); + } + } else if (!fragmentIdToSeqToAddressMap.containsKey(scanNode.getFragmentId())) { + int bucketNum = 1; + fragmentIdToSeqToAddressMap.put(scanNode.getFragmentId(), new HashMap<>()); + bucketShuffleJoinController.fragmentIdBucketSeqToScanRangeMap + .put(scanNode.getFragmentId(), new BucketSeqToScanRange()); + bucketShuffleJoinController.fragmentIdToBucketNumMap + .put(scanNode.getFragmentId(), bucketNum); + scanNode.getFragment().setBucketNum(bucketNum); + } + + BucketSeqToScanRange bucketSeqToScanRange = bucketShuffleJoinController + .fragmentIdBucketSeqToScanRangeMap.get(scanNode.getFragmentId()); + + Map> scanNodeIdToReplicas + = bucketSeqToScanRange.computeIfAbsent(bucketIndex, set -> Maps.newLinkedHashMap()); + List tablets = scanNodeIdToReplicas.computeIfAbsent( + scanNode.getId().asInt(), id -> new ArrayList<>()); + tablets.addAll(scanRanges.params); + } + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index f49e4f19d31364..d835dc86128496 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -18,6 +18,7 @@ package org.apache.doris.qe; import org.apache.doris.analysis.SetVar; +import org.apache.doris.analysis.StatementBase; import org.apache.doris.analysis.StringLiteral; import org.apache.doris.catalog.Env; import org.apache.doris.common.Config; @@ -26,10 +27,14 @@ import org.apache.doris.common.io.Text; import org.apache.doris.common.io.Writable; import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.analyzer.UnboundResultSink; +import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.metrics.Event; import org.apache.doris.nereids.metrics.EventSwitchParser; import org.apache.doris.nereids.parser.Dialect; import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.planner.GroupCommitBlockSink; import org.apache.doris.qe.VariableMgr.VarAttr; import org.apache.doris.thrift.TGroupCommitMode; @@ -304,6 +309,7 @@ public class SessionVariable implements Serializable, Writable { public static final String NTH_OPTIMIZED_PLAN = "nth_optimized_plan"; public static final String ENABLE_NEREIDS_PLANNER = "enable_nereids_planner"; + public static final String ENABLE_NEREIDS_DISTRIBUTE_PLANNER = "enable_nereids_distribute_planner"; public static final String DISABLE_NEREIDS_RULES = "disable_nereids_rules"; public static final String ENABLE_NEREIDS_RULES = "enable_nereids_rules"; public static final String ENABLE_NEW_COST_MODEL = "enable_new_cost_model"; @@ -1231,7 +1237,7 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { @VariableMgr.VarAttr(name = DISABLE_NEREIDS_RULES, needForward = true) private String disableNereidsRules = ""; - @VariableMgr.VarAttr(name = ENABLE_NEREIDS_RULES, needForward = true, varType = VariableAnnotation.REMOVED) + @VariableMgr.VarAttr(name = ENABLE_NEREIDS_RULES, needForward = true) public String enableNereidsRules = ""; @VariableMgr.VarAttr(name = ENABLE_NEW_COST_MODEL, needForward = true) @@ -1243,6 +1249,17 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { @VariableMgr.VarAttr(name = NEREIDS_STAR_SCHEMA_SUPPORT) private boolean nereidsStarSchemaSupport = true; + @VariableMgr.VarAttr(name = ENABLE_NEREIDS_DISTRIBUTE_PLANNER, needForward = true, + fuzzy = false, varType = VariableAnnotation.EXPERIMENTAL, description = { + "使用新的nereids的分布式规划器的开关,这个分布式规划器可以规划出一些更高效的查询计划,比如在某些情况下," + + "可以把左表shuffle到右表去做bucket shuffle join", + "The switch to use new DistributedPlanner of nereids, this planner can planning some " + + "more efficient query plans, e.g. in certain situations, shuffle left side to " + + "right side to do bucket shuffle join" + } + ) + private boolean enableNereidsDistributePlanner = false; + @VariableMgr.VarAttr(name = REWRITE_OR_TO_IN_PREDICATE_THRESHOLD, fuzzy = true) private int rewriteOrToInPredicateThreshold = 2; @@ -3068,6 +3085,41 @@ public void setEnableNereidsPlanner(boolean enableNereidsPlanner) { this.enableNereidsPlanner = enableNereidsPlanner; } + /** canUseNereidsDistributePlanner */ + public static boolean canUseNereidsDistributePlanner() { + // TODO: support cloud mode + if (Config.isCloudMode()) { + return false; + } + ConnectContext connectContext = ConnectContext.get(); + if (connectContext == null) { + return false; + } + StatementContext statementContext = connectContext.getStatementContext(); + if (statementContext == null) { + return false; + } + StatementBase parsedStatement = statementContext.getParsedStatement(); + if (!(parsedStatement instanceof LogicalPlanAdapter)) { + return false; + } + LogicalPlan logicalPlan = ((LogicalPlanAdapter) parsedStatement).getLogicalPlan(); + SessionVariable sessionVariable = connectContext.getSessionVariable(); + // TODO: support other sink + if (logicalPlan instanceof UnboundResultSink && sessionVariable.enableNereidsDistributePlanner) { + return true; + } + return false; + } + + public boolean isEnableNereidsDistributePlanner() { + return enableNereidsDistributePlanner; + } + + public void setEnableNereidsDistributePlanner(boolean enableNereidsDistributePlanner) { + this.enableNereidsDistributePlanner = enableNereidsDistributePlanner; + } + public int getNthOptimizedPlan() { return nthOptimizedPlan; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java index 021c5fa36ffdb7..8114df770d49ef 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java @@ -1876,9 +1876,17 @@ public void executeAndSendResult(boolean isOutfileQuery, boolean isSendFields, // this branch is for legacy planner, to be removed coordBase = new PointQueryExec(planner, analyzer, context.getSessionVariable().getMaxMsgSizeOfResultReceiver()); + } else if (planner instanceof NereidsPlanner && ((NereidsPlanner) planner).getDistributedPlans() != null) { + coord = new NereidsCoordinator(context, analyzer, + planner, context.getStatsErrorEstimator(), + (NereidsPlanner) planner); + profile.addExecutionProfile(coord.getExecutionProfile()); + QeProcessorImpl.INSTANCE.registerQuery(context.queryId(), + new QeProcessorImpl.QueryInfo(context, originStmt.originStmt, coord)); + coordBase = coord; } else { - coord = EnvFactory.getInstance().createCoordinator(context, analyzer, - planner, context.getStatsErrorEstimator()); + coord = EnvFactory.getInstance().createCoordinator( + context, analyzer, planner, context.getStatsErrorEstimator()); profile.addExecutionProfile(coord.getExecutionProfile()); QeProcessorImpl.INSTANCE.registerQuery(context.queryId(), new QeProcessorImpl.QueryInfo(context, originStmt.originStmt, coord)); diff --git a/fe/fe-core/src/test/java/org/apache/doris/qe/HmsQueryCacheTest.java b/fe/fe-core/src/test/java/org/apache/doris/qe/HmsQueryCacheTest.java index 0414b5d148949b..86c460e6bfd603 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/qe/HmsQueryCacheTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/qe/HmsQueryCacheTest.java @@ -517,8 +517,13 @@ public void testNotHitSqlCache() throws Exception { init((HMSExternalCatalog) mgr.getCatalog(HMS_CATALOG)); StatementBase parseStmt = parseAndAnalyzeStmt("select * from hms_ctl.hms_db.hms_tbl", connectContext); List scanNodes = Arrays.asList(hiveScanNode1); + + CacheAnalyzer ca2 = new CacheAnalyzer(connectContext, parseStmt, scanNodes); + ca2.checkCacheMode(0); + long latestPartitionTime = ca2.getLatestTable().latestPartitionTime; + CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); - ca.checkCacheMode(0); + ca.checkCacheMode(latestPartitionTime); Assert.assertEquals(CacheAnalyzer.CacheMode.None, ca.getCacheMode()); } @@ -527,8 +532,13 @@ public void testNotHitSqlCacheByNereids() { init((HMSExternalCatalog) mgr.getCatalog(HMS_CATALOG)); StatementBase parseStmt = analyzeAndGetStmtByNereids("select * from hms_ctl.hms_db.hms_tbl", connectContext); List scanNodes = Arrays.asList(hiveScanNode1); + + CacheAnalyzer ca2 = new CacheAnalyzer(connectContext, parseStmt, scanNodes); + ca2.checkCacheModeForNereids(0); + long latestPartitionTime = ca2.getLatestTable().latestPartitionTime; + CacheAnalyzer ca = new CacheAnalyzer(connectContext, parseStmt, scanNodes); - ca.checkCacheModeForNereids(0); + ca.checkCacheModeForNereids(latestPartitionTime); Assert.assertEquals(CacheAnalyzer.CacheMode.None, ca.getCacheMode()); } diff --git a/regression-test/data/nereids_syntax_p0/distribute/colocate_union_numbers.out b/regression-test/data/nereids_syntax_p0/distribute/colocate_union_numbers.out new file mode 100644 index 00000000000000..6d15eb60047250 --- /dev/null +++ b/regression-test/data/nereids_syntax_p0/distribute/colocate_union_numbers.out @@ -0,0 +1,10 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !union_all -- +0 +0 +1 +1 +2 +2 +3 + diff --git a/regression-test/data/nereids_syntax_p0/distribute/local_shuffle.out b/regression-test/data/nereids_syntax_p0/distribute/local_shuffle.out new file mode 100644 index 00000000000000..82ed128f7566d6 --- /dev/null +++ b/regression-test/data/nereids_syntax_p0/distribute/local_shuffle.out @@ -0,0 +1,36 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !read_single_olap_table -- +1 1 +2 2 + +-- !broadcast_join -- +2 2 2 2 + +-- !shuffle_join -- +2 2 2 2 + +-- !bucket_shuffle_join -- +2 2 2 2 + +-- !colocate_join -- +2 2 2 2 + +-- !bucket_shuffle_with_prune_tablets -- +\N \N 2 2 +\N \N 3 3 + +-- !bucket_shuffle_with_prune_tablets2 -- +2 2 \N \N +3 3 \N \N + +-- !bucket_shuffle_with_prune_tablets3 -- +1 1 \N \N + +-- !fillup_bucket -- +\N 2 +\N 3 + +-- !shuffle_left -- +1 1 1 +2 2 2 + diff --git a/regression-test/data/nereids_syntax_p0/distribute/prune_bucket_with_bucket_shuffle_join.out b/regression-test/data/nereids_syntax_p0/distribute/prune_bucket_with_bucket_shuffle_join.out new file mode 100644 index 00000000000000..acbf30fb3b1f5e --- /dev/null +++ b/regression-test/data/nereids_syntax_p0/distribute/prune_bucket_with_bucket_shuffle_join.out @@ -0,0 +1,5 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !fillup_bucket -- +\N 2.000 +\N 3.000 + diff --git a/regression-test/data/nereids_syntax_p0/distribute/query_constant.out b/regression-test/data/nereids_syntax_p0/distribute/query_constant.out new file mode 100644 index 00000000000000..ffdd0b7bd9b410 --- /dev/null +++ b/regression-test/data/nereids_syntax_p0/distribute/query_constant.out @@ -0,0 +1,12 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !query_one_row -- +100 abc + +-- !union_all -- +100 hello +200 world + +-- !union -- +100 hello +200 world + diff --git a/regression-test/data/nereids_syntax_p0/distribute/shuffle.out b/regression-test/data/nereids_syntax_p0/distribute/shuffle.out new file mode 100644 index 00000000000000..e94e38f150ead0 --- /dev/null +++ b/regression-test/data/nereids_syntax_p0/distribute/shuffle.out @@ -0,0 +1,8 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !4_phase_agg -- +1 2 2 +2 2 2 +3 2 2 +4 2 2 +5 2 2 + diff --git a/regression-test/data/nereids_syntax_p0/distribute/shuffle_left_join.out b/regression-test/data/nereids_syntax_p0/distribute/shuffle_left_join.out new file mode 100644 index 00000000000000..99d095d87f7886 --- /dev/null +++ b/regression-test/data/nereids_syntax_p0/distribute/shuffle_left_join.out @@ -0,0 +1,9 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !shuffle_left_and_right -- +1 1 1 1 +2 2 2 2 + +-- !shuffle_left -- +1 1 1 1 +2 2 2 2 + diff --git a/regression-test/suites/nereids_p0/test_forbid_unknown_col_stats.groovy b/regression-test/suites/nereids_p0/test_forbid_unknown_col_stats.groovy index 9270b80aaba386..9473b88fd65417 100644 --- a/regression-test/suites/nereids_p0/test_forbid_unknown_col_stats.groovy +++ b/regression-test/suites/nereids_p0/test_forbid_unknown_col_stats.groovy @@ -47,7 +47,7 @@ suite("test_forbid_unknown_col_stats") { test{ sql "select * from test_forbid_unknown_col_stats_tbl;" - exception "tables with unknown column stats: OlapScanNode{tid=0, tblName=test_forbid_unknown_col_stats_tbl, keyRanges=, preds= limit=-1}" + exception "tables with unknown column stats: OlapScanNode{id=0, tid=0, tblName=test_forbid_unknown_col_stats_tbl, keyRanges=, preds= limit=-1}" } sql "select count() from __internal_schema.column_statistics" diff --git a/regression-test/suites/nereids_syntax_p0/distribute/colocate_union_numbers.groovy b/regression-test/suites/nereids_syntax_p0/distribute/colocate_union_numbers.groovy new file mode 100644 index 00000000000000..b247c5bde25a6c --- /dev/null +++ b/regression-test/suites/nereids_syntax_p0/distribute/colocate_union_numbers.groovy @@ -0,0 +1,70 @@ +// 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. + +suite("colocate_union_numbers") { + multi_sql """ + set enable_nereids_distribute_planner=false; + set enable_pipeline_x_engine=true; + set disable_join_reorder=true; + set enable_local_shuffle=false; + set force_to_local_shuffle=false; + """ + + def extractFragment = { String sqlStr, String containsString, Closure checkExchangeNum -> + explain { + sql sqlStr + check { result -> + log.info("Explain result:\n${result}") + + assertTrue(result.contains(containsString)) + + def fragmentContainsJoin = result.split("PLAN FRAGMENT") + .toList() + .stream() + .filter { it.contains(containsString) } + .findFirst() + .get() + + log.info("Fragment:\n${fragmentContainsJoin}") + + checkExchangeNum(fragmentContainsJoin.count("VEXCHANGE")) + } + } + } + + def sqlStr = """ + select * from numbers('number'='3')a + union all + select * from numbers('number'='4')b + """ + + extractFragment(sqlStr, "VUNION") { exchangeNum -> + assertTrue(exchangeNum == 2) + } + + multi_sql """ + set enable_nereids_distribute_planner=true; + set enable_pipeline_x_engine=true; + set disable_join_reorder=true; + """ + + extractFragment(sqlStr, "VUNION") { exchangeNum -> + assertTrue(exchangeNum == 0) + } + + order_qt_union_all sqlStr +} diff --git a/regression-test/suites/nereids_syntax_p0/distribute/local_shuffle.groovy b/regression-test/suites/nereids_syntax_p0/distribute/local_shuffle.groovy new file mode 100644 index 00000000000000..997230b1a067b0 --- /dev/null +++ b/regression-test/suites/nereids_syntax_p0/distribute/local_shuffle.groovy @@ -0,0 +1,187 @@ +// 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. + +suite("local_shuffle") { + multi_sql """ + drop table if exists test_local_shuffle1; + drop table if exists test_local_shuffle2; + + CREATE TABLE `test_local_shuffle1` ( + id int, + id2 int + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "colocate_with" = "test_local_shuffle_with_colocate" + ); + + CREATE TABLE `test_local_shuffle2` ( + id int, + id2 int + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "colocate_with" = "test_local_shuffle_with_colocate" + ); + + insert into test_local_shuffle1 values (1, 1), (2, 2); + insert into test_local_shuffle2 values (2, 2), (3, 3); + + set enable_nereids_distribute_planner=true; + set enable_pipeline_x_engine=true; + set disable_join_reorder=true; + set enable_local_shuffle=true; + set force_to_local_shuffle=true; + """ + + order_qt_read_single_olap_table "select * from test_local_shuffle1" + + order_qt_broadcast_join """ + select * + from test_local_shuffle1 + join [broadcast] + test_local_shuffle2 + on test_local_shuffle1.id=test_local_shuffle2.id + """ + + order_qt_shuffle_join """ + select * + from test_local_shuffle1 + join [shuffle] + test_local_shuffle2 + on test_local_shuffle1.id2=test_local_shuffle2.id2 + """ + + order_qt_bucket_shuffle_join """ + select * + from test_local_shuffle1 + join [shuffle] + test_local_shuffle2 + on test_local_shuffle1.id2=test_local_shuffle2.id2 + """ + + order_qt_colocate_join """ + select * + from test_local_shuffle1 + join [shuffle] + test_local_shuffle2 + on test_local_shuffle1.id=test_local_shuffle2.id + """ + + order_qt_bucket_shuffle_with_prune_tablets """ + select * + from + ( + select * + from test_local_shuffle1 + where id=1 + ) a + right outer join [shuffle] + test_local_shuffle2 + on a.id=test_local_shuffle2.id2 + """ + + order_qt_bucket_shuffle_with_prune_tablets2 """ + select * + from + test_local_shuffle2 + left outer join [shuffle] + ( + select * + from test_local_shuffle1 + where id=1 + ) a + on a.id=test_local_shuffle2.id2 + """ + + order_qt_bucket_shuffle_with_prune_tablets3 """ + select * + from + ( + select * + from test_local_shuffle1 + where id=1 + ) a + left outer join [shuffle] + test_local_shuffle2 + on a.id=test_local_shuffle2.id2 + """ + + multi_sql """ + drop table if exists test_local_shuffle3; + CREATE TABLE IF NOT EXISTS test_local_shuffle3 ( + c0 int + ) + DISTRIBUTED BY HASH (c0) BUCKETS 10 PROPERTIES ("replication_num" = "1"); + + drop table if exists test_local_shuffle4; + CREATE TABLE IF NOT EXISTS test_local_shuffle4 ( + c0 int + ) + DISTRIBUTED BY HASH (c0) BUCKETS 10 PROPERTIES ("replication_num" = "1"); + INSERT INTO test_local_shuffle3 (c0) VALUES (1), (3); + INSERT INTO test_local_shuffle4 (c0) VALUES (2), (3); + + sync; + + set enable_nereids_distribute_planner=true; + set enable_pipeline_x_engine=true; + set disable_join_reorder=true; + set enable_local_shuffle=true; + set force_to_local_shuffle=true; + """ + + order_qt_fillup_bucket """ + SELECT cast(a.c0 as int), cast(b.c0 as int) FROM + (select * from test_local_shuffle3 where c0 =1)a + RIGHT OUTER JOIN + (select * from test_local_shuffle4)b + ON a.c0 = b.c0 + """ + + multi_sql """ + drop table if exists test_shuffle_left_with_local_shuffle; + CREATE TABLE `test_shuffle_left_with_local_shuffle` ( + id int, + id2 int + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + + insert into test_shuffle_left_with_local_shuffle values (1, 1), (2, 2), (3, 4); + """ + + order_qt_shuffle_left """ + select * + from + ( + select id2 + from test_shuffle_left_with_local_shuffle + group by id2 + ) a + inner join [shuffle] + test_shuffle_left_with_local_shuffle b + on a.id2=b.id; + """ +} diff --git a/regression-test/suites/nereids_syntax_p0/distribute/prune_bucket_with_bucket_shuffle_join.groovy b/regression-test/suites/nereids_syntax_p0/distribute/prune_bucket_with_bucket_shuffle_join.groovy new file mode 100644 index 00000000000000..3ce412a0075b2a --- /dev/null +++ b/regression-test/suites/nereids_syntax_p0/distribute/prune_bucket_with_bucket_shuffle_join.groovy @@ -0,0 +1,98 @@ +// 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. + +suite("prune_bucket_with_bucket_shuffle_join") { + multi_sql """ + drop table if exists test_outer_join1; + CREATE TABLE IF NOT EXISTS test_outer_join1 ( + c0 DECIMALV3(8,3) + ) + DISTRIBUTED BY HASH (c0) BUCKETS 10 PROPERTIES ("replication_num" = "1"); + + drop table if exists test_outer_join2; + CREATE TABLE IF NOT EXISTS test_outer_join2 ( + c0 DECIMALV3(8,3) + ) + DISTRIBUTED BY HASH (c0) BUCKETS 10 PROPERTIES ("replication_num" = "1"); + INSERT INTO test_outer_join1 (c0) VALUES (1), (3); + INSERT INTO test_outer_join2 (c0) VALUES (2), (3); + + sync; + + set enable_nereids_distribute_planner=false; + set enable_pipeline_x_engine=true; + set disable_join_reorder=true; + set enable_local_shuffle=false; + set force_to_local_shuffle=false; + """ + + def extractFragment = { String sqlStr, String containsString, Closure checkExchangeNum -> + explain { + sql sqlStr + check { result -> + log.info("Explain result:\n${result}") + + assertTrue(result.contains(containsString)) + + def fragmentContainsJoin = result.split("PLAN FRAGMENT") + .toList() + .stream() + .filter { it.contains(containsString) } + .findFirst() + .get() + + log.info("Fragment:\n${fragmentContainsJoin}") + + checkExchangeNum(fragmentContainsJoin.count("VEXCHANGE")) + } + } + } + + String sqlStr = """ + 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 + """ + + extractFragment(sqlStr, "RIGHT OUTER JOIN(PARTITIONED)") { exchangeNum -> + assertTrue(exchangeNum > 1) + } + + multi_sql """ + set enable_nereids_distribute_planner=true; + set enable_pipeline_x_engine=true; + set disable_join_reorder=true; + """ + + extractFragment(sqlStr, "RIGHT OUTER JOIN(BUCKET_SHUFFLE)") { exchangeNum -> + assertTrue(exchangeNum == 1) + } + + explain { + sql "distributed plan ${sqlStr}" + check { explainStr -> + log.info("Distributed plan:\n${explainStr}") + + // some tablets of left table are pruned + assertTrue(explainStr.count("tablet ") < 20) + } + } + + order_qt_fillup_bucket sqlStr +} diff --git a/regression-test/suites/nereids_syntax_p0/distribute/query_constant.groovy b/regression-test/suites/nereids_syntax_p0/distribute/query_constant.groovy new file mode 100644 index 00000000000000..19521294975934 --- /dev/null +++ b/regression-test/suites/nereids_syntax_p0/distribute/query_constant.groovy @@ -0,0 +1,39 @@ +// 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. + +suite("query_constant") { + multi_sql """ + set enable_nereids_distribute_planner=true; + set enable_pipeline_x_engine=true; + set enable_local_shuffle=false; + set force_to_local_shuffle=false; + """ + + order_qt_query_one_row "select 100 id, 'abc' name" + + order_qt_union_all """ + select 100 id, 'hello' name + union all + select 200 id, 'world' name + """ + + order_qt_union """ + select 100 id, 'hello' name + union + select 200 id, 'world' name + """ +} diff --git a/regression-test/suites/nereids_syntax_p0/distribute/shuffle.groovy b/regression-test/suites/nereids_syntax_p0/distribute/shuffle.groovy new file mode 100644 index 00000000000000..42ec0df05459a8 --- /dev/null +++ b/regression-test/suites/nereids_syntax_p0/distribute/shuffle.groovy @@ -0,0 +1,34 @@ +// 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. + +suite("shuffle") { + createTestTable "test_shuffle" + + multi_sql """ + set enable_nereids_distribute_planner=true; + set enable_pipeline_x_engine=true; + set enable_local_shuffle=false; + set force_to_local_shuffle=false; + """ + + order_qt_4_phase_agg """ + select /*+SET_VAR(disable_nereids_rules='TWO_PHASE_AGGREGATE_WITH_MULTI_DISTINCT,TWO_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI,THREE_PHASE_AGGREGATE_WITH_COUNT_DISTINCT_MULTI,THREE_PHASE_AGGREGATE_WITH_DISTINCT,FOUR_PHASE_AGGREGATE_WITH_DISTINCT')*/ + id, count(distinct value) + from test_shuffle + group by id + """ +} diff --git a/regression-test/suites/nereids_syntax_p0/distribute/shuffle_left_join.groovy b/regression-test/suites/nereids_syntax_p0/distribute/shuffle_left_join.groovy new file mode 100644 index 00000000000000..8c56c257b0e2d9 --- /dev/null +++ b/regression-test/suites/nereids_syntax_p0/distribute/shuffle_left_join.groovy @@ -0,0 +1,117 @@ +import java.util.stream.Collectors + +// 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. + +suite("shuffle_left_join") { + multi_sql """ + drop table if exists test_shuffle_left; + + CREATE TABLE `test_shuffle_left` ( + id int, + id2 int + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + + insert into test_shuffle_left values (1, 1), (2, 2), (3, 4); + + sync; + + set enable_nereids_distribute_planner=false; + set enable_pipeline_x_engine=true; + set disable_join_reorder=true; + set enable_local_shuffle=false; + set force_to_local_shuffle=false; + """ + + def extractFragment = { String sqlStr, String containsString, Closure checkExchangeNum -> + explain { + sql sqlStr + check { result -> + log.info("Explain result:\n${result}") + + assertTrue(result.contains(containsString)) + + def fragmentContainsJoin = result.split("PLAN FRAGMENT") + .toList() + .stream() + .filter { it.contains(containsString) } + .findFirst() + .get() + + log.info("Fragment:\n${fragmentContainsJoin}") + + checkExchangeNum(fragmentContainsJoin.count("VEXCHANGE")) + } + } + } + + def sqlStr = """ + select * + from + ( + select id2 + from test_shuffle_left + group by id2 + ) a + inner join [shuffle] + test_shuffle_left b + on a.id2=b.id; + """ + + extractFragment(sqlStr, "INNER JOIN(PARTITIONED)") { exchangeNum -> + assertTrue(exchangeNum == 2) + } + + order_qt_shuffle_left_and_right sqlStr + + multi_sql """ + set enable_nereids_distribute_planner=true; + set enable_pipeline_x_engine=true; + set disable_join_reorder=true; + """ + + def variables = sql "show variables" + def variableString = variables.stream() + .map { it.toString() } + .collect(Collectors.joining("\n")) + logger.info("Variables:\n${variableString}") + + extractFragment(sqlStr, "INNER JOIN(BUCKET_SHUFFLE)") { exchangeNum -> + assertTrue(exchangeNum == 1) + } + + explain { + sql "plan $sqlStr" + check { explainStr -> + log.info("explain plan:\n${explainStr}") + } + } + + def rows = sql sqlStr + def rowsString = rows.stream() + .map { it.toString() } + .collect(Collectors.joining("\n")) + logger.info("Rows:\n${rowsString}") + + + order_qt_shuffle_left sqlStr +}