Skip to content

Commit b57c93b

Browse files
committed
[SPARK-38772][SQL] Formatting the log plan in AdaptiveSparkPlanExec
### What changes were proposed in this pull request? Use sideBySide to format the log plan in `AdaptiveSparkPlanExec`. Before: ``` 12:08:36.876 ERROR org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec: Plan changed from SortMergeJoin [key#13], [a#23], Inner :- Sort [key#13 ASC NULLS FIRST], false, 0 : +- ShuffleQueryStage 0 : +- Exchange hashpartitioning(key#13, 5), ENSURE_REQUIREMENTS, [id=#110] : +- *(1) Filter (isnotnull(value#14) AND (value#14 = 1)) : +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).value, true, false, true) AS value#14] : +- Scan[obj#12] +- Sort [a#23 ASC NULLS FIRST], false, 0 +- ShuffleQueryStage 1 +- Exchange hashpartitioning(a#23, 5), ENSURE_REQUIREMENTS, [id=#129] +- *(2) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData2, true])).a AS a#23, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData2, true])).b AS b#24] +- Scan[obj#22] to BroadcastHashJoin [key#13], [a#23], Inner, BuildLeft, false :- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#145] : +- ShuffleQueryStage 0 : +- Exchange hashpartitioning(key#13, 5), ENSURE_REQUIREMENTS, [id=#110] : +- *(1) Filter (isnotnull(value#14) AND (value#14 = 1)) : +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).value, true, false, true) AS value#14] : +- Scan[obj#12] +- ShuffleQueryStage 1 +- Exchange hashpartitioning(a#23, 5), ENSURE_REQUIREMENTS, [id=#129] +- *(2) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData2, true])).a AS a#23, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData2, true])).b AS b#24] +- Scan[obj#22] ``` After: ``` 15:57:59.481 ERROR org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec: Plan changed: !SortMergeJoin [key#13], [a#23], Inner BroadcastHashJoin [key#13], [a#23], Inner, BuildLeft, false !:- Sort [key#13 ASC NULLS FIRST], false, 0 :- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#145] : +- ShuffleQueryStage 0 : +- ShuffleQueryStage 0 : +- Exchange hashpartitioning(key#13, 5), ENSURE_REQUIREMENTS, [id=#110] : +- Exchange hashpartitioning(key#13, 5), ENSURE_REQUIREMENTS, [id=#110] : +- *(1) Filter (isnotnull(value#14) AND (value#14 = 1)) : +- *(1) Filter (isnotnull(value#14) AND (value#14 = 1)) : +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).value, true, false, true) AS value#14] : +- *(1) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).key AS key#13, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData, true])).value, true, false, true) AS value#14] : +- Scan[obj#12] : +- Scan[obj#12] !+- Sort [a#23 ASC NULLS FIRST], false, 0 +- ShuffleQueryStage 1 ! +- ShuffleQueryStage 1 +- Exchange hashpartitioning(a#23, 5), ENSURE_REQUIREMENTS, [id=#129] ! +- Exchange hashpartitioning(a#23, 5), ENSURE_REQUIREMENTS, [id=#129] +- *(2) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData2, true])).a AS a#23, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData2, true])).b AS b#24] ! +- *(2) SerializeFromObject [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData2, true])).a AS a#23, knownnotnull(assertnotnull(input[0, org.apache.spark.sql.test.SQLTestData$TestData2, true])).b AS b#24] +- Scan[obj#22] ! +- Scan[obj#22] ``` ### Why are the changes needed? Enhance readability. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual testing. Closes #36045 from wangyum/SPARK-38772. Authored-by: Yuming Wang <[email protected]> Signed-off-by: Yuming Wang <[email protected]>
1 parent 6d9bfb6 commit b57c93b

File tree

1 file changed

+4
-2
lines changed

1 file changed

+4
-2
lines changed

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

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer}
3535
import org.apache.spark.sql.catalyst.plans.physical.{Distribution, UnspecifiedDistribution}
3636
import org.apache.spark.sql.catalyst.rules.{PlanChangeLogger, Rule}
3737
import org.apache.spark.sql.catalyst.trees.TreeNodeTag
38+
import org.apache.spark.sql.catalyst.util.sideBySide
3839
import org.apache.spark.sql.errors.QueryExecutionErrors
3940
import org.apache.spark.sql.execution._
4041
import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec._
@@ -306,7 +307,8 @@ case class AdaptiveSparkPlanExec(
306307
val newCost = costEvaluator.evaluateCost(newPhysicalPlan)
307308
if (newCost < origCost ||
308309
(newCost == origCost && currentPhysicalPlan != newPhysicalPlan)) {
309-
logOnLevel(s"Plan changed from $currentPhysicalPlan to $newPhysicalPlan")
310+
logOnLevel("Plan changed:\n" +
311+
sideBySide(currentPhysicalPlan.treeString, newPhysicalPlan.treeString).mkString("\n"))
310312
cleanUpTempTags(newPhysicalPlan)
311313
currentPhysicalPlan = newPhysicalPlan
312314
currentLogicalPlan = newLogicalPlan
@@ -335,7 +337,7 @@ case class AdaptiveSparkPlanExec(
335337
if (!isSubquery && currentPhysicalPlan.exists(_.subqueries.nonEmpty)) {
336338
getExecutionId.foreach(onUpdatePlan(_, Seq.empty))
337339
}
338-
logOnLevel(s"Final plan: $currentPhysicalPlan")
340+
logOnLevel(s"Final plan:\n$currentPhysicalPlan")
339341
}
340342

341343
override def executeCollect(): Array[InternalRow] = {

0 commit comments

Comments
 (0)