Skip to content

Commit f0f83b5

Browse files
EnricoMicloud-fan
authored andcommitted
[SPARK-40588] FileFormatWriter materializes AQE plan before accessing outputOrdering
The `FileFormatWriter` materializes an `AdaptiveQueryPlan` before accessing the plan's `outputOrdering`. This is required for Spark 3.0 to 3.3. Spark 3.4 does not need this because `FileFormatWriter` gets the final plan. `FileFormatWriter` enforces an ordering if the written plan does not provide that ordering. An `AdaptiveQueryPlan` does not know its final ordering (Spark 3.0 to 3.3), in which case `FileFormatWriter` enforces the ordering (e.g. by column `"a"`) even if the plan provides a compatible ordering (e.g. by columns `"a", "b"`). In case of spilling, that order (e.g. by columns `"a", "b"`) gets broken (see SPARK-40588). This fixes SPARK-40588, which was introduced in 3.0. This restores behaviour from Spark 2.4. The final plan that is written to files cannot be extracted from `FileFormatWriter`. The bug explained in [SPARK-40588](https://issues.apache.org/jira/browse/SPARK-40588) can only be asserted on the result files when spilling occurs. This is very hard to control in an unit test scenario. Therefore, this was tested manually. The [example to reproduce this issue](https://issues.apache.org/jira/browse/SPARK-40588?focusedCommentId=17621032&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17621032) given in SPARK-40588 now produces sorted files. The actual plan written into the files changed from ``` Sort [input[0, bigint, false] ASC NULLS FIRST], false, 0 +- AdaptiveSparkPlan isFinalPlan=false +- Sort [day#2L ASC NULLS FIRST, id#4L ASC NULLS FIRST], false, 0 +- Exchange hashpartitioning(day#2L, 2), REPARTITION_BY_NUM, [id=#30] +- BroadcastNestedLoopJoin BuildLeft, Inner :- BroadcastExchange IdentityBroadcastMode, [id=#28] : +- Project [id#0L AS day#2L] : +- Range (0, 2, step=1, splits=2) +- Range (0, 10000000, step=1, splits=2) ``` where `FileFormatWriter` enforces order with `Sort [input[0, bigint, false] ASC NULLS FIRST], false, 0`, to ``` *(3) Sort [day#2L ASC NULLS FIRST, id#4L ASC NULLS FIRST], false, 0 +- AQEShuffleRead coalesced +- ShuffleQueryStage 1 +- Exchange hashpartitioning(day#2L, 200), REPARTITION_BY_COL, [id=#68] +- *(2) BroadcastNestedLoopJoin BuildLeft, Inner :- BroadcastQueryStage 0 : +- BroadcastExchange IdentityBroadcastMode, [id=#42] : +- *(1) Project [id#0L AS day#2L] : +- *(1) Range (0, 2, step=1, splits=2) +- *(2) Range (0, 1000000, step=1, splits=2) ``` where the sort given by the user is the outermost sort now. Closes #38358 from EnricoMi/branch-3.3-materialize-aqe-plan. Authored-by: Enrico Minack <[email protected]> Signed-off-by: Wenchen Fan <[email protected]> (cherry picked from commit f0cad7a) Signed-off-by: Wenchen Fan <[email protected]>
1 parent f4ebe8f commit f0f83b5

File tree

2 files changed

+16
-4
lines changed

2 files changed

+16
-4
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -217,6 +217,8 @@ case class AdaptiveSparkPlanExec(
217217
.map(_.toLong).filter(SQLExecution.getQueryExecution(_) eq context.qe)
218218
}
219219

220+
def finalPhysicalPlan: SparkPlan = withFinalPlanUpdate(identity)
221+
220222
private def getFinalPhysicalPlan(): SparkPlan = lock.synchronized {
221223
if (isFinalPlan) return currentPhysicalPlan
222224

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala

Lines changed: 14 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -40,6 +40,7 @@ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
4040
import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils}
4141
import org.apache.spark.sql.errors.QueryExecutionErrors
4242
import org.apache.spark.sql.execution.{ProjectExec, SortExec, SparkPlan, SQLExecution, UnsafeExternalRowSorter}
43+
import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
4344
import org.apache.spark.sql.internal.SQLConf
4445
import org.apache.spark.sql.types.StringType
4546
import org.apache.spark.unsafe.types.UTF8String
@@ -160,8 +161,17 @@ object FileFormatWriter extends Logging {
160161

161162
// We should first sort by partition columns, then bucket id, and finally sorting columns.
162163
val requiredOrdering = partitionColumns ++ bucketIdExpression ++ sortColumns
164+
165+
// SPARK-40588: plan may contain an AdaptiveSparkPlanExec, which does not know
166+
// its final plan's ordering, so we have to materialize that plan first
167+
def materializeAdaptiveSparkPlan(plan: SparkPlan): SparkPlan = plan match {
168+
case a: AdaptiveSparkPlanExec => a.finalPhysicalPlan
169+
case p: SparkPlan => p.withNewChildren(p.children.map(materializeAdaptiveSparkPlan))
170+
}
171+
val materializedPlan = materializeAdaptiveSparkPlan(empty2NullPlan)
172+
163173
// the sort order doesn't matter
164-
val actualOrdering = empty2NullPlan.outputOrdering.map(_.child)
174+
val actualOrdering = materializedPlan.outputOrdering.map(_.child)
165175
val orderingMatched = if (requiredOrdering.length > actualOrdering.length) {
166176
false
167177
} else {
@@ -183,7 +193,7 @@ object FileFormatWriter extends Logging {
183193

184194
try {
185195
val (rdd, concurrentOutputWriterSpec) = if (orderingMatched) {
186-
(empty2NullPlan.execute(), None)
196+
(materializedPlan.execute(), None)
187197
} else {
188198
// SPARK-21165: the `requiredOrdering` is based on the attributes from analyzed plan, and
189199
// the physical plan may have different attribute ids due to optimizer removing some
@@ -193,12 +203,12 @@ object FileFormatWriter extends Logging {
193203
val sortPlan = SortExec(
194204
orderingExpr,
195205
global = false,
196-
child = empty2NullPlan)
206+
child = materializedPlan)
197207

198208
val maxWriters = sparkSession.sessionState.conf.maxConcurrentOutputFileWriters
199209
val concurrentWritersEnabled = maxWriters > 0 && sortColumns.isEmpty
200210
if (concurrentWritersEnabled) {
201-
(empty2NullPlan.execute(),
211+
(materializedPlan.execute(),
202212
Some(ConcurrentOutputWriterSpec(maxWriters, () => sortPlan.createSorter())))
203213
} else {
204214
(sortPlan.execute(), None)

0 commit comments

Comments
 (0)