-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-24556][SQL] Always rewrite output partitioning in ReusedExchangeExec and InMemoryTableScanExec #21564
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
f37139b
0ef99cc
405ba94
9da85e0
85dc1bc
7d1c2f2
6744a9e
dcd0ce9
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,13 +18,13 @@ | |
| package org.apache.spark.sql.execution | ||
|
|
||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.{execution, Row} | ||
| import org.apache.spark.sql.{execution, DataFrame, Row} | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.expressions._ | ||
| import org.apache.spark.sql.catalyst.plans._ | ||
| import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Range, Repartition, Sort, Union} | ||
| import org.apache.spark.sql.catalyst.plans.physical._ | ||
| import org.apache.spark.sql.execution.columnar.InMemoryRelation | ||
| import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} | ||
| import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReusedExchangeExec, ReuseExchange, ShuffleExchangeExec} | ||
| import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} | ||
| import org.apache.spark.sql.functions._ | ||
|
|
@@ -686,6 +686,66 @@ class PlannerSuite extends SharedSQLContext { | |
| Range(1, 2, 1, 1))) | ||
| df.queryExecution.executedPlan.execute() | ||
| } | ||
|
|
||
| test("SPARK-24556: always rewrite output partitioning in ReusedExchangeExec " + | ||
| "and InMemoryTableScanExec") { | ||
| def checkOutputPartitioningRewrite( | ||
| plans: Seq[SparkPlan], | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. now we can take a single spark plan
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. How do you think if we merge
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @cloud-fan I still use |
||
| expectedPartitioningClass: Class[_]): Unit = { | ||
| assert(plans.size == 1) | ||
| val plan = plans.head | ||
| val partitioning = plan.outputPartitioning | ||
| assert(partitioning.getClass == expectedPartitioningClass) | ||
| val partitionedAttrs = partitioning.asInstanceOf[Expression].references | ||
| assert(partitionedAttrs.subsetOf(plan.outputSet)) | ||
| } | ||
|
|
||
| def checkReusedExchangeOutputPartitioningRewrite( | ||
| df: DataFrame, | ||
| expectedPartitioningClass: Class[_]): Unit = { | ||
| val reusedExchange = df.queryExecution.executedPlan.collect { | ||
| case r: ReusedExchangeExec => r | ||
| } | ||
| checkOutputPartitioningRewrite(reusedExchange, expectedPartitioningClass) | ||
| } | ||
|
|
||
| def checkInMemoryTableScanOutputPartitioningRewrite( | ||
| df: DataFrame, | ||
| expectedPartitioningClass: Class[_]): Unit = { | ||
| val inMemoryScan = df.queryExecution.executedPlan.collect { | ||
| case m: InMemoryTableScanExec => m | ||
| } | ||
| checkOutputPartitioningRewrite(inMemoryScan, expectedPartitioningClass) | ||
| } | ||
|
|
||
| // ReusedExchange is HashPartitioning | ||
| val df1 = Seq(1 -> "a").toDF("i", "j").repartition($"i") | ||
| val df2 = Seq(1 -> "a").toDF("i", "j").repartition($"i") | ||
| checkReusedExchangeOutputPartitioningRewrite(df1.union(df2), classOf[HashPartitioning]) | ||
|
|
||
| // ReusedExchange is RangePartitioning | ||
| val df3 = Seq(1 -> "a").toDF("i", "j").orderBy($"i") | ||
| val df4 = Seq(1 -> "a").toDF("i", "j").orderBy($"i") | ||
| checkReusedExchangeOutputPartitioningRewrite(df3.union(df4), classOf[RangePartitioning]) | ||
|
|
||
| // InMemoryTableScan is HashPartitioning | ||
| Seq(1 -> "a").toDF("i", "j").repartition($"i").persist() | ||
| checkInMemoryTableScanOutputPartitioningRewrite( | ||
| Seq(1 -> "a").toDF("i", "j").repartition($"i"), classOf[HashPartitioning]) | ||
|
|
||
| // InMemoryTableScan is RangePartitioning | ||
| spark.range(1, 100, 1, 10).toDF().persist() | ||
| checkInMemoryTableScanOutputPartitioningRewrite( | ||
| spark.range(1, 100, 1, 10).toDF(), classOf[RangePartitioning]) | ||
|
|
||
| // InMemoryTableScan is PartitioningCollection | ||
| withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { | ||
| Seq(1 -> "a").toDF("i", "j").join(Seq(1 -> "a").toDF("m", "n"), $"i" === $"m").persist() | ||
| checkInMemoryTableScanOutputPartitioningRewrite( | ||
| Seq(1 -> "a").toDF("i", "j").join(Seq(1 -> "a").toDF("m", "n"), $"i" === $"m"), | ||
| classOf[PartitioningCollection]) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| // Used for unit-testing EnsureRequirements | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM