Skip to content

Commit 71ff7d1

Browse files
committed
Code de-duplication among toString and toFile
1 parent d1188e3 commit 71ff7d1

File tree

1 file changed

+27
-28
lines changed

1 file changed

+27
-28
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala

Lines changed: 27 additions & 28 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
package org.apache.spark.sql.execution
1919

20-
import java.io.{BufferedWriter, OutputStreamWriter}
20+
import java.io.{OutputStreamWriter, StringWriter, Writer}
2121
import java.nio.charset.StandardCharsets
2222
import java.sql.{Date, Timestamp}
2323

@@ -102,7 +102,6 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) {
102102
protected def stringOrError[A](f: => A): String =
103103
try f.toString catch { case e: AnalysisException => e.toString }
104104

105-
106105
/**
107106
* Returns the result as a hive compatible sequence of strings. This is used in tests and
108107
* `SparkSQLDriver` for CLI applications.
@@ -193,23 +192,32 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) {
193192
""".stripMargin.trim
194193
}
195194

195+
private def writeOrError(writer: Writer)(f: Writer => Unit): Unit =
196+
try f(writer) catch { case e: AnalysisException => writer.write(e.toString) }
197+
198+
private def writePlans(writer: Writer): Unit = {
199+
writer.write("== Parsed Logical Plan ==\n")
200+
writeOrError(writer)(logical.treeString(_, verbose = true, addSuffix = false))
201+
writer.write("== Analyzed Logical Plan ==\n")
202+
writeOrError(writer) { writer =>
203+
analyzed.output.foreach(o => writer.write(s"${o.name}: ${o.dataType.simpleString}"))
204+
}
205+
writer.write("\n")
206+
writeOrError(writer)(analyzed.treeString(_, verbose = true, addSuffix = false))
207+
writer.write("== Optimized Logical Plan ==\n")
208+
writeOrError(writer)(optimizedPlan.treeString(_, verbose = true, addSuffix = false))
209+
writer.write("== Physical Plan ==\n")
210+
writeOrError(writer)(executedPlan.treeString(_, verbose = true, addSuffix = false))
211+
}
212+
196213
override def toString: String = withRedaction {
197-
def output = Utils.truncatedString(
198-
analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}"), ", ")
199-
val analyzedPlan = Seq(
200-
stringOrError(output),
201-
stringOrError(analyzed.treeString(verbose = true))
202-
).filter(_.nonEmpty).mkString("\n")
203-
204-
s"""== Parsed Logical Plan ==
205-
|${stringOrError(logical.treeString(verbose = true))}
206-
|== Analyzed Logical Plan ==
207-
|$analyzedPlan
208-
|== Optimized Logical Plan ==
209-
|${stringOrError(optimizedPlan.treeString(verbose = true))}
210-
|== Physical Plan ==
211-
|${stringOrError(executedPlan.treeString(verbose = true))}
212-
""".stripMargin.trim
214+
val writer = new StringWriter()
215+
try {
216+
writePlans(writer)
217+
writer.toString
218+
} finally {
219+
writer.close()
220+
}
213221
}
214222

215223
def stringWithStats: String = withRedaction {
@@ -267,16 +275,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) {
267275

268276
try {
269277
SparkEnv.get.conf.set(Utils.MAX_TO_STRING_FIELDS, Int.MaxValue.toString)
270-
writer.write("== Parsed Logical Plan ==\n")
271-
logical.treeString(writer, verbose = true, addSuffix = false)
272-
writer.write("== Analyzed Logical Plan ==\n")
273-
analyzed.output.foreach(o => writer.write(s"${o.name}: ${o.dataType.simpleString}"))
274-
writer.write("\n")
275-
analyzed.treeString(writer, verbose = true, addSuffix = false)
276-
writer.write("== Optimized Logical Plan ==\n")
277-
optimizedPlan.treeString(writer, verbose = true, addSuffix = false)
278-
writer.write("== Physical Plan ==\n")
279-
executedPlan.treeString(writer, verbose = true, addSuffix = false)
278+
writePlans(writer)
280279
writer.write("== Whole Stage Codegen ==\n")
281280
org.apache.spark.sql.execution.debug.writeCodegen(writer, executedPlan)
282281
} finally {

0 commit comments

Comments
 (0)