Skip to content

Commit f9479b6

Browse files
committed
Create a trait ShuffledJoin for SMJ and SHJ
1 parent dface2a commit f9479b6

File tree

3 files changed

+47
-30
lines changed

3 files changed

+47
-30
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala

Lines changed: 2 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -40,27 +40,14 @@ case class ShuffledHashJoinExec(
4040
condition: Option[Expression],
4141
left: SparkPlan,
4242
right: SparkPlan)
43-
extends HashJoin {
43+
extends HashJoin with ShuffledJoin {
4444

4545
override lazy val metrics = Map(
4646
"numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"),
4747
"buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size of build side"),
4848
"buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build hash map"))
4949

50-
override def outputPartitioning: Partitioning = joinType match {
51-
case _: InnerLike =>
52-
PartitioningCollection(Seq(left.outputPartitioning, right.outputPartitioning))
53-
// For left and right outer joins, the output is partitioned by the streamed input's join keys.
54-
case LeftOuter => left.outputPartitioning
55-
case RightOuter => right.outputPartitioning
56-
case LeftExistence(_) => left.outputPartitioning
57-
case x =>
58-
throw new IllegalArgumentException(
59-
s"${getClass.getSimpleName} should not take $x as the JoinType")
60-
}
61-
62-
override def requiredChildDistribution: Seq[Distribution] =
63-
HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil
50+
override def outputPartitioning: Partitioning = super[ShuffledJoin].outputPartitioning
6451

6552
private def buildHashedRelation(iter: Iterator[InternalRow]): HashedRelation = {
6653
val buildDataSize = longMetric("buildDataSize")
Lines changed: 43 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,43 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.execution.joins
19+
20+
import org.apache.spark.sql.catalyst.plans.{FullOuter, InnerLike, LeftExistence, LeftOuter, RightOuter}
21+
import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution, Partitioning, PartitioningCollection, UnknownPartitioning}
22+
23+
/**
24+
* Holds common logic for join operators by shuffling two child relations
25+
* using the join keys.
26+
*/
27+
trait ShuffledJoin extends BaseJoinExec {
28+
override def requiredChildDistribution: Seq[Distribution] = {
29+
HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil
30+
}
31+
32+
override def outputPartitioning: Partitioning = joinType match {
33+
case _: InnerLike =>
34+
PartitioningCollection(Seq(left.outputPartitioning, right.outputPartitioning))
35+
case LeftOuter => left.outputPartitioning
36+
case RightOuter => right.outputPartitioning
37+
case FullOuter => UnknownPartitioning(left.outputPartitioning.numPartitions)
38+
case LeftExistence(_) => left.outputPartitioning
39+
case x =>
40+
throw new IllegalArgumentException(
41+
s"ShuffledJoin should not take $x as the JoinType")
42+
}
43+
}

sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala

Lines changed: 2 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -41,7 +41,7 @@ case class SortMergeJoinExec(
4141
condition: Option[Expression],
4242
left: SparkPlan,
4343
right: SparkPlan,
44-
isSkewJoin: Boolean = false) extends BaseJoinExec with CodegenSupport {
44+
isSkewJoin: Boolean = false) extends ShuffledJoin with CodegenSupport {
4545

4646
override lazy val metrics = Map(
4747
"numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"))
@@ -72,26 +72,13 @@ case class SortMergeJoinExec(
7272
}
7373
}
7474

75-
override def outputPartitioning: Partitioning = joinType match {
76-
case _: InnerLike =>
77-
PartitioningCollection(Seq(left.outputPartitioning, right.outputPartitioning))
78-
// For left and right outer joins, the output is partitioned by the streamed input's join keys.
79-
case LeftOuter => left.outputPartitioning
80-
case RightOuter => right.outputPartitioning
81-
case FullOuter => UnknownPartitioning(left.outputPartitioning.numPartitions)
82-
case LeftExistence(_) => left.outputPartitioning
83-
case x =>
84-
throw new IllegalArgumentException(
85-
s"${getClass.getSimpleName} should not take $x as the JoinType")
86-
}
87-
8875
override def requiredChildDistribution: Seq[Distribution] = {
8976
if (isSkewJoin) {
9077
// We re-arrange the shuffle partitions to deal with skew join, and the new children
9178
// partitioning doesn't satisfy `HashClusteredDistribution`.
9279
UnspecifiedDistribution :: UnspecifiedDistribution :: Nil
9380
} else {
94-
HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil
81+
super.requiredChildDistribution
9582
}
9683
}
9784

0 commit comments

Comments
 (0)