[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user HyukjinKwon commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r232604420 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala --- @@ -176,9 +176,9 @@ case class TakeOrderedAndProjectExec( override def outputPartitioning: Partitioning = SinglePartition - override def simpleString: String = { -val orderByString = Utils.truncatedString(sortOrder, "[", ",", "]") -val outputString = Utils.truncatedString(output, "[", ",", "]") + override def simpleString(maxFields: Option[Int]): String = { --- End diff -- Can we just get rid of the `maxFields`? I think this makes the PR hard to read. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user MaxGekk commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r224418619 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala --- @@ -167,6 +172,58 @@ package object util { builder.toString() } + /** + * The performance overhead of creating and logging strings for wide schemas can be large. To + * limit the impact, we bound the number of fields to include by default. This can be overridden + * by setting the 'spark.debug.maxToStringFields' conf in SparkEnv or by settings the SQL config + * `spark.sql.debug.maxToStringFields`. + */ + private[spark] def maxNumToStringFields: Int = { +val legacyLimit = if (SparkEnv.get != null) { --- End diff -- I removed old core config and leaved only SQL config --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user MaxGekk commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r224415554 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala --- @@ -52,7 +52,7 @@ case class CatalystDataToAvro(child: Expression) extends UnaryExpression { out.toByteArray } - override def simpleString: String = { + override def simpleString(maxFields: Option[Int]): String = { s"to_avro(${child.sql}, ${child.dataType.simpleString})" --- End diff -- passed --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user MaxGekk commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r224415510 --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala --- @@ -633,4 +633,14 @@ package object config { .stringConf .toSequence .createWithDefault(Nil) + + private[spark] val MAX_TO_STRING_FIELDS = +ConfigBuilder("spark.debug.maxToStringFields") + .internal() + .doc("Maximum number of fields of sequence-like entries that can be converted to strings " + --- End diff -- I am going to change it to `Maximum number of fields of a tree node that can be ...` for the SQL config. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user MaxGekk commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r224391712 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala --- @@ -167,6 +172,58 @@ package object util { builder.toString() } + /** + * The performance overhead of creating and logging strings for wide schemas can be large. To + * limit the impact, we bound the number of fields to include by default. This can be overridden + * by setting the 'spark.debug.maxToStringFields' conf in SparkEnv or by settings the SQL config + * `spark.sql.debug.maxToStringFields`. + */ + private[spark] def maxNumToStringFields: Int = { +val legacyLimit = if (SparkEnv.get != null) { --- End diff -- Taking into account that old config wasn't well documented and could use mostly in debugging, I think we can remove it in Spark 3.0. Initially the PR targeted to Spark 2.4, in the minor version removing a public config can break user apps potentially. If you are ok to remove it, I will do that. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user MaxGekk commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r224389798 --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala --- @@ -633,4 +633,14 @@ package object config { .stringConf .toSequence .createWithDefault(Nil) + + private[spark] val MAX_TO_STRING_FIELDS = +ConfigBuilder("spark.debug.maxToStringFields") + .internal() + .doc("Maximum number of fields of sequence-like entries that can be converted to strings " + --- End diff -- I don't know how else I can describe all kind of classes where the parameter is applicable. If you have better words, you are welcome. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user hvanhovell commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r223983702 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala --- @@ -189,23 +192,34 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { """.stripMargin.trim } + private def writeOrError(writer: Writer)(f: Writer => Unit): Unit = +try f(writer) catch { case e: AnalysisException => writer.write(e.toString) } --- End diff -- Please use multiple lines here. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user hvanhovell commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r223983537 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala --- @@ -167,6 +172,58 @@ package object util { builder.toString() } + /** + * The performance overhead of creating and logging strings for wide schemas can be large. To + * limit the impact, we bound the number of fields to include by default. This can be overridden + * by setting the 'spark.debug.maxToStringFields' conf in SparkEnv or by settings the SQL config + * `spark.sql.debug.maxToStringFields`. + */ + private[spark] def maxNumToStringFields: Int = { +val legacyLimit = if (SparkEnv.get != null) { + SparkEnv.get.conf.get(config.MAX_TO_STRING_FIELDS) +} else { + config.MAX_TO_STRING_FIELDS.defaultValue.get +} +val sqlConfLimit = SQLConf.get.maxToStringFields + +Math.max(sqlConfLimit, legacyLimit) + } + + /** Whether we have warned about plan string truncation yet. */ + private val truncationWarningPrinted = new AtomicBoolean(false) + + /** + * Format a sequence with semantics similar to calling .mkString(). Any elements beyond + * maxNumToStringFields will be dropped and replaced by a "... N more fields" placeholder. + * + * @return the trimmed and formatted string. + */ + def truncatedString[T]( + seq: Seq[T], + start: String, + sep: String, + end: String, + maxFields: Option[Int]): String = { +val maxNumFields = maxFields.getOrElse(maxNumToStringFields) --- End diff -- You should document this behavior. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user hvanhovell commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r223982046 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala --- @@ -167,6 +172,58 @@ package object util { builder.toString() } + /** + * The performance overhead of creating and logging strings for wide schemas can be large. To + * limit the impact, we bound the number of fields to include by default. This can be overridden + * by setting the 'spark.debug.maxToStringFields' conf in SparkEnv or by settings the SQL config + * `spark.sql.debug.maxToStringFields`. + */ + private[spark] def maxNumToStringFields: Int = { +val legacyLimit = if (SparkEnv.get != null) { --- End diff -- Just for context why do you want to retain the legacy behavior? It is probably fine to break it. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user hvanhovell commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r223980665 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala --- @@ -455,21 +457,37 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { }.mkString(", ") /** ONE line description of this node. */ - def simpleString: String = s"$nodeName $argString".trim + def simpleString(maxFields: Option[Int]): String = { --- End diff -- Please document the `maxFields` parameter. I am especially interested in what `None` represents here. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user hvanhovell commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r223979931 --- Diff: external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala --- @@ -52,7 +52,7 @@ case class CatalystDataToAvro(child: Expression) extends UnaryExpression { out.toByteArray } - override def simpleString: String = { + override def simpleString(maxFields: Option[Int]): String = { s"to_avro(${child.sql}, ${child.dataType.simpleString})" --- End diff -- Should we also pass the maxFields to `child.dataType`? For example `StructType` fields are truncated. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user hvanhovell commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r223979392 --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala --- @@ -633,4 +633,14 @@ package object config { .stringConf .toSequence .createWithDefault(Nil) + + private[spark] val MAX_TO_STRING_FIELDS = +ConfigBuilder("spark.debug.maxToStringFields") + .internal() + .doc("Maximum number of fields of sequence-like entries that can be converted to strings " + --- End diff -- What is a sequence like entry? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user MaxGekk commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r221011834 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala --- @@ -167,6 +170,56 @@ package object util { builder.toString() } + /** + * The performance overhead of creating and logging strings for wide schemas can be large. To + * limit the impact, we bound the number of fields to include by default. This can be overridden + * by setting the 'spark.debug.maxToStringFields' conf in SparkEnv. + */ + val DEFAULT_MAX_TO_STRING_FIELDS = 25 + + private[spark] def maxNumToStringFields = { +if (SparkEnv.get != null) { + SparkEnv.get.conf.getInt("spark.debug.maxToStringFields", DEFAULT_MAX_TO_STRING_FIELDS) --- End diff -- I have added new SQL config and I am testing it for now. My concern is how to combine old config, new config and passed parameter. I am going to take maximum of them like: ```scala private[spark] def maxNumToStringFields = { val legacyLimit = if (SparkEnv.get != null) { SparkEnv.get.conf.getInt("spark.debug.maxToStringFields", DEFAULT_MAX_TO_STRING_FIELDS) } else { DEFAULT_MAX_TO_STRING_FIELDS } val sqlConfLimit = SQLConf.get.maxToStringFields Math.max(sqlConfLimit, legacyLimit) } ``` --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user HyukjinKwon commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r220412301 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala --- @@ -167,6 +170,56 @@ package object util { builder.toString() } + /** + * The performance overhead of creating and logging strings for wide schemas can be large. To + * limit the impact, we bound the number of fields to include by default. This can be overridden + * by setting the 'spark.debug.maxToStringFields' conf in SparkEnv. + */ + val DEFAULT_MAX_TO_STRING_FIELDS = 25 + + private[spark] def maxNumToStringFields = { +if (SparkEnv.get != null) { + SparkEnv.get.conf.getInt("spark.debug.maxToStringFields", DEFAULT_MAX_TO_STRING_FIELDS) --- End diff -- I think it makes sense to me. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r220409552 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala --- @@ -167,6 +170,56 @@ package object util { builder.toString() } + /** + * The performance overhead of creating and logging strings for wide schemas can be large. To + * limit the impact, we bound the number of fields to include by default. This can be overridden + * by setting the 'spark.debug.maxToStringFields' conf in SparkEnv. + */ + val DEFAULT_MAX_TO_STRING_FIELDS = 25 + + private[spark] def maxNumToStringFields = { +if (SparkEnv.get != null) { + SparkEnv.get.conf.getInt("spark.debug.maxToStringFields", DEFAULT_MAX_TO_STRING_FIELDS) --- End diff -- Shall we have both deprecated old config and new sql config, then remove old config in next major version? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user MaxGekk commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r220200837 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala --- @@ -167,6 +170,56 @@ package object util { builder.toString() } + /** + * The performance overhead of creating and logging strings for wide schemas can be large. To + * limit the impact, we bound the number of fields to include by default. This can be overridden + * by setting the 'spark.debug.maxToStringFields' conf in SparkEnv. + */ + val DEFAULT_MAX_TO_STRING_FIELDS = 25 + + private[spark] def maxNumToStringFields = { +if (SparkEnv.get != null) { + SparkEnv.get.conf.getInt("spark.debug.maxToStringFields", DEFAULT_MAX_TO_STRING_FIELDS) --- End diff -- Renaming it can potentially break user's apps. Can we do that in current minor version? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r220195279 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala --- @@ -167,6 +170,56 @@ package object util { builder.toString() } + /** + * The performance overhead of creating and logging strings for wide schemas can be large. To + * limit the impact, we bound the number of fields to include by default. This can be overridden + * by setting the 'spark.debug.maxToStringFields' conf in SparkEnv. + */ + val DEFAULT_MAX_TO_STRING_FIELDS = 25 + + private[spark] def maxNumToStringFields = { +if (SparkEnv.get != null) { + SparkEnv.get.conf.getInt("spark.debug.maxToStringFields", DEFAULT_MAX_TO_STRING_FIELDS) --- End diff -- Since it is moved to sql, shall we name it as `spark.sql.debug.maxToStringFields`? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user zsxwing commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r219977185 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala --- @@ -196,7 +196,7 @@ case class RDDScanExec( } } - override def simpleString: String = { + override def simpleString(maxFields: Option[Int]): String = { s"$nodeName${Utils.truncatedString(output, "[", ",", "]")}" --- End diff -- `maxFields` should be used here. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user zsxwing commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r219969937 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala --- @@ -250,5 +265,22 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { def codegenToSeq(): Seq[(String, String)] = { org.apache.spark.sql.execution.debug.codegenStringSeq(executedPlan) } + +/** + * Dumps debug information about query execution into the specified file. + */ +def toFile(path: String): Unit = { + val filePath = new Path(path) + val fs = FileSystem.get(filePath.toUri, sparkSession.sessionState.newHadoopConf()) + val writer = new OutputStreamWriter(fs.create(filePath)) --- End diff -- `val writer = new BufferedWriter(new OutputStreamWriter(fs.create(filePath), UTF_8))` --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r219729921 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala --- @@ -250,5 +265,22 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { def codegenToSeq(): Seq[(String, String)] = { org.apache.spark.sql.execution.debug.codegenStringSeq(executedPlan) } + +/** + * Dumps debug information about query execution into the specified file. + */ +def toFile(path: String): Unit = { + val filePath = new Path(path) + val fs = FileSystem.get(filePath.toUri, sparkSession.sessionState.newHadoopConf()) + val writer = new OutputStreamWriter(fs.create(filePath)) --- End diff -- cc @zsxwing Could you help review this function? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r219729889 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala --- @@ -250,5 +265,22 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { def codegenToSeq(): Seq[(String, String)] = { org.apache.spark.sql.execution.debug.codegenStringSeq(executedPlan) } + +/** + * Dumps debug information about query execution into the specified file. + */ +def toFile(path: String): Unit = { + val filePath = new Path(path) + val fs = FileSystem.get(filePath.toUri, sparkSession.sessionState.newHadoopConf()) --- End diff -- val fs = filePath.getFileSystem(spark.sessionState.newHadoopConf()) --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r219729210 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala --- @@ -668,11 +670,19 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) override def generateTreeString( depth: Int, lastChildren: Seq[Boolean], - builder: StringBuilder, + writer: Writer, verbose: Boolean, prefix: String = "", - addSuffix: Boolean = false): StringBuilder = { -child.generateTreeString(depth, lastChildren, builder, verbose, s"*($codegenStageId) ") + addSuffix: Boolean = false, + maxFields: Option[Int]): Unit = { +child.generateTreeString( + depth, + lastChildren, + writer, + verbose, + s"*($codegenStageId) ", + false, --- End diff -- named boolean : `addSuffix = false` --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user MaxGekk commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r219276920 --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala --- @@ -16,11 +16,33 @@ */ package org.apache.spark.sql.execution +import scala.io.Source + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} import org.apache.spark.sql.test.SharedSQLContext class QueryExecutionSuite extends SharedSQLContext { + test("dumping query execution info to a file") { +withTempDir { dir => + val path = dir.getCanonicalPath + "/plans.txt" --- End diff -- > Also add a few cases. @gatorsmile I added a few tests. Please, take a look at them. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r218307450 --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala --- @@ -16,11 +16,33 @@ */ package org.apache.spark.sql.execution +import scala.io.Source + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} import org.apache.spark.sql.test.SharedSQLContext class QueryExecutionSuite extends SharedSQLContext { + test("dumping query execution info to a file") { +withTempDir { dir => + val path = dir.getCanonicalPath + "/plans.txt" --- End diff -- Also add a few cases. For example, the path already exists? the path is not legal? and so on. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r218303899 --- Diff: core/src/main/scala/org/apache/spark/util/Utils.scala --- @@ -99,10 +99,11 @@ private[spark] object Utils extends Logging { * by setting the 'spark.debug.maxToStringFields' conf in SparkEnv. */ val DEFAULT_MAX_TO_STRING_FIELDS = 25 + val MAX_TO_STRING_FIELDS = "spark.debug.maxToStringFields" --- End diff -- Can we move it to org.apache.spark.internal.config? I think our Core module should do the same thing like what we are doing in SQLConf. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user rednaxelafx commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r218235962 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala --- @@ -189,23 +192,32 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { """.stripMargin.trim } + private def writeOrError(writer: Writer)(f: Writer => Unit): Unit = +try f(writer) catch { case e: AnalysisException => writer.write(e.toString) } + + private def writePlans(writer: Writer): Unit = { +writer.write("== Parsed Logical Plan ==\n") +writeOrError(writer)(logical.treeString(_, verbose = true, addSuffix = false)) +writer.write("== Analyzed Logical Plan ==\n") +writeOrError(writer) { writer => + analyzed.output.foreach(o => writer.write(s"${o.name}: ${o.dataType.simpleString}")) --- End diff -- If you want to get the best out of this approach, it might be better to avoid string interpolation here and do the explicit `writer.write` calls for the things you're interpolating on. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user MaxGekk commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r218211903 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala --- @@ -250,5 +254,36 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { def codegenToSeq(): Seq[(String, String)] = { org.apache.spark.sql.execution.debug.codegenStringSeq(executedPlan) } + +/** + * Dumps debug information about query execution into the specified file. + */ +def toFile(path: String): Unit = { + val maxFields = SparkEnv.get.conf.getInt(Utils.MAX_TO_STRING_FIELDS, +Utils.DEFAULT_MAX_TO_STRING_FIELDS) + val filePath = new Path(path) + val fs = FileSystem.get(filePath.toUri, sparkSession.sessionState.newHadoopConf()) + val writer = new BufferedWriter(new OutputStreamWriter(fs.create(filePath))) + + try { +SparkEnv.get.conf.set(Utils.MAX_TO_STRING_FIELDS, Int.MaxValue.toString) --- End diff -- > It is generally a bad idea to change this conf as people expect that it is immutable. Maybe it is generally a bad idea but the purpose of the `toFile` is to call it during debug of an issue. Currently when someone need to output all fields, he/she has to set `spark.debug.maxToStringFields` before `explain()` and revert it back after `explain()`. If you want, I can add a comment which tells to an user that the `toFile` changes the config during its invoke. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user MaxGekk commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r218209230 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala --- @@ -250,5 +254,36 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { def codegenToSeq(): Seq[(String, String)] = { org.apache.spark.sql.execution.debug.codegenStringSeq(executedPlan) } + +/** + * Dumps debug information about query execution into the specified file. + */ +def toFile(path: String): Unit = { + val maxFields = SparkEnv.get.conf.getInt(Utils.MAX_TO_STRING_FIELDS, +Utils.DEFAULT_MAX_TO_STRING_FIELDS) + val filePath = new Path(path) + val fs = FileSystem.get(filePath.toUri, sparkSession.sessionState.newHadoopConf()) + val writer = new BufferedWriter(new OutputStreamWriter(fs.create(filePath))) + + try { +SparkEnv.get.conf.set(Utils.MAX_TO_STRING_FIELDS, Int.MaxValue.toString) +writer.write("== Parsed Logical Plan ==\n") --- End diff -- Combined --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user MaxGekk commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r218197544 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala --- @@ -469,7 +470,17 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { def treeString: String = treeString(verbose = true) def treeString(verbose: Boolean, addSuffix: Boolean = false): String = { -generateTreeString(0, Nil, new StringBuilder, verbose = verbose, addSuffix = addSuffix).toString +val baos = new ByteArrayOutputStream() --- End diff -- This is done --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user MaxGekk commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r218187690 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala --- @@ -250,5 +254,36 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { def codegenToSeq(): Seq[(String, String)] = { org.apache.spark.sql.execution.debug.codegenStringSeq(executedPlan) } + +/** + * Dumps debug information about query execution into the specified file. + */ +def toFile(path: String): Unit = { + val maxFields = SparkEnv.get.conf.getInt(Utils.MAX_TO_STRING_FIELDS, +Utils.DEFAULT_MAX_TO_STRING_FIELDS) + val filePath = new Path(path) + val fs = FileSystem.get(filePath.toUri, sparkSession.sessionState.newHadoopConf()) + val writer = new BufferedWriter(new OutputStreamWriter(fs.create(filePath))) + + try { +SparkEnv.get.conf.set(Utils.MAX_TO_STRING_FIELDS, Int.MaxValue.toString) --- End diff -- Herman, I am trying to do that but I am already doubt it is right ways. I have to add new argument to `verboseString` and `simpleString` which is broadly used everywhere. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user MaxGekk commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r217928631 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala --- @@ -469,7 +470,17 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { def treeString: String = treeString(verbose = true) def treeString(verbose: Boolean, addSuffix: Boolean = false): String = { -generateTreeString(0, Nil, new StringBuilder, verbose = verbose, addSuffix = addSuffix).toString +val baos = new ByteArrayOutputStream() --- End diff -- In this particular method, there is no benefits. This was changed to reused the method which accepts `OutputStream` instead of `StringBuilder`. Benefit of `OutputStream` over `StringBuilder` is no full materialization in memory and no string size limit. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user hvanhovell commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r217928428 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala --- @@ -250,5 +254,36 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { def codegenToSeq(): Seq[(String, String)] = { org.apache.spark.sql.execution.debug.codegenStringSeq(executedPlan) } + +/** + * Dumps debug information about query execution into the specified file. + */ +def toFile(path: String): Unit = { + val maxFields = SparkEnv.get.conf.getInt(Utils.MAX_TO_STRING_FIELDS, +Utils.DEFAULT_MAX_TO_STRING_FIELDS) + val filePath = new Path(path) + val fs = FileSystem.get(filePath.toUri, sparkSession.sessionState.newHadoopConf()) + val writer = new BufferedWriter(new OutputStreamWriter(fs.create(filePath))) + + try { +SparkEnv.get.conf.set(Utils.MAX_TO_STRING_FIELDS, Int.MaxValue.toString) +writer.write("== Parsed Logical Plan ==\n") --- End diff -- Can we combine this entire block with what is done in the `toString()` method? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user hvanhovell commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r217928334 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala --- @@ -250,5 +254,36 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { def codegenToSeq(): Seq[(String, String)] = { org.apache.spark.sql.execution.debug.codegenStringSeq(executedPlan) } + +/** + * Dumps debug information about query execution into the specified file. + */ +def toFile(path: String): Unit = { + val maxFields = SparkEnv.get.conf.getInt(Utils.MAX_TO_STRING_FIELDS, +Utils.DEFAULT_MAX_TO_STRING_FIELDS) + val filePath = new Path(path) + val fs = FileSystem.get(filePath.toUri, sparkSession.sessionState.newHadoopConf()) + val writer = new BufferedWriter(new OutputStreamWriter(fs.create(filePath))) + + try { +SparkEnv.get.conf.set(Utils.MAX_TO_STRING_FIELDS, Int.MaxValue.toString) --- End diff -- It is generally a bad idea to change this conf as people expect that it is immutable. Also this change has some far reaching consequences, others will now also be exposed to a different `Utils.MAX_TO_STRING_FIELDS` value when calling `explain()`. Can you please just pass the parameter down the tree? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user hvanhovell commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r217928262 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala --- @@ -469,7 +470,17 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { def treeString: String = treeString(verbose = true) def treeString(verbose: Boolean, addSuffix: Boolean = false): String = { -generateTreeString(0, Nil, new StringBuilder, verbose = verbose, addSuffix = addSuffix).toString +val baos = new ByteArrayOutputStream() --- End diff -- What is the benefit of using this instead of using a `java.io.StringWriter` or `org.apache.commons.io.output.StringBuilderWriter`? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r217915212 --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala --- @@ -16,11 +16,33 @@ */ package org.apache.spark.sql.execution +import scala.io.Source + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} import org.apache.spark.sql.test.SharedSQLContext class QueryExecutionSuite extends SharedSQLContext { + test("dumping query execution info to a file") { +withTempDir { dir => + val path = dir.getCanonicalPath + s"/plans.txt" --- End diff -- We don't need string interpolation here. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user hvanhovell commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r217915071 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala --- @@ -250,5 +253,35 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { def codegenToSeq(): Seq[(String, String)] = { org.apache.spark.sql.execution.debug.codegenStringSeq(executedPlan) } + +/** + * Dumps debug information about query execution into the specified file. + */ +def toFile(path: String): Unit = { + val filePath = new Path(path) + val fs = FileSystem.get(filePath.toUri, sparkSession.sparkContext.hadoopConfiguration) --- End diff -- Why use the hadoop configuration of the `SparkContext`? It is probably better to use the one that `sparkSession.sessionState.newHadoopConf()` provides. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #22429: [SPARK-25440][SQL] Dumping query execution info t...
Github user hvanhovell commented on a diff in the pull request: https://github.com/apache/spark/pull/22429#discussion_r217913739 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala --- @@ -469,7 +470,13 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { def treeString: String = treeString(verbose = true) def treeString(verbose: Boolean, addSuffix: Boolean = false): String = { -generateTreeString(0, Nil, new StringBuilder, verbose = verbose, addSuffix = addSuffix).toString +val bos = new ByteArrayOutputStream() +treeString(bos, verbose, addSuffix) +bos.toString + } + + def treeString(os: OutputStream, verbose: Boolean, addSuffix: Boolean): Unit = { --- End diff -- Can you please use a `java.io.Writer` or something else you can directly write a string to? You are now using `getBytes()` everywhere and that is far from cheap because it needs to encode the chars and allocate a byte array for each string. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org