Skip to content

Commit 56448c6

Browse files
rednaxelafxcloud-fan
authored andcommitted
[SPARK-26352][SQL] join reorder should not change the order of output attributes
## What changes were proposed in this pull request? The optimizer rule `org.apache.spark.sql.catalyst.optimizer.ReorderJoin` performs join reordering on inner joins. This was introduced from SPARK-12032 (#10073) in 2015-12. After it had reordered the joins, though, it didn't check whether or not the output attribute order is still the same as before. Thus, it's possible to have a mismatch between the reordered output attributes order vs the schema that a DataFrame thinks it has. The same problem exists in the CBO version of join reordering (`CostBasedJoinReorder`) too. This can be demonstrated with the example: ```scala spark.sql("create table table_a (x int, y int) using parquet") spark.sql("create table table_b (i int, j int) using parquet") spark.sql("create table table_c (a int, b int) using parquet") val df = spark.sql(""" with df1 as (select * from table_a cross join table_b) select * from df1 join table_c on a = x and b = i """) ``` here's what the DataFrame thinks: ``` scala> df.printSchema root |-- x: integer (nullable = true) |-- y: integer (nullable = true) |-- i: integer (nullable = true) |-- j: integer (nullable = true) |-- a: integer (nullable = true) |-- b: integer (nullable = true) ``` here's what the optimized plan thinks, after join reordering: ``` scala> df.queryExecution.optimizedPlan.output.foreach(a => println(s"|-- ${a.name}: ${a.dataType.typeName}")) |-- x: integer |-- y: integer |-- a: integer |-- b: integer |-- i: integer |-- j: integer ``` If we exclude the `ReorderJoin` rule (using Spark 2.4's optimizer rule exclusion feature), it's back to normal: ``` scala> spark.conf.set("spark.sql.optimizer.excludedRules", "org.apache.spark.sql.catalyst.optimizer.ReorderJoin") scala> val df = spark.sql("with df1 as (select * from table_a cross join table_b) select * from df1 join table_c on a = x and b = i") df: org.apache.spark.sql.DataFrame = [x: int, y: int ... 4 more fields] scala> df.queryExecution.optimizedPlan.output.foreach(a => println(s"|-- ${a.name}: ${a.dataType.typeName}")) |-- x: integer |-- y: integer |-- i: integer |-- j: integer |-- a: integer |-- b: integer ``` Note that this output attribute ordering problem leads to data corruption, and can manifest itself in various symptoms: * Silently corrupting data, if the reordered columns happen to either have matching types or have sufficiently-compatible types (e.g. all fixed length primitive types are considered as "sufficiently compatible" in an `UnsafeRow`), then only the resulting data is going to be wrong but it might not trigger any alarms immediately. Or * Weird Java-level exceptions like `java.lang.NegativeArraySizeException`, or even SIGSEGVs. ## How was this patch tested? Added new unit test in `JoinReorderSuite` and new end-to-end test in `JoinSuite`. Also made `JoinReorderSuite` and `StarJoinReorderSuite` assert more strongly on maintaining output attribute order. Closes #23303 from rednaxelafx/fix-join-reorder. Authored-by: Kris Mok <rednaxelafx@gmail.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
1 parent db1c5b1 commit 56448c6

File tree

7 files changed

+116
-10
lines changed

7 files changed

+116
-10
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -48,6 +48,7 @@ object CostBasedJoinReorder extends Rule[LogicalPlan] with PredicateHelper {
4848
if projectList.forall(_.isInstanceOf[Attribute]) =>
4949
reorder(p, p.output)
5050
}
51+
5152
// After reordering is finished, convert OrderedJoin back to Join
5253
result transformDown {
5354
case OrderedJoin(left, right, jt, cond) => Join(left, right, jt, cond)
@@ -175,11 +176,20 @@ object JoinReorderDP extends PredicateHelper with Logging {
175176
assert(topOutputSet == p.outputSet)
176177
// Keep the same order of final output attributes.
177178
p.copy(projectList = output)
179+
case finalPlan if !sameOutput(finalPlan, output) =>
180+
Project(output, finalPlan)
178181
case finalPlan =>
179182
finalPlan
180183
}
181184
}
182185

186+
private def sameOutput(plan: LogicalPlan, expectedOutput: Seq[Attribute]): Boolean = {
187+
val thisOutput = plan.output
188+
thisOutput.length == expectedOutput.length && thisOutput.zip(expectedOutput).forall {
189+
case (a1, a2) => a1.semanticEquals(a2)
190+
}
191+
}
192+
183193
/** Find all possible plans at the next level, based on existing levels. */
184194
private def searchLevel(
185195
existingLevels: Seq[JoinPlanMap],

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala

Lines changed: 10 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -86,9 +86,9 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper {
8686
}
8787

8888
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
89-
case ExtractFiltersAndInnerJoins(input, conditions)
89+
case p @ ExtractFiltersAndInnerJoins(input, conditions)
9090
if input.size > 2 && conditions.nonEmpty =>
91-
if (SQLConf.get.starSchemaDetection && !SQLConf.get.cboEnabled) {
91+
val reordered = if (SQLConf.get.starSchemaDetection && !SQLConf.get.cboEnabled) {
9292
val starJoinPlan = StarSchemaDetection.reorderStarJoins(input, conditions)
9393
if (starJoinPlan.nonEmpty) {
9494
val rest = input.filterNot(starJoinPlan.contains(_))
@@ -99,6 +99,14 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper {
9999
} else {
100100
createOrderedJoin(input, conditions)
101101
}
102+
103+
if (p.sameOutput(reordered)) {
104+
reordered
105+
} else {
106+
// Reordering the joins have changed the order of the columns.
107+
// Inject a projection to make sure we restore to the expected ordering.
108+
Project(p.output, reordered)
109+
}
102110
}
103111
}
104112

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -102,16 +102,19 @@ class JoinOptimizationSuite extends PlanTest {
102102
x.join(y).join(z).where(("x.b".attr === "z.b".attr) && ("y.d".attr === "z.a".attr)),
103103
x.join(z, condition = Some("x.b".attr === "z.b".attr))
104104
.join(y, condition = Some("y.d".attr === "z.a".attr))
105+
.select(Seq("x.a", "x.b", "x.c", "y.d", "z.a", "z.b", "z.c").map(_.attr): _*)
105106
),
106107
(
107108
x.join(y, Cross).join(z, Cross)
108109
.where(("x.b".attr === "z.b".attr) && ("y.d".attr === "z.a".attr)),
109110
x.join(z, Cross, Some("x.b".attr === "z.b".attr))
110111
.join(y, Cross, Some("y.d".attr === "z.a".attr))
112+
.select(Seq("x.a", "x.b", "x.c", "y.d", "z.a", "z.b", "z.c").map(_.attr): _*)
111113
),
112114
(
113115
x.join(y, Inner).join(z, Cross).where("x.b".attr === "z.a".attr),
114116
x.join(z, Cross, Some("x.b".attr === "z.a".attr)).join(y, Inner)
117+
.select(Seq("x.a", "x.b", "x.c", "y.d", "z.a", "z.b", "z.c").map(_.attr): _*)
115118
)
116119
)
117120

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala

Lines changed: 34 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -20,8 +20,8 @@ package org.apache.spark.sql.catalyst.optimizer
2020
import org.apache.spark.sql.catalyst.dsl.expressions._
2121
import org.apache.spark.sql.catalyst.dsl.plans._
2222
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap}
23-
import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest}
24-
import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan}
23+
import org.apache.spark.sql.catalyst.plans.{Cross, Inner, PlanTest}
24+
import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LocalRelation, LogicalPlan}
2525
import org.apache.spark.sql.catalyst.rules.RuleExecutor
2626
import org.apache.spark.sql.catalyst.statsEstimation.{StatsEstimationTestBase, StatsTestPlan}
2727
import org.apache.spark.sql.internal.SQLConf.{CBO_ENABLED, JOIN_REORDER_ENABLED}
@@ -124,7 +124,8 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase {
124124
// the original order (t1 J t2) J t3.
125125
val bestPlan =
126126
t1.join(t3, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100")))
127-
.join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")))
127+
.join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")))
128+
.select(outputsOf(t1, t2, t3): _*)
128129

129130
assertEqualPlans(originalPlan, bestPlan)
130131
}
@@ -139,7 +140,9 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase {
139140
val bestPlan =
140141
t1.join(t3, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100")))
141142
.join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")))
143+
.select(outputsOf(t1, t2, t3): _*) // this is redundant but we'll take it for now
142144
.join(t4)
145+
.select(outputsOf(t1, t2, t4, t3): _*)
143146

144147
assertEqualPlans(originalPlan, bestPlan)
145148
}
@@ -202,6 +205,7 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase {
202205
t1.join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")))
203206
.join(t4.join(t3, Inner, Some(nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))),
204207
Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2")))
208+
.select(outputsOf(t1, t4, t2, t3): _*)
205209

206210
assertEqualPlans(originalPlan, bestPlan)
207211
}
@@ -219,6 +223,23 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase {
219223
}
220224
}
221225

226+
test("SPARK-26352: join reordering should not change the order of attributes") {
227+
// This test case does not rely on CBO.
228+
// It's similar to the test case above, but catches a reordering bug that the one above doesn't
229+
val tab1 = LocalRelation('x.int, 'y.int)
230+
val tab2 = LocalRelation('i.int, 'j.int)
231+
val tab3 = LocalRelation('a.int, 'b.int)
232+
val original =
233+
tab1.join(tab2, Cross)
234+
.join(tab3, Inner, Some('a === 'x && 'b === 'i))
235+
val expected =
236+
tab1.join(tab3, Inner, Some('a === 'x))
237+
.join(tab2, Cross, Some('b === 'i))
238+
.select(outputsOf(tab1, tab2, tab3): _*)
239+
240+
assertEqualPlans(original, expected)
241+
}
242+
222243
test("reorder recursively") {
223244
// Original order:
224245
// Join
@@ -266,8 +287,17 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase {
266287
private def assertEqualPlans(
267288
originalPlan: LogicalPlan,
268289
groundTruthBestPlan: LogicalPlan): Unit = {
269-
val optimized = Optimize.execute(originalPlan.analyze)
290+
val analyzed = originalPlan.analyze
291+
val optimized = Optimize.execute(analyzed)
270292
val expected = groundTruthBestPlan.analyze
293+
294+
assert(analyzed.sameOutput(expected)) // if this fails, the expected plan itself is incorrect
295+
assert(analyzed.sameOutput(optimized))
296+
271297
compareJoinOrder(optimized, expected)
272298
}
299+
300+
private def outputsOf(plans: LogicalPlan*): Seq[Attribute] = {
301+
plans.map(_.output).reduce(_ ++ _)
302+
}
273303
}

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala

Lines changed: 20 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -218,6 +218,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas
218218
.join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk")))
219219
.join(t2, Inner, Some(nameToAttr("f1_c2") === nameToAttr("t2_c1")))
220220
.join(t1, Inner, Some(nameToAttr("f1_c1") === nameToAttr("t1_c1")))
221+
.select(outputsOf(f1, t1, t2, d1, d2): _*)
221222

222223
assertEqualPlans(query, expected)
223224
}
@@ -256,6 +257,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas
256257
.join(t3.join(t2, Inner, Some(nameToAttr("t2_c2") === nameToAttr("t3_c1"))), Inner,
257258
Some(nameToAttr("d1_c2") === nameToAttr("t2_c1")))
258259
.join(t1, Inner, Some(nameToAttr("t1_c1") === nameToAttr("f1_c1")))
260+
.select(outputsOf(d1, t1, t2, f1, d2, t3): _*)
259261

260262
assertEqualPlans(query, expected)
261263
}
@@ -297,6 +299,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas
297299
Some(nameToAttr("t3_c1") === nameToAttr("t4_c1")))
298300
.join(t1.join(t2, Inner, Some(nameToAttr("t1_c1") === nameToAttr("t2_c1"))), Inner,
299301
Some(nameToAttr("t1_c2") === nameToAttr("t4_c2")))
302+
.select(outputsOf(d1, t1, t2, t3, t4, f1, d2): _*)
300303

301304
assertEqualPlans(query, expected)
302305
}
@@ -347,6 +350,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas
347350
Some(nameToAttr("d3_c2") === nameToAttr("t1_c1")))
348351
.join(t5.join(t6, Inner, Some(nameToAttr("t5_c2") === nameToAttr("t6_c2"))), Inner,
349352
Some(nameToAttr("d2_c2") === nameToAttr("t5_c1")))
353+
.select(outputsOf(d1, t3, t4, f1, d2, t5, t6, d3, t1, t2): _*)
350354

351355
assertEqualPlans(query, expected)
352356
}
@@ -375,6 +379,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas
375379
f1.join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk")))
376380
.join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk")))
377381
.join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk")))
382+
.select(outputsOf(d1, d2, f1, d3): _*)
378383

379384
assertEqualPlans(query, expected)
380385
}
@@ -400,13 +405,27 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas
400405
f1.join(t3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("t3_c1")))
401406
.join(t2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("t2_c1")))
402407
.join(t1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("t1_c1")))
408+
.select(outputsOf(t1, f1, t2, t3): _*)
403409

404410
assertEqualPlans(query, expected)
405411
}
406412

407413
private def assertEqualPlans( plan1: LogicalPlan, plan2: LogicalPlan): Unit = {
408-
val optimized = Optimize.execute(plan1.analyze)
414+
val analyzed = plan1.analyze
415+
val optimized = Optimize.execute(analyzed)
409416
val expected = plan2.analyze
417+
418+
assert(equivalentOutput(analyzed, expected)) // if this fails, the expected itself is incorrect
419+
assert(equivalentOutput(analyzed, optimized))
420+
410421
compareJoinOrder(optimized, expected)
411422
}
423+
424+
private def outputsOf(plans: LogicalPlan*): Seq[Attribute] = {
425+
plans.map(_.output).reduce(_ ++ _)
426+
}
427+
428+
private def equivalentOutput(plan1: LogicalPlan, plan2: LogicalPlan): Boolean = {
429+
normalizeExprIds(plan1).output == normalizeExprIds(plan2).output
430+
}
412431
}

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala

Lines changed: 25 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -182,6 +182,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase {
182182
.join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk1")))
183183
.join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk1")))
184184
.join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1")))
185+
.select(outputsOf(d1, d2, f1, d3, s3): _*)
185186

186187
assertEqualPlans(query, expected)
187188
}
@@ -220,6 +221,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase {
220221
.join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk1")))
221222
.join(d2, Inner, Some(nameToAttr("f1_fk2") < nameToAttr("d2_pk1")))
222223
.join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1")))
224+
.select(outputsOf(d1, f1, d2, s3, d3): _*)
223225

224226
assertEqualPlans(query, expected)
225227
}
@@ -255,7 +257,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase {
255257
.join(d3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1")))
256258
.join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1")))
257259
.join(s3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("s3_c2")))
258-
260+
.select(outputsOf(d1, f1, d2, s3, d3): _*)
259261

260262
assertEqualPlans(query, expected)
261263
}
@@ -292,6 +294,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase {
292294
.join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk1")))
293295
.join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_c2")))
294296
.join(s3, Inner, Some(nameToAttr("d3_fk1") < nameToAttr("s3_pk1")))
297+
.select(outputsOf(d1, f1, d2, s3, d3): _*)
295298

296299
assertEqualPlans(query, expected)
297300
}
@@ -395,6 +398,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase {
395398
.join(d2.where(nameToAttr("d2_c2") === 2), Inner,
396399
Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1")))
397400
.join(s3, Inner, Some(nameToAttr("f11_fk1") === nameToAttr("s3_pk1")))
401+
.select(outputsOf(d1, f11, f1, d2, s3): _*)
398402

399403
assertEqualPlans(query, equivQuery)
400404
}
@@ -430,6 +434,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase {
430434
.join(d2.where(nameToAttr("d2_c2") === 2), Inner,
431435
Some(nameToAttr("f1_fk2") === nameToAttr("d2_c4")))
432436
.join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1")))
437+
.select(outputsOf(d1, d3, f1, d2, s3): _*)
433438

434439
assertEqualPlans(query, expected)
435440
}
@@ -465,6 +470,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase {
465470
.join(d2.where(nameToAttr("d2_c2") === 2), Inner,
466471
Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1")))
467472
.join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1")))
473+
.select(outputsOf(d1, d3, f1, d2, s3): _*)
468474

469475
assertEqualPlans(query, expected)
470476
}
@@ -499,6 +505,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase {
499505
.join(d2.where(nameToAttr("d2_c2") === 2),
500506
Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1")))
501507
.join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1")))
508+
.select(outputsOf(d1, d3, f1, d2, s3): _*)
502509

503510
assertEqualPlans(query, expected)
504511
}
@@ -532,6 +539,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase {
532539
.join(d3, Inner, Some(nameToAttr("f1_fk3") < nameToAttr("d3_pk1")))
533540
.join(d2, Inner, Some(nameToAttr("f1_fk2") < nameToAttr("d2_pk1")))
534541
.join(s3, Inner, Some(nameToAttr("d3_fk1") < nameToAttr("s3_pk1")))
542+
.select(outputsOf(d1, d3, f1, d2, s3): _*)
535543

536544
assertEqualPlans(query, expected)
537545
}
@@ -565,13 +573,27 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase {
565573
.join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk1")))
566574
.join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1")))
567575
.join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1")))
576+
.select(outputsOf(d1, d3, f1, d2, s3): _*)
568577

569578
assertEqualPlans(query, expected)
570579
}
571580

572-
private def assertEqualPlans( plan1: LogicalPlan, plan2: LogicalPlan): Unit = {
573-
val optimized = Optimize.execute(plan1.analyze)
581+
private def assertEqualPlans(plan1: LogicalPlan, plan2: LogicalPlan): Unit = {
582+
val analyzed = plan1.analyze
583+
val optimized = Optimize.execute(analyzed)
574584
val expected = plan2.analyze
585+
586+
assert(equivalentOutput(analyzed, expected)) // if this fails, the expected itself is incorrect
587+
assert(equivalentOutput(analyzed, optimized))
588+
575589
compareJoinOrder(optimized, expected)
576590
}
591+
592+
private def outputsOf(plans: LogicalPlan*): Seq[Attribute] = {
593+
plans.map(_.output).reduce(_ ++ _)
594+
}
595+
596+
private def equivalentOutput(plan1: LogicalPlan, plan2: LogicalPlan): Boolean = {
597+
normalizeExprIds(plan1).output == normalizeExprIds(plan2).output
598+
}
577599
}

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

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -895,4 +895,18 @@ class JoinSuite extends QueryTest with SharedSQLContext {
895895
checkAnswer(res, Row(0, 0, 0))
896896
}
897897
}
898+
899+
test("SPARK-26352: join reordering should not change the order of columns") {
900+
withTable("tab1", "tab2", "tab3") {
901+
spark.sql("select 1 as x, 100 as y").write.saveAsTable("tab1")
902+
spark.sql("select 42 as i, 200 as j").write.saveAsTable("tab2")
903+
spark.sql("select 1 as a, 42 as b").write.saveAsTable("tab3")
904+
905+
val df = spark.sql("""
906+
with tmp as (select * from tab1 cross join tab2)
907+
select * from tmp join tab3 on a = x and b = i
908+
""")
909+
checkAnswer(df, Row(1, 100, 42, 200, 1, 42))
910+
}
911+
}
898912
}

0 commit comments

Comments
 (0)