Skip to content

Commit 925203b

Browse files
committed
address comments
1 parent 07ce92f commit 925203b

File tree

3 files changed

+16
-9
lines changed

3 files changed

+16
-9
lines changed

sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -145,10 +145,10 @@ private[sql] class SQLConf extends Serializable {
145145
getConf(AUTO_BROADCASTJOIN_THRESHOLD, (10 * 1024 * 1024).toString).toInt
146146

147147
/**
148-
* By default it will choose sort merge join.
148+
* By default not choose sort merge join.
149149
*/
150150
private[spark] def autoSortMergeJoin: Boolean =
151-
getConf(AUTO_SORTMERGEJOIN, true.toString).toBoolean
151+
getConf(AUTO_SORTMERGEJOIN, false.toString).toBoolean
152152

153153
/**
154154
* The default size in bytes to assign to a logical operator's estimation statistics. By default,

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

Lines changed: 1 addition & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -140,14 +140,11 @@ case class SortMergeJoin(
140140
stop = ordering.compare(leftKey, rightKey) == 0
141141
}
142142
if (rightMatches.size > 0) {
143-
stop = false
144143
leftMatches = new CompactBuffer[Row]()
145144
val leftMatch = leftKey.copy()
146-
while (!stop && leftElement != null) {
145+
while (ordering.compare(leftKey, leftMatch) == 0 && leftElement != null) {
147146
leftMatches += leftElement
148147
fetchLeft()
149-
// exit loop when run out of left matches
150-
stop = ordering.compare(leftKey, leftMatch) != 0
151148
}
152149
}
153150

sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala

Lines changed: 13 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -51,6 +51,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
5151
case j: CartesianProduct => j
5252
case j: BroadcastNestedLoopJoin => j
5353
case j: BroadcastLeftSemiJoinHash => j
54+
case j: ShuffledHashJoin => j
5455
case j: SortMergeJoin => j
5556
}
5657

@@ -63,6 +64,8 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
6364
test("join operator selection") {
6465
cacheManager.clearCache()
6566

67+
val AUTO_SORTMERGEJOIN: Boolean = conf.autoSortMergeJoin
68+
conf.setConf("spark.sql.autoSortMergeJoin", "false")
6669
Seq(
6770
("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[LeftSemiJoinHash]),
6871
("SELECT * FROM testData LEFT SEMI JOIN testData2", classOf[LeftSemiJoinBNL]),
@@ -76,9 +79,9 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
7679
("SELECT * FROM testData FULL OUTER JOIN testData2 WHERE key = 2", classOf[CartesianProduct]),
7780
("SELECT * FROM testData JOIN testData2 WHERE key > a", classOf[CartesianProduct]),
7881
("SELECT * FROM testData FULL OUTER JOIN testData2 WHERE key > a", classOf[CartesianProduct]),
79-
("SELECT * FROM testData JOIN testData2 ON key = a", classOf[SortMergeJoin]),
80-
("SELECT * FROM testData JOIN testData2 ON key = a and key = 2", classOf[SortMergeJoin]),
81-
("SELECT * FROM testData JOIN testData2 ON key = a where key = 2", classOf[SortMergeJoin]),
82+
("SELECT * FROM testData JOIN testData2 ON key = a", classOf[ShuffledHashJoin]),
83+
("SELECT * FROM testData JOIN testData2 ON key = a and key = 2", classOf[ShuffledHashJoin]),
84+
("SELECT * FROM testData JOIN testData2 ON key = a where key = 2", classOf[ShuffledHashJoin]),
8285
("SELECT * FROM testData LEFT JOIN testData2 ON key = a", classOf[HashOuterJoin]),
8386
("SELECT * FROM testData RIGHT JOIN testData2 ON key = a where key = 2",
8487
classOf[HashOuterJoin]),
@@ -92,6 +95,13 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach {
9295
("SELECT * FROM testData full JOIN testData2 ON (key * a != key + a)",
9396
classOf[BroadcastNestedLoopJoin])
9497
).foreach { case (query, joinClass) => assertJoin(query, joinClass) }
98+
conf.setConf("spark.sql.autoSortMergeJoin", "true")
99+
Seq(
100+
("SELECT * FROM testData JOIN testData2 ON key = a", classOf[SortMergeJoin]),
101+
("SELECT * FROM testData JOIN testData2 ON key = a and key = 2", classOf[SortMergeJoin]),
102+
("SELECT * FROM testData JOIN testData2 ON key = a where key = 2", classOf[SortMergeJoin])
103+
).foreach { case (query, joinClass) => assertJoin(query, joinClass) }
104+
conf.setConf("spark.sql.autoSortMergeJoin", AUTO_SORTMERGEJOIN.toString)
95105
}
96106

97107
test("broadcasted hash join operator selection") {

0 commit comments

Comments
 (0)