From 19b9a684b6d0985cf563257d6321fd6f14458d36 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 15 Sep 2018 14:22:42 +0200 Subject: [PATCH 01/41] Stub implementation and a test --- .../spark/sql/execution/QueryExecution.scala | 16 ++++++++++++++++ .../sql/execution/QueryExecutionSuite.scala | 16 ++++++++++++++++ 2 files changed, 32 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 64f49e2d0d4e6..79ac60ce515b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.execution import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} +import org.apache.hadoop.fs.{FileSystem, Path} + import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow @@ -250,5 +252,19 @@ 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) + val fos = fs.create(filePath) + try { + fos.writeBytes("Hello, World!") + } finally { + fos.close() + } + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index 964440346deb0..f4ece6e154adc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -16,6 +16,11 @@ */ package org.apache.spark.sql.execution +import java.nio.file.{Files, Paths} + +import scala.io.Source +import scala.reflect.io.Path + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} import org.apache.spark.sql.test.SharedSQLContext @@ -49,4 +54,15 @@ class QueryExecutionSuite extends SharedSQLContext { val error = intercept[Error](qe.toString) assert(error.getMessage.contains("error")) } + + test("debug to file") { + withTempDir { dir => + val path = dir.getCanonicalPath + s"/plans.txt" + val df = spark.range(0, 10) + df.queryExecution.debug.toFile(path) + + assert(Source.fromFile(path).getLines.toList == List( + "Hello, World!")) + } + } } From 90832f9571e8cafde622069dec4f837141b07c30 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 15 Sep 2018 14:57:24 +0200 Subject: [PATCH 02/41] Saving all plans to file --- .../spark/sql/catalyst/trees/TreeNode.scala | 46 +++++++++++++++++++ .../spark/sql/execution/QueryExecution.scala | 14 ++++-- .../sql/execution/QueryExecutionSuite.scala | 9 +++- 3 files changed, 65 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index becfa8d982213..b58e382836e4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.trees +import java.io.DataOutputStream import java.util.UUID import scala.collection.Map @@ -472,6 +473,10 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { generateTreeString(0, Nil, new StringBuilder, verbose = verbose, addSuffix = addSuffix).toString } + def treeString(os: DataOutputStream): Unit = { + treeToOutputStream(0, Nil, os, verbose = true, addSuffix = false) + } + /** * Returns a string representation of the nodes in this tree, where each operator is numbered. * The numbers can be used with [[TreeNode.apply]] to easily access specific subtrees. @@ -572,6 +577,47 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { builder } + def treeToOutputStream( + depth: Int, + lastChildren: Seq[Boolean], + dos: DataOutputStream, + verbose: Boolean, + prefix: String = "", + addSuffix: Boolean = false): Unit = { + + if (depth > 0) { + lastChildren.init.foreach { isLast => + dos.writeBytes(if (isLast) " " else ": ") + } + dos.writeBytes(if (lastChildren.last) "+- " else ":- ") + } + + val str = if (verbose) { + if (addSuffix) verboseStringWithSuffix else verboseString + } else { + simpleString + } + dos.writeBytes(prefix) + dos.writeBytes(str) + dos.writeBytes("\n") + + if (innerChildren.nonEmpty) { + innerChildren.init.foreach(_.treeToOutputStream( + depth + 2, lastChildren :+ children.isEmpty :+ false, dos, verbose, + addSuffix = addSuffix)) + innerChildren.last.treeToOutputStream( + depth + 2, lastChildren :+ children.isEmpty :+ true, dos, verbose, + addSuffix = addSuffix) + } + + if (children.nonEmpty) { + children.init.foreach(_.treeToOutputStream( + depth + 1, lastChildren :+ false, dos, verbose, prefix, addSuffix)) + children.last.treeToOutputStream( + depth + 1, lastChildren :+ true, dos, verbose, prefix, addSuffix) + } + } + /** * Returns a 'scala code' representation of this `TreeNode` and its children. Intended for use * when debugging where the prettier toString function is obfuscating the actual structure. In the diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 79ac60ce515b9..5528b3c6191c0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -259,11 +259,19 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { def toFile(path: String): Unit = { val filePath = new Path(path) val fs = FileSystem.get(filePath.toUri, sparkSession.sparkContext.hadoopConfiguration) - val fos = fs.create(filePath) + val dataStream = fs.create(filePath) try { - fos.writeBytes("Hello, World!") + dataStream.writeBytes("== Parsed Logical Plan ==\n") + logical.treeString(dataStream) + dataStream.writeBytes("== Analyzed Logical Plan ==\n") + analyzed.treeString(dataStream) + dataStream.writeBytes("== Optimized Logical Plan ==\n") + optimizedPlan.treeString(dataStream) + dataStream.writeBytes("== Physical Plan ==\n") + executedPlan.treeString(dataStream) + dataStream.flush() } finally { - fos.close() + dataStream.close() } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index f4ece6e154adc..0ec78180753c9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -62,7 +62,14 @@ class QueryExecutionSuite extends SharedSQLContext { df.queryExecution.debug.toFile(path) assert(Source.fromFile(path).getLines.toList == List( - "Hello, World!")) + "== Parsed Logical Plan ==", + "Range (0, 10, step=1, splits=Some(2))", + "== Analyzed Logical Plan ==", + "Range (0, 10, step=1, splits=Some(2))", + "== Optimized Logical Plan ==", + "Range (0, 10, step=1, splits=Some(2))", + "== Physical Plan ==", "WholeStageCodegen", + "+- Range (0, 10, step=1, splits=2)")) } } } From 673ae565e42acc00df9acfba670c0491172ffb19 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 15 Sep 2018 15:02:49 +0200 Subject: [PATCH 03/41] Output attributes --- .../spark/sql/execution/QueryExecution.scala | 24 ++++++++++--------- .../sql/execution/QueryExecutionSuite.scala | 1 + 2 files changed, 14 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 5528b3c6191c0..99c235bd75eec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -259,19 +259,21 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { def toFile(path: String): Unit = { val filePath = new Path(path) val fs = FileSystem.get(filePath.toUri, sparkSession.sparkContext.hadoopConfiguration) - val dataStream = fs.create(filePath) + val dos = fs.create(filePath) try { - dataStream.writeBytes("== Parsed Logical Plan ==\n") - logical.treeString(dataStream) - dataStream.writeBytes("== Analyzed Logical Plan ==\n") - analyzed.treeString(dataStream) - dataStream.writeBytes("== Optimized Logical Plan ==\n") - optimizedPlan.treeString(dataStream) - dataStream.writeBytes("== Physical Plan ==\n") - executedPlan.treeString(dataStream) - dataStream.flush() + dos.writeBytes("== Parsed Logical Plan ==\n") + logical.treeString(dos) + dos.writeBytes("== Analyzed Logical Plan ==\n") + analyzed.output.foreach(o => dos.writeBytes(s"${o.name}: ${o.dataType.simpleString}")) + dos.writeBytes("\n") + analyzed.treeString(dos) + dos.writeBytes("== Optimized Logical Plan ==\n") + optimizedPlan.treeString(dos) + dos.writeBytes("== Physical Plan ==\n") + executedPlan.treeString(dos) + dos.flush() } finally { - dataStream.close() + dos.close() } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index 0ec78180753c9..b511b7b792058 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -65,6 +65,7 @@ class QueryExecutionSuite extends SharedSQLContext { "== Parsed Logical Plan ==", "Range (0, 10, step=1, splits=Some(2))", "== Analyzed Logical Plan ==", + "id: bigint", "Range (0, 10, step=1, splits=Some(2))", "== Optimized Logical Plan ==", "Range (0, 10, step=1, splits=Some(2))", From fbde8120122c83eecbad2d060f959e467ecb4ff0 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 15 Sep 2018 15:14:50 +0200 Subject: [PATCH 04/41] Output whole stage codegen --- .../apache/spark/sql/execution/QueryExecution.scala | 2 ++ .../apache/spark/sql/execution/debug/package.scala | 12 ++++++++++++ .../spark/sql/execution/QueryExecutionSuite.scala | 3 ++- 3 files changed, 16 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 99c235bd75eec..8471423249cd3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -272,6 +272,8 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { dos.writeBytes("== Physical Plan ==\n") executedPlan.treeString(dos) dos.flush() + dos.writeBytes("== Whole Stage Codegen ==\n") + org.apache.spark.sql.execution.debug.codegen(dos, executedPlan) } finally { dos.close() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index 366e1fe6a4aaa..590da09b46009 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution +import java.io.DataOutputStream import java.util.Collections import scala.collection.JavaConverters._ @@ -81,6 +82,17 @@ package object debug { output } + def codegen(dos: DataOutputStream, plan: SparkPlan): Unit = { + val codegenSeq = codegenStringSeq(plan) + dos.writeBytes(s"Found ${codegenSeq.size} WholeStageCodegen subtrees.\n") + for (((subtree, code), i) <- codegenSeq.zipWithIndex) { + dos.writeBytes(s"== Subtree ${i + 1} / ${codegenSeq.size} ==\n") + dos.writeBytes(subtree) + dos.writeBytes("\nGenerated code:\n") + dos.writeBytes(s"${code}\n") + } + } + /** * Get WholeStageCodegenExec subtrees and the codegen in a query plan * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index b511b7b792058..0e6510625d5ab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -61,7 +61,8 @@ class QueryExecutionSuite extends SharedSQLContext { val df = spark.range(0, 10) df.queryExecution.debug.toFile(path) - assert(Source.fromFile(path).getLines.toList == List( + assert(Source.fromFile(path).getLines.toList + .takeWhile(_ != "== Whole Stage Codegen ==") == List( "== Parsed Logical Plan ==", "Range (0, 10, step=1, splits=Some(2))", "== Analyzed Logical Plan ==", From dca19d33ed516bea8e3d113c5e81e3e1e2f2d77d Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 15 Sep 2018 15:25:43 +0200 Subject: [PATCH 05/41] Reusing codegenToOutputStream --- .../spark/sql/execution/QueryExecution.scala | 2 +- .../spark/sql/execution/debug/package.scala | 27 +++++++------------ 2 files changed, 11 insertions(+), 18 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 8471423249cd3..07512ec41101f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -273,7 +273,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { executedPlan.treeString(dos) dos.flush() dos.writeBytes("== Whole Stage Codegen ==\n") - org.apache.spark.sql.execution.debug.codegen(dos, executedPlan) + org.apache.spark.sql.execution.debug.codegenToOutputStream(dos, executedPlan) } finally { dos.close() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index 590da09b46009..c3608b9fc8dd3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -import java.io.DataOutputStream +import java.io.{ByteArrayOutputStream, OutputStream} import java.util.Collections import scala.collection.JavaConverters._ @@ -31,7 +31,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, Codegen import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.execution.streaming.{StreamExecution, StreamingQueryWrapper} -import org.apache.spark.sql.execution.streaming.continuous.WriteToContinuousDataSourceExec import org.apache.spark.sql.streaming.StreamingQuery import org.apache.spark.util.{AccumulatorV2, LongAccumulator} @@ -71,25 +70,19 @@ package object debug { * @return single String containing all WholeStageCodegen subtrees and corresponding codegen */ def codegenString(plan: SparkPlan): String = { - val codegenSeq = codegenStringSeq(plan) - var output = s"Found ${codegenSeq.size} WholeStageCodegen subtrees.\n" - for (((subtree, code), i) <- codegenSeq.zipWithIndex) { - output += s"== Subtree ${i + 1} / ${codegenSeq.size} ==\n" - output += subtree - output += "\nGenerated code:\n" - output += s"${code}\n" - } - output + val bos = new ByteArrayOutputStream() + codegenToOutputStream(bos, plan) + bos.toString } - def codegen(dos: DataOutputStream, plan: SparkPlan): Unit = { + def codegenToOutputStream(os: OutputStream, plan: SparkPlan): Unit = { val codegenSeq = codegenStringSeq(plan) - dos.writeBytes(s"Found ${codegenSeq.size} WholeStageCodegen subtrees.\n") + os.write(s"Found ${codegenSeq.size} WholeStageCodegen subtrees.\n".getBytes) for (((subtree, code), i) <- codegenSeq.zipWithIndex) { - dos.writeBytes(s"== Subtree ${i + 1} / ${codegenSeq.size} ==\n") - dos.writeBytes(subtree) - dos.writeBytes("\nGenerated code:\n") - dos.writeBytes(s"${code}\n") + os.write(s"== Subtree ${i + 1} / ${codegenSeq.size} ==\n".getBytes()) + os.write(subtree.getBytes) + os.write("\nGenerated code:\n".getBytes) + os.write(s"${code}\n".getBytes) } } From 66351a09f60f0c9b21abe12bdb559a36761e8a8c Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 15 Sep 2018 16:16:22 +0200 Subject: [PATCH 06/41] Code de-duplication --- .../spark/sql/catalyst/trees/TreeNode.scala | 77 +++++-------------- .../spark/sql/execution/QueryExecution.scala | 8 +- .../sql/execution/WholeStageCodegenExec.scala | 13 ++-- .../sql/execution/QueryExecutionSuite.scala | 4 +- 4 files changed, 31 insertions(+), 71 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index b58e382836e4c..4a183175dff4e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.trees -import java.io.DataOutputStream +import java.io.{ByteArrayOutputStream, OutputStream} import java.util.UUID import scala.collection.Map @@ -470,11 +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: DataOutputStream): Unit = { - treeToOutputStream(0, Nil, os, verbose = true, addSuffix = false) + def treeString(os: OutputStream, verbose: Boolean, addSuffix: Boolean): Unit = { + generateTreeString(0, Nil, os, verbose, "", addSuffix) } /** @@ -526,7 +528,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { protected def innerChildren: Seq[TreeNode[_]] = Seq.empty /** - * Appends the string representation of this node and its children to the given StringBuilder. + * Appends the string representation of this node and its children to the given OutputStream. * * The `i`-th element in `lastChildren` indicates whether the ancestor of the current node at * depth `i + 1` is the last child of its own parent node. The depth of the root node is 0, and @@ -537,16 +539,16 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { def generateTreeString( depth: Int, lastChildren: Seq[Boolean], - builder: StringBuilder, + os: OutputStream, verbose: Boolean, prefix: String = "", - addSuffix: Boolean = false): StringBuilder = { + addSuffix: Boolean = false): Unit = { if (depth > 0) { lastChildren.init.foreach { isLast => - builder.append(if (isLast) " " else ": ") + os.write((if (isLast) " " else ": ").getBytes) } - builder.append(if (lastChildren.last) "+- " else ":- ") + os.write((if (lastChildren.last) "+- " else ":- ").getBytes) } val str = if (verbose) { @@ -554,67 +556,24 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } else { simpleString } - builder.append(prefix) - builder.append(str) - builder.append("\n") + os.write(prefix.getBytes) + os.write(str.getBytes) + os.write("\n".getBytes) if (innerChildren.nonEmpty) { innerChildren.init.foreach(_.generateTreeString( - depth + 2, lastChildren :+ children.isEmpty :+ false, builder, verbose, + depth + 2, lastChildren :+ children.isEmpty :+ false, os, verbose, addSuffix = addSuffix)) innerChildren.last.generateTreeString( - depth + 2, lastChildren :+ children.isEmpty :+ true, builder, verbose, + depth + 2, lastChildren :+ children.isEmpty :+ true, os, verbose, addSuffix = addSuffix) } if (children.nonEmpty) { children.init.foreach(_.generateTreeString( - depth + 1, lastChildren :+ false, builder, verbose, prefix, addSuffix)) + depth + 1, lastChildren :+ false, os, verbose, prefix, addSuffix)) children.last.generateTreeString( - depth + 1, lastChildren :+ true, builder, verbose, prefix, addSuffix) - } - - builder - } - - def treeToOutputStream( - depth: Int, - lastChildren: Seq[Boolean], - dos: DataOutputStream, - verbose: Boolean, - prefix: String = "", - addSuffix: Boolean = false): Unit = { - - if (depth > 0) { - lastChildren.init.foreach { isLast => - dos.writeBytes(if (isLast) " " else ": ") - } - dos.writeBytes(if (lastChildren.last) "+- " else ":- ") - } - - val str = if (verbose) { - if (addSuffix) verboseStringWithSuffix else verboseString - } else { - simpleString - } - dos.writeBytes(prefix) - dos.writeBytes(str) - dos.writeBytes("\n") - - if (innerChildren.nonEmpty) { - innerChildren.init.foreach(_.treeToOutputStream( - depth + 2, lastChildren :+ children.isEmpty :+ false, dos, verbose, - addSuffix = addSuffix)) - innerChildren.last.treeToOutputStream( - depth + 2, lastChildren :+ children.isEmpty :+ true, dos, verbose, - addSuffix = addSuffix) - } - - if (children.nonEmpty) { - children.init.foreach(_.treeToOutputStream( - depth + 1, lastChildren :+ false, dos, verbose, prefix, addSuffix)) - children.last.treeToOutputStream( - depth + 1, lastChildren :+ true, dos, verbose, prefix, addSuffix) + depth + 1, lastChildren :+ true, os, verbose, prefix, addSuffix) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 07512ec41101f..8847e607410f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -262,15 +262,15 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { val dos = fs.create(filePath) try { dos.writeBytes("== Parsed Logical Plan ==\n") - logical.treeString(dos) + logical.treeString(dos, verbose = true, addSuffix = false) dos.writeBytes("== Analyzed Logical Plan ==\n") analyzed.output.foreach(o => dos.writeBytes(s"${o.name}: ${o.dataType.simpleString}")) dos.writeBytes("\n") - analyzed.treeString(dos) + analyzed.treeString(dos, verbose = true, addSuffix = false) dos.writeBytes("== Optimized Logical Plan ==\n") - optimizedPlan.treeString(dos) + optimizedPlan.treeString(dos, verbose = true, addSuffix = false) dos.writeBytes("== Physical Plan ==\n") - executedPlan.treeString(dos) + executedPlan.treeString(dos, verbose = true, addSuffix = false) dos.flush() dos.writeBytes("== Whole Stage Codegen ==\n") org.apache.spark.sql.execution.debug.codegenToOutputStream(dos, executedPlan) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 1fc4de9e56015..084650d677da7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution +import java.io.OutputStream import java.util.Locale import java.util.function.Supplier @@ -392,11 +393,11 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp override def generateTreeString( depth: Int, lastChildren: Seq[Boolean], - builder: StringBuilder, + os: OutputStream, verbose: Boolean, prefix: String = "", - addSuffix: Boolean = false): StringBuilder = { - child.generateTreeString(depth, lastChildren, builder, verbose, "") + addSuffix: Boolean = false): Unit = { + child.generateTreeString(depth, lastChildren, os, verbose, "") } override def needCopyResult: Boolean = false @@ -668,11 +669,11 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) override def generateTreeString( depth: Int, lastChildren: Seq[Boolean], - builder: StringBuilder, + os: OutputStream, verbose: Boolean, prefix: String = "", - addSuffix: Boolean = false): StringBuilder = { - child.generateTreeString(depth, lastChildren, builder, verbose, s"*($codegenStageId) ") + addSuffix: Boolean = false): Unit = { + child.generateTreeString(depth, lastChildren, os, verbose, s"*($codegenStageId) ") } override def needStopCheck: Boolean = true diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index 0e6510625d5ab..2c8d714b3b42a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -70,8 +70,8 @@ class QueryExecutionSuite extends SharedSQLContext { "Range (0, 10, step=1, splits=Some(2))", "== Optimized Logical Plan ==", "Range (0, 10, step=1, splits=Some(2))", - "== Physical Plan ==", "WholeStageCodegen", - "+- Range (0, 10, step=1, splits=2)")) + "== Physical Plan ==", + "*(1) Range (0, 10, step=1, splits=2)")) } } } From 2ee75bcd4d495eb6031581ad7a38e757c254175b Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 15 Sep 2018 16:41:26 +0200 Subject: [PATCH 07/41] Do not truncate fields --- core/src/main/scala/org/apache/spark/util/Utils.scala | 3 ++- .../org/apache/spark/sql/execution/QueryExecution.scala | 5 +++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 14f68cd6f3509..151d79f961fb3 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/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" private[spark] def maxNumToStringFields = { if (SparkEnv.get != null) { - SparkEnv.get.conf.getInt("spark.debug.maxToStringFields", DEFAULT_MAX_TO_STRING_FIELDS) + SparkEnv.get.conf.getInt(MAX_TO_STRING_FIELDS, DEFAULT_MAX_TO_STRING_FIELDS) } else { DEFAULT_MAX_TO_STRING_FIELDS } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 8847e607410f4..89bab4a015a1e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -22,6 +22,7 @@ import java.sql.{Date, Timestamp} import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.spark.SparkEnv import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow @@ -260,7 +261,10 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { val filePath = new Path(path) val fs = FileSystem.get(filePath.toUri, sparkSession.sparkContext.hadoopConfiguration) val dos = fs.create(filePath) + val maxFields = SparkEnv.get.conf.getInt(Utils.MAX_TO_STRING_FIELDS, + Utils.DEFAULT_MAX_TO_STRING_FIELDS) try { + SparkEnv.get.conf.set(Utils.MAX_TO_STRING_FIELDS, Int.MaxValue.toString) dos.writeBytes("== Parsed Logical Plan ==\n") logical.treeString(dos, verbose = true, addSuffix = false) dos.writeBytes("== Analyzed Logical Plan ==\n") @@ -276,6 +280,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { org.apache.spark.sql.execution.debug.codegenToOutputStream(dos, executedPlan) } finally { dos.close() + SparkEnv.get.conf.set(Utils.MAX_TO_STRING_FIELDS, maxFields.toString) } } } From 9b2a3e664564182e325ef08785c998c6ff9b5367 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 15 Sep 2018 21:42:48 +0200 Subject: [PATCH 08/41] Moving the test up because previous one leaved a garbage --- .../sql/execution/QueryExecutionSuite.scala | 43 +++++++++---------- 1 file changed, 20 insertions(+), 23 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index 2c8d714b3b42a..227397f05c1bc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -16,16 +16,33 @@ */ package org.apache.spark.sql.execution -import java.nio.file.{Files, Paths} - import scala.io.Source -import scala.reflect.io.Path 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" + val df = spark.range(0, 10) + df.queryExecution.debug.toFile(path) + + assert(Source.fromFile(path).getLines.toList + .takeWhile(_ != "== Whole Stage Codegen ==") == List( + "== Parsed Logical Plan ==", + "Range (0, 10, step=1, splits=Some(2))", + "== Analyzed Logical Plan ==", + "id: bigint", + "Range (0, 10, step=1, splits=Some(2))", + "== Optimized Logical Plan ==", + "Range (0, 10, step=1, splits=Some(2))", + "== Physical Plan ==", + "*(1) Range (0, 10, step=1, splits=2)")) + } + } + test("toString() exception/error handling") { spark.experimental.extraStrategies = Seq( new SparkStrategy { @@ -54,24 +71,4 @@ class QueryExecutionSuite extends SharedSQLContext { val error = intercept[Error](qe.toString) assert(error.getMessage.contains("error")) } - - test("debug to file") { - withTempDir { dir => - val path = dir.getCanonicalPath + s"/plans.txt" - val df = spark.range(0, 10) - df.queryExecution.debug.toFile(path) - - assert(Source.fromFile(path).getLines.toList - .takeWhile(_ != "== Whole Stage Codegen ==") == List( - "== Parsed Logical Plan ==", - "Range (0, 10, step=1, splits=Some(2))", - "== Analyzed Logical Plan ==", - "id: bigint", - "Range (0, 10, step=1, splits=Some(2))", - "== Optimized Logical Plan ==", - "Range (0, 10, step=1, splits=Some(2))", - "== Physical Plan ==", - "*(1) Range (0, 10, step=1, splits=2)")) - } - } } From 51c196e9dd3cf79df4c76a55b96e3e3680d975b2 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 16 Sep 2018 16:43:35 +0200 Subject: [PATCH 09/41] Removing string interpolation in the test --- .../org/apache/spark/sql/execution/QueryExecutionSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index 227397f05c1bc..009ce2b4ddf39 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -25,7 +25,7 @@ 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" + val path = dir.getCanonicalPath + "/plans.txt" val df = spark.range(0, 10) df.queryExecution.debug.toFile(path) From c66a616791bdbe13725bf7e7ea36db817df50ea0 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 16 Sep 2018 16:45:03 +0200 Subject: [PATCH 10/41] Getting Hadoop's conf from session state --- .../scala/org/apache/spark/sql/execution/QueryExecution.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 89bab4a015a1e..b3fa6e195e639 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -259,7 +259,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { */ def toFile(path: String): Unit = { val filePath = new Path(path) - val fs = FileSystem.get(filePath.toUri, sparkSession.sparkContext.hadoopConfiguration) + val fs = FileSystem.get(filePath.toUri, sparkSession.sessionState.newHadoopConf()) val dos = fs.create(filePath) val maxFields = SparkEnv.get.conf.getInt(Utils.MAX_TO_STRING_FIELDS, Utils.DEFAULT_MAX_TO_STRING_FIELDS) From ed57c8ef6d6261d1960bfeff15079d5652d393dc Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 16 Sep 2018 17:19:41 +0200 Subject: [PATCH 11/41] Using java.io.Writer --- .../spark/sql/catalyst/trees/TreeNode.scala | 40 ++++++++++--------- .../spark/sql/execution/QueryExecution.scala | 31 +++++++------- .../sql/execution/WholeStageCodegenExec.scala | 10 ++--- .../spark/sql/execution/debug/package.scala | 24 ++++++----- 4 files changed, 57 insertions(+), 48 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 4a183175dff4e..92faed352c77f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.trees -import java.io.{ByteArrayOutputStream, OutputStream} +import java.io._ import java.util.UUID import scala.collection.Map @@ -31,7 +31,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.ScalaReflection._ import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, FunctionResource} +import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.JoinType @@ -470,13 +470,17 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { def treeString: String = treeString(verbose = true) def treeString(verbose: Boolean, addSuffix: Boolean = false): String = { - val bos = new ByteArrayOutputStream() - treeString(bos, verbose, addSuffix) - bos.toString + val baos = new ByteArrayOutputStream() + val writer = new BufferedWriter(new OutputStreamWriter(baos)) + + treeString(writer, verbose, addSuffix) + writer.flush() + + baos.toString } - def treeString(os: OutputStream, verbose: Boolean, addSuffix: Boolean): Unit = { - generateTreeString(0, Nil, os, verbose, "", addSuffix) + def treeString(writer: Writer, verbose: Boolean, addSuffix: Boolean): Unit = { + generateTreeString(0, Nil, writer, verbose, "", addSuffix) } /** @@ -528,7 +532,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { protected def innerChildren: Seq[TreeNode[_]] = Seq.empty /** - * Appends the string representation of this node and its children to the given OutputStream. + * Appends the string representation of this node and its children to the given Writer. * * The `i`-th element in `lastChildren` indicates whether the ancestor of the current node at * depth `i + 1` is the last child of its own parent node. The depth of the root node is 0, and @@ -539,16 +543,16 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { def generateTreeString( depth: Int, lastChildren: Seq[Boolean], - os: OutputStream, + writer: Writer, verbose: Boolean, prefix: String = "", addSuffix: Boolean = false): Unit = { if (depth > 0) { lastChildren.init.foreach { isLast => - os.write((if (isLast) " " else ": ").getBytes) + writer.write(if (isLast) " " else ": ") } - os.write((if (lastChildren.last) "+- " else ":- ").getBytes) + writer.write(if (lastChildren.last) "+- " else ":- ") } val str = if (verbose) { @@ -556,24 +560,24 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } else { simpleString } - os.write(prefix.getBytes) - os.write(str.getBytes) - os.write("\n".getBytes) + writer.write(prefix) + writer.write(str) + writer.write("\n") if (innerChildren.nonEmpty) { innerChildren.init.foreach(_.generateTreeString( - depth + 2, lastChildren :+ children.isEmpty :+ false, os, verbose, + depth + 2, lastChildren :+ children.isEmpty :+ false, writer, verbose, addSuffix = addSuffix)) innerChildren.last.generateTreeString( - depth + 2, lastChildren :+ children.isEmpty :+ true, os, verbose, + depth + 2, lastChildren :+ children.isEmpty :+ true, writer, verbose, addSuffix = addSuffix) } if (children.nonEmpty) { children.init.foreach(_.generateTreeString( - depth + 1, lastChildren :+ false, os, verbose, prefix, addSuffix)) + depth + 1, lastChildren :+ false, writer, verbose, prefix, addSuffix)) children.last.generateTreeString( - depth + 1, lastChildren :+ true, os, verbose, prefix, addSuffix) + depth + 1, lastChildren :+ true, writer, verbose, prefix, addSuffix) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index b3fa6e195e639..863f8ceee5095 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution +import java.io.{BufferedWriter, OutputStreamWriter} import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} @@ -260,26 +261,26 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { def toFile(path: String): Unit = { val filePath = new Path(path) val fs = FileSystem.get(filePath.toUri, sparkSession.sessionState.newHadoopConf()) - val dos = fs.create(filePath) + val writer = new BufferedWriter(new OutputStreamWriter(fs.create(filePath))) val maxFields = SparkEnv.get.conf.getInt(Utils.MAX_TO_STRING_FIELDS, Utils.DEFAULT_MAX_TO_STRING_FIELDS) try { SparkEnv.get.conf.set(Utils.MAX_TO_STRING_FIELDS, Int.MaxValue.toString) - dos.writeBytes("== Parsed Logical Plan ==\n") - logical.treeString(dos, verbose = true, addSuffix = false) - dos.writeBytes("== Analyzed Logical Plan ==\n") - analyzed.output.foreach(o => dos.writeBytes(s"${o.name}: ${o.dataType.simpleString}")) - dos.writeBytes("\n") - analyzed.treeString(dos, verbose = true, addSuffix = false) - dos.writeBytes("== Optimized Logical Plan ==\n") - optimizedPlan.treeString(dos, verbose = true, addSuffix = false) - dos.writeBytes("== Physical Plan ==\n") - executedPlan.treeString(dos, verbose = true, addSuffix = false) - dos.flush() - dos.writeBytes("== Whole Stage Codegen ==\n") - org.apache.spark.sql.execution.debug.codegenToOutputStream(dos, executedPlan) + writer.write("== Parsed Logical Plan ==\n") + logical.treeString(writer, verbose = true, addSuffix = false) + writer.write("== Analyzed Logical Plan ==\n") + analyzed.output.foreach(o => writer.write(s"${o.name}: ${o.dataType.simpleString}")) + writer.write("\n") + analyzed.treeString(writer, verbose = true, addSuffix = false) + writer.write("== Optimized Logical Plan ==\n") + optimizedPlan.treeString(writer, verbose = true, addSuffix = false) + writer.write("== Physical Plan ==\n") + executedPlan.treeString(writer, verbose = true, addSuffix = false) + writer.flush() + writer.write("== Whole Stage Codegen ==\n") + org.apache.spark.sql.execution.debug.writerCodegen(writer, executedPlan) } finally { - dos.close() + writer.close() SparkEnv.get.conf.set(Utils.MAX_TO_STRING_FIELDS, maxFields.toString) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 084650d677da7..9cb1c9e5ffeb2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -import java.io.OutputStream +import java.io.Writer import java.util.Locale import java.util.function.Supplier @@ -393,11 +393,11 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp override def generateTreeString( depth: Int, lastChildren: Seq[Boolean], - os: OutputStream, + writer: Writer, verbose: Boolean, prefix: String = "", addSuffix: Boolean = false): Unit = { - child.generateTreeString(depth, lastChildren, os, verbose, "") + child.generateTreeString(depth, lastChildren, writer, verbose, "") } override def needCopyResult: Boolean = false @@ -669,11 +669,11 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) override def generateTreeString( depth: Int, lastChildren: Seq[Boolean], - os: OutputStream, + writer: Writer, verbose: Boolean, prefix: String = "", addSuffix: Boolean = false): Unit = { - child.generateTreeString(depth, lastChildren, os, verbose, s"*($codegenStageId) ") + child.generateTreeString(depth, lastChildren, writer, verbose, s"*($codegenStageId) ") } override def needStopCheck: Boolean = true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index c3608b9fc8dd3..b266d9beccded 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -import java.io.{ByteArrayOutputStream, OutputStream} +import java.io._ import java.util.Collections import scala.collection.JavaConverters._ @@ -70,19 +70,23 @@ package object debug { * @return single String containing all WholeStageCodegen subtrees and corresponding codegen */ def codegenString(plan: SparkPlan): String = { - val bos = new ByteArrayOutputStream() - codegenToOutputStream(bos, plan) - bos.toString + val baos = new ByteArrayOutputStream() + val writer = new BufferedWriter(new OutputStreamWriter(baos)) + + writerCodegen(writer, plan) + writer.flush() + + baos.toString } - def codegenToOutputStream(os: OutputStream, plan: SparkPlan): Unit = { + def writerCodegen(writer: Writer, plan: SparkPlan): Unit = { val codegenSeq = codegenStringSeq(plan) - os.write(s"Found ${codegenSeq.size} WholeStageCodegen subtrees.\n".getBytes) + writer.write(s"Found ${codegenSeq.size} WholeStageCodegen subtrees.\n") for (((subtree, code), i) <- codegenSeq.zipWithIndex) { - os.write(s"== Subtree ${i + 1} / ${codegenSeq.size} ==\n".getBytes()) - os.write(subtree.getBytes) - os.write("\nGenerated code:\n".getBytes) - os.write(s"${code}\n".getBytes) + writer.write(s"== Subtree ${i + 1} / ${codegenSeq.size} ==\n") + writer.write(subtree) + writer.write("\nGenerated code:\n") + writer.write(s"${code}\n") } } From ce2c08688bb8b51e97f686c95279a5f42b52116a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 16 Sep 2018 17:34:25 +0200 Subject: [PATCH 12/41] Using java.io.Writer --- .../org/apache/spark/sql/execution/QueryExecution.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 863f8ceee5095..6a3c936ddf9d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -259,11 +259,12 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { * 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))) - val maxFields = SparkEnv.get.conf.getInt(Utils.MAX_TO_STRING_FIELDS, - Utils.DEFAULT_MAX_TO_STRING_FIELDS) + try { SparkEnv.get.conf.set(Utils.MAX_TO_STRING_FIELDS, Int.MaxValue.toString) writer.write("== Parsed Logical Plan ==\n") From 7abf14c736b90a7f2990356f91d0f90df45274b8 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 17 Sep 2018 20:14:32 +0200 Subject: [PATCH 13/41] Using StringWriter --- .../spark/sql/catalyst/trees/TreeNode.scala | 14 +++++++------- .../spark/sql/execution/QueryExecution.scala | 2 +- .../spark/sql/execution/debug/package.scala | 15 ++++++++------- 3 files changed, 16 insertions(+), 15 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 92faed352c77f..995dd717cd779 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -470,13 +470,13 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { def treeString: String = treeString(verbose = true) def treeString(verbose: Boolean, addSuffix: Boolean = false): String = { - val baos = new ByteArrayOutputStream() - val writer = new BufferedWriter(new OutputStreamWriter(baos)) - - treeString(writer, verbose, addSuffix) - writer.flush() - - baos.toString + val writer = new StringWriter() + try { + treeString(writer, verbose, addSuffix) + writer.toString + } finally { + writer.close() + } } def treeString(writer: Writer, verbose: Boolean, addSuffix: Boolean): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 6a3c936ddf9d9..cb894e4d85b98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -279,7 +279,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { executedPlan.treeString(writer, verbose = true, addSuffix = false) writer.flush() writer.write("== Whole Stage Codegen ==\n") - org.apache.spark.sql.execution.debug.writerCodegen(writer, executedPlan) + org.apache.spark.sql.execution.debug.writeCodegen(writer, executedPlan) } finally { writer.close() SparkEnv.get.conf.set(Utils.MAX_TO_STRING_FIELDS, maxFields.toString) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index b266d9beccded..fc78ef87b89a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -70,16 +70,17 @@ package object debug { * @return single String containing all WholeStageCodegen subtrees and corresponding codegen */ def codegenString(plan: SparkPlan): String = { - val baos = new ByteArrayOutputStream() - val writer = new BufferedWriter(new OutputStreamWriter(baos)) + val writer = new StringWriter() - writerCodegen(writer, plan) - writer.flush() - - baos.toString + try { + writeCodegen(writer, plan) + writer.toString + } finally { + writer.close() + } } - def writerCodegen(writer: Writer, plan: SparkPlan): Unit = { + def writeCodegen(writer: Writer, plan: SparkPlan): Unit = { val codegenSeq = codegenStringSeq(plan) writer.write(s"Found ${codegenSeq.size} WholeStageCodegen subtrees.\n") for (((subtree, code), i) <- codegenSeq.zipWithIndex) { From d1188e38ed9bcabeecce511a648d16335e101fde Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 17 Sep 2018 20:24:56 +0200 Subject: [PATCH 14/41] Removing unneeded buffering and flushing --- .../scala/org/apache/spark/sql/execution/QueryExecution.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index cb894e4d85b98..193bf995a0c9f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -263,7 +263,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { 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))) + val writer = new OutputStreamWriter(fs.create(filePath)) try { SparkEnv.get.conf.set(Utils.MAX_TO_STRING_FIELDS, Int.MaxValue.toString) @@ -277,7 +277,6 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { optimizedPlan.treeString(writer, verbose = true, addSuffix = false) writer.write("== Physical Plan ==\n") executedPlan.treeString(writer, verbose = true, addSuffix = false) - writer.flush() writer.write("== Whole Stage Codegen ==\n") org.apache.spark.sql.execution.debug.writeCodegen(writer, executedPlan) } finally { From 71ff7d1387fbe7d30299fe38471bce26fe73dad5 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 17 Sep 2018 22:05:26 +0200 Subject: [PATCH 15/41] Code de-duplication among toString and toFile --- .../spark/sql/execution/QueryExecution.scala | 55 +++++++++---------- 1 file changed, 27 insertions(+), 28 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 193bf995a0c9f..48671bdc32c25 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -import java.io.{BufferedWriter, OutputStreamWriter} +import java.io.{OutputStreamWriter, StringWriter, Writer} import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} @@ -102,7 +102,6 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { protected def stringOrError[A](f: => A): String = try f.toString catch { case e: AnalysisException => e.toString } - /** * Returns the result as a hive compatible sequence of strings. This is used in tests and * `SparkSQLDriver` for CLI applications. @@ -193,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}")) + } + writer.write("\n") + writeOrError(writer)(analyzed.treeString(_, verbose = true, addSuffix = false)) + writer.write("== Optimized Logical Plan ==\n") + writeOrError(writer)(optimizedPlan.treeString(_, verbose = true, addSuffix = false)) + writer.write("== Physical Plan ==\n") + writeOrError(writer)(executedPlan.treeString(_, verbose = true, addSuffix = false)) + } + override def toString: String = withRedaction { - def output = Utils.truncatedString( - analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}"), ", ") - val analyzedPlan = Seq( - stringOrError(output), - stringOrError(analyzed.treeString(verbose = true)) - ).filter(_.nonEmpty).mkString("\n") - - s"""== Parsed Logical Plan == - |${stringOrError(logical.treeString(verbose = true))} - |== Analyzed Logical Plan == - |$analyzedPlan - |== Optimized Logical Plan == - |${stringOrError(optimizedPlan.treeString(verbose = true))} - |== Physical Plan == - |${stringOrError(executedPlan.treeString(verbose = true))} - """.stripMargin.trim + val writer = new StringWriter() + try { + writePlans(writer) + writer.toString + } finally { + writer.close() + } } def stringWithStats: String = withRedaction { @@ -267,16 +275,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { try { SparkEnv.get.conf.set(Utils.MAX_TO_STRING_FIELDS, Int.MaxValue.toString) - writer.write("== Parsed Logical Plan ==\n") - logical.treeString(writer, verbose = true, addSuffix = false) - writer.write("== Analyzed Logical Plan ==\n") - analyzed.output.foreach(o => writer.write(s"${o.name}: ${o.dataType.simpleString}")) - writer.write("\n") - analyzed.treeString(writer, verbose = true, addSuffix = false) - writer.write("== Optimized Logical Plan ==\n") - optimizedPlan.treeString(writer, verbose = true, addSuffix = false) - writer.write("== Physical Plan ==\n") - executedPlan.treeString(writer, verbose = true, addSuffix = false) + writePlans(writer) writer.write("== Whole Stage Codegen ==\n") org.apache.spark.sql.execution.debug.writeCodegen(writer, executedPlan) } finally { From ac94a860e3c497d4f0b3861b149c546b8a59e0dc Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 18 Sep 2018 12:05:15 +0200 Subject: [PATCH 16/41] Using StringBuilderWriter and fix tests --- .../spark/sql/execution/QueryExecution.scala | 27 ++++++++++--------- .../spark/sql/execution/debug/package.scala | 4 ++- 2 files changed, 18 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 48671bdc32c25..9f7c01eef3c84 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -17,10 +17,11 @@ package org.apache.spark.sql.execution -import java.io.{OutputStreamWriter, StringWriter, Writer} +import java.io.{OutputStreamWriter, Writer} import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} +import org.apache.commons.io.output.StringBuilderWriter import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.SparkEnv @@ -196,22 +197,24 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { try f(writer) catch { case e: AnalysisException => writer.write(e.toString) } private def writePlans(writer: Writer): Unit = { + val (verbose, addSuffix) = (true, false) + 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}")) - } + writeOrError(writer)(logical.treeString(_, verbose, addSuffix)) + writer.write("\n== Analyzed Logical Plan ==\n") + val analyzedOutput = stringOrError(Utils.truncatedString( + analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}"), ", ")) + writer.write(analyzedOutput) writer.write("\n") - writeOrError(writer)(analyzed.treeString(_, verbose = true, addSuffix = false)) - writer.write("== Optimized Logical Plan ==\n") - writeOrError(writer)(optimizedPlan.treeString(_, verbose = true, addSuffix = false)) - writer.write("== Physical Plan ==\n") - writeOrError(writer)(executedPlan.treeString(_, verbose = true, addSuffix = false)) + writeOrError(writer)(analyzed.treeString(_, verbose, addSuffix)) + writer.write("\n== Optimized Logical Plan ==\n") + writeOrError(writer)(optimizedPlan.treeString(_, verbose, addSuffix)) + writer.write("\n== Physical Plan ==\n") + writeOrError(writer)(executedPlan.treeString(_, verbose, addSuffix)) } override def toString: String = withRedaction { - val writer = new StringWriter() + val writer = new StringBuilderWriter() try { writePlans(writer) writer.toString diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index fc78ef87b89a7..59653cadb6cd5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -22,6 +22,8 @@ import java.util.Collections import scala.collection.JavaConverters._ +import org.apache.commons.io.output.StringBuilderWriter + import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql._ @@ -70,7 +72,7 @@ package object debug { * @return single String containing all WholeStageCodegen subtrees and corresponding codegen */ def codegenString(plan: SparkPlan): String = { - val writer = new StringWriter() + val writer = new StringBuilderWriter() try { writeCodegen(writer, plan) From f2906d9e569b74fd174cb2d0555dcc40b80c9cac Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 18 Sep 2018 12:26:09 +0200 Subject: [PATCH 17/41] Do not change maxFields so far --- core/src/main/scala/org/apache/spark/util/Utils.scala | 3 +-- .../org/apache/spark/sql/execution/QueryExecution.scala | 6 +----- .../apache/spark/sql/execution/QueryExecutionSuite.scala | 6 +++++- 3 files changed, 7 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 151d79f961fb3..14f68cd6f3509 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -99,11 +99,10 @@ 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" private[spark] def maxNumToStringFields = { if (SparkEnv.get != null) { - SparkEnv.get.conf.getInt(MAX_TO_STRING_FIELDS, DEFAULT_MAX_TO_STRING_FIELDS) + SparkEnv.get.conf.getInt("spark.debug.maxToStringFields", DEFAULT_MAX_TO_STRING_FIELDS) } else { DEFAULT_MAX_TO_STRING_FIELDS } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 9f7c01eef3c84..ccf75e1eedc5a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -270,20 +270,16 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { * 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 OutputStreamWriter(fs.create(filePath)) try { - SparkEnv.get.conf.set(Utils.MAX_TO_STRING_FIELDS, Int.MaxValue.toString) writePlans(writer) - writer.write("== Whole Stage Codegen ==\n") + writer.write("\n== Whole Stage Codegen ==\n") org.apache.spark.sql.execution.debug.writeCodegen(writer, executedPlan) } finally { writer.close() - SparkEnv.get.conf.set(Utils.MAX_TO_STRING_FIELDS, maxFields.toString) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index 009ce2b4ddf39..1d450435b4063 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -33,13 +33,17 @@ class QueryExecutionSuite extends SharedSQLContext { .takeWhile(_ != "== Whole Stage Codegen ==") == List( "== Parsed Logical Plan ==", "Range (0, 10, step=1, splits=Some(2))", + "", "== Analyzed Logical Plan ==", "id: bigint", "Range (0, 10, step=1, splits=Some(2))", + "", "== Optimized Logical Plan ==", "Range (0, 10, step=1, splits=Some(2))", + "", "== Physical Plan ==", - "*(1) Range (0, 10, step=1, splits=2)")) + "*(1) Range (0, 10, step=1, splits=2)", + "")) } } From d3fede192123bd490b5a6a875559fa43d43bfc3e Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 18 Sep 2018 12:45:06 +0200 Subject: [PATCH 18/41] Added tests --- .../sql/execution/QueryExecutionSuite.scala | 63 ++++++++++++++----- 1 file changed, 48 insertions(+), 15 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index 1d450435b4063..a5922d7c825db 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -23,30 +23,63 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} import org.apache.spark.sql.test.SharedSQLContext class QueryExecutionSuite extends SharedSQLContext { + def checkDumpedPlans(path: String, expected: Int): Unit = { + assert(Source.fromFile(path).getLines.toList + .takeWhile(_ != "== Whole Stage Codegen ==") == List( + "== Parsed Logical Plan ==", + s"Range (0, $expected, step=1, splits=Some(2))", + "", + "== Analyzed Logical Plan ==", + "id: bigint", + s"Range (0, $expected, step=1, splits=Some(2))", + "", + "== Optimized Logical Plan ==", + s"Range (0, $expected, step=1, splits=Some(2))", + "", + "== Physical Plan ==", + s"*(1) Range (0, $expected, step=1, splits=2)", + "")) + } test("dumping query execution info to a file") { withTempDir { dir => val path = dir.getCanonicalPath + "/plans.txt" val df = spark.range(0, 10) df.queryExecution.debug.toFile(path) - assert(Source.fromFile(path).getLines.toList - .takeWhile(_ != "== Whole Stage Codegen ==") == List( - "== Parsed Logical Plan ==", - "Range (0, 10, step=1, splits=Some(2))", - "", - "== Analyzed Logical Plan ==", - "id: bigint", - "Range (0, 10, step=1, splits=Some(2))", - "", - "== Optimized Logical Plan ==", - "Range (0, 10, step=1, splits=Some(2))", - "", - "== Physical Plan ==", - "*(1) Range (0, 10, step=1, splits=2)", - "")) + checkDumpedPlans(path, expected = 10) + } + } + + test("dumping query execution info to an existing file") { + withTempDir { dir => + val path = dir.getCanonicalPath + "/plans.txt" + val df = spark.range(0, 10) + df.queryExecution.debug.toFile(path) + + val df2 = spark.range(0, 1) + df2.queryExecution.debug.toFile(path) + checkDumpedPlans(path, expected = 1) } } + test("dumping query execution info to non-existing folder") { + withTempDir { dir => + val path = dir.getCanonicalPath + "/newfolder/plans.txt" + val df = spark.range(0, 100) + df.queryExecution.debug.toFile(path) + checkDumpedPlans(path, expected = 100) + } + } + + test("dumping query execution info by invalid path") { + val path = "1234567890://plans.txt" + val exception = intercept[IllegalArgumentException] { + spark.range(0, 100).queryExecution.debug.toFile(path) + } + + assert(exception.getMessage.contains("Illegal character in scheme name")) + } + test("toString() exception/error handling") { spark.experimental.extraStrategies = Seq( new SparkStrategy { From c1538389bf3327467bd65e7f9fe8e18577cc1a17 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 18 Sep 2018 12:50:05 +0200 Subject: [PATCH 19/41] Using StringBuilderWriter in treeString --- .../scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 995dd717cd779..501b8d7662a70 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -23,6 +23,7 @@ import java.util.UUID import scala.collection.Map import scala.reflect.ClassTag +import org.apache.commons.io.output.StringBuilderWriter import org.apache.commons.lang3.ClassUtils import org.json4s.JsonAST._ import org.json4s.JsonDSL._ @@ -470,7 +471,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { def treeString: String = treeString(verbose = true) def treeString(verbose: Boolean, addSuffix: Boolean = false): String = { - val writer = new StringWriter() + val writer = new StringBuilderWriter() try { treeString(writer, verbose, addSuffix) writer.toString From 6fe08bf45952653af3cf49eecafd016dd11cc1a5 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 18 Sep 2018 14:16:38 +0200 Subject: [PATCH 20/41] Propagating numFields to truncatedString --- .../scala/org/apache/spark/util/Utils.scala | 15 +++++++ .../spark/sql/avro/AvroDataToCatalyst.scala | 2 +- .../spark/sql/avro/CatalystDataToAvro.scala | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 4 +- .../sql/catalyst/analysis/CheckAnalysis.scala | 6 +-- .../sql/catalyst/analysis/TypeCoercion.scala | 2 +- .../catalyst/encoders/ExpressionEncoder.scala | 4 +- .../sql/catalyst/expressions/Expression.scala | 4 +- .../expressions/codegen/javaCode.scala | 2 +- .../sql/catalyst/expressions/generators.scala | 2 +- .../expressions/higherOrderFunctions.scala | 4 +- .../spark/sql/catalyst/expressions/misc.scala | 4 +- .../expressions/namedExpressions.scala | 4 +- .../spark/sql/catalyst/plans/QueryPlan.scala | 6 ++- .../catalyst/plans/logical/LogicalPlan.scala | 4 +- .../plans/logical/basicLogicalOperators.scala | 8 ++-- .../spark/sql/catalyst/trees/TreeNode.scala | 39 +++++++++++-------- .../aggregate/PercentileSuite.scala | 2 +- .../sql/execution/DataSourceScanExec.scala | 11 ++++-- .../spark/sql/execution/ExistingRDD.scala | 4 +- .../spark/sql/execution/QueryExecution.scala | 16 ++++---- .../spark/sql/execution/SparkPlanInfo.scala | 2 +- .../sql/execution/WholeStageCodegenExec.scala | 21 +++++++--- .../aggregate/HashAggregateExec.scala | 13 ++++--- .../aggregate/ObjectHashAggregateExec.scala | 12 +++--- .../aggregate/SortAggregateExec.scala | 12 +++--- .../execution/basicPhysicalOperators.scala | 4 +- .../execution/columnar/InMemoryRelation.scala | 6 ++- .../datasources/LogicalRelation.scala | 4 +- .../SaveIntoDataSourceCommand.scala | 2 +- .../spark/sql/execution/datasources/ddl.scala | 2 +- .../datasources/v2/DataSourceV2Relation.scala | 6 ++- .../datasources/v2/DataSourceV2ScanExec.scala | 2 +- .../spark/sql/execution/debug/package.scala | 2 +- .../apache/spark/sql/execution/limit.scala | 6 +-- .../apache/spark/sql/execution/subquery.scala | 2 +- .../CreateHiveTableAsSelectCommand.scala | 2 +- 37 files changed, 146 insertions(+), 97 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 14f68cd6f3509..a74841e9db065 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -137,9 +137,24 @@ private[spark] object Utils extends Logging { } } + def truncatedString[T]( + seq: Seq[T], + start: String, + sep: String, + end: String, + maxFields: Option[Int]): String = { + val maxNumFields = maxFields.getOrElse(maxNumToStringFields) + truncatedString(seq, start, sep, end, maxNumFields) + } + /** Shorthand for calling truncatedString() without start or end strings. */ def truncatedString[T](seq: Seq[T], sep: String): String = truncatedString(seq, "", sep, "") + def truncatedString[T](seq: Seq[T], sep: String, maxFields: Option[Int]): String = { + truncatedString(seq, "", sep, "", maxFields) + } + + /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala index 915769fa708b0..ee4ab140b80bd 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala @@ -51,7 +51,7 @@ case class AvroDataToCatalyst(child: Expression, jsonFormatSchema: String) deserializer.deserialize(result) } - override def simpleString: String = { + override def simpleString(maxFields: Option[Int]): String = { s"from_avro(${child.sql}, ${dataType.simpleString})" } diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala index 141ff3782adfb..43a1825f5434b 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala +++ b/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})" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 580133dd971b1..24922f5c718ce 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -951,7 +951,7 @@ class Analyzer( case plan if containsDeserializer(plan.expressions) => plan case q: LogicalPlan => - logTrace(s"Attempting to resolve ${q.simpleString}") + logTrace(s"Attempting to resolve ${q.simpleString()}") q.mapExpressions(resolve(_, q)) } @@ -1745,7 +1745,7 @@ class Analyzer( case p if p.expressions.exists(hasGenerator) => throw new AnalysisException("Generators are not supported outside the SELECT clause, but " + - "got: " + p.simpleString) + "got: " + p.simpleString()) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 6a91d556b2f3e..a2fea116ab2c8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -308,7 +308,7 @@ trait CheckAnalysis extends PredicateHelper { val missingAttributes = o.missingInput.mkString(",") val input = o.inputSet.mkString(",") val msgForMissingAttributes = s"Resolved attribute(s) $missingAttributes missing " + - s"from $input in operator ${operator.simpleString}." + s"from $input in operator ${operator.simpleString()}." val resolver = plan.conf.resolver val attrsWithSameName = o.missingInput.filter { missing => @@ -373,7 +373,7 @@ trait CheckAnalysis extends PredicateHelper { s"""nondeterministic expressions are only allowed in |Project, Filter, Aggregate or Window, found: | ${o.expressions.map(_.sql).mkString(",")} - |in operator ${operator.simpleString} + |in operator ${operator.simpleString()} """.stripMargin) case _: UnresolvedHint => @@ -385,7 +385,7 @@ trait CheckAnalysis extends PredicateHelper { } extendedCheckRules.foreach(_(plan)) plan.foreachUp { - case o if !o.resolved => failAnalysis(s"unresolved operator ${o.simpleString}") + case o if !o.resolved => failAnalysis(s"unresolved operator ${o.simpleString()}") case _ => } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 288b6358fbff1..e51703d7b2bdf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -1043,7 +1043,7 @@ trait TypeCoercionRule extends Rule[LogicalPlan] with Logging { case Some(newType) if a.dataType == newType.dataType => a case Some(newType) => logDebug( - s"Promoting $a from ${a.dataType} to ${newType.dataType} in ${q.simpleString}") + s"Promoting $a from ${a.dataType} to ${newType.dataType} in ${q.simpleString()}") newType } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index cbea3c017a265..7b328ff28782c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -290,7 +290,7 @@ case class ExpressionEncoder[T]( } catch { case e: Exception => throw new RuntimeException( - s"Error while encoding: $e\n${serializer.map(_.simpleString).mkString("\n")}", e) + s"Error while encoding: $e\n${serializer.map(_.simpleString()).mkString("\n")}", e) } /** @@ -302,7 +302,7 @@ case class ExpressionEncoder[T]( constructProjection(row).get(0, ObjectType(clsTag.runtimeClass)).asInstanceOf[T] } catch { case e: Exception => - throw new RuntimeException(s"Error while decoding: $e\n${deserializer.simpleString}", e) + throw new RuntimeException(s"Error while decoding: $e\n${deserializer.simpleString()}", e) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 773aefc0ac1f9..ba2b0fc8ad9b7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -230,9 +230,9 @@ abstract class Expression extends TreeNode[Expression] { // Marks this as final, Expression.verboseString should never be called, and thus shouldn't be // overridden by concrete classes. - final override def verboseString: String = simpleString + final override def verboseString(maxFields: Option[Int]): String = simpleString(maxFields) - override def simpleString: String = toString + override def simpleString(maxFields: Option[Int]): String = toString override def toString: String = prettyName + Utils.truncatedString( flatArguments.toSeq, "(", ", ", ")") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala index 17d4a0dc4e884..7e88e0533727d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala @@ -197,7 +197,7 @@ trait Block extends TreeNode[Block] with JavaCode { case _ => code"$this\n$other" } - override def verboseString: String = toString + override def verboseString(maxFields: Option[Int]): String = toString } object Block { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index d6e67b9ac3d10..9899a91c90285 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -101,7 +101,7 @@ case class UserDefinedGenerator( inputRow = new InterpretedProjection(children) convertToScala = { val inputSchema = StructType(children.map { e => - StructField(e.simpleString, e.dataType, nullable = true) + StructField(e.simpleString(), e.dataType, nullable = true) }) CatalystTypeConverters.createToScalaConverter(inputSchema) }.asInstanceOf[InternalRow => Row] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala index b07d9466ba0d1..9970814e3ec08 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala @@ -54,7 +54,9 @@ case class NamedLambdaVariable( override def toString: String = s"lambda $name#${exprId.id}$typeSuffix" - override def simpleString: String = s"lambda $name#${exprId.id}: ${dataType.simpleString}" + override def simpleString(maxFields: Option[Int]): String = { + s"lambda $name#${exprId.id}: ${dataType.simpleString}" + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index 0cdeda9b10516..2a650dccacb83 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -40,7 +40,7 @@ case class PrintToStderr(child: Expression) extends UnaryExpression { input } - private val outputPrefix = s"Result of ${child.simpleString} is " + private val outputPrefix = s"Result of ${child.simpleString()} is " override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val outputPrefixField = ctx.addReferenceObj("outputPrefix", outputPrefix) @@ -72,7 +72,7 @@ case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCa override def prettyName: String = "assert_true" - private val errMsg = s"'${child.simpleString}' is not true!" + private val errMsg = s"'${child.simpleString()}' is not true!" override def eval(input: InternalRow) : Any = { val v = child.eval(input) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 584a2946bd564..8a6fab08100b0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -326,7 +326,9 @@ case class AttributeReference( // Since the expression id is not in the first constructor it is missing from the default // tree string. - override def simpleString: String = s"$name#${exprId.id}: ${dataType.simpleString}" + override def simpleString(maxFields: Option[Int]): String = { + s"$name#${exprId.id}: ${dataType.simpleString}" + } override def sql: String = { val qualifierPrefix = if (qualifier.nonEmpty) qualifier.mkString(".") + "." else "" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index b1ffdca091461..bf8d92f304483 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -172,9 +172,11 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT */ protected def statePrefix = if (missingInput.nonEmpty && children.nonEmpty) "!" else "" - override def simpleString: String = statePrefix + super.simpleString + override def simpleString(maxFields: Option[Int]): String = { + statePrefix + super.simpleString(maxFields) + } - override def verboseString: String = simpleString + override def verboseString(maxFields: Option[Int]): String = simpleString(maxFields) /** * All the subqueries of current plan. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 5f136629eb15b..3dcf8aff069e9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -36,8 +36,8 @@ abstract class LogicalPlan /** Returns true if this subtree has data from a streaming data source. */ def isStreaming: Boolean = children.exists(_.isStreaming == true) - override def verboseStringWithSuffix: String = { - super.verboseString + statsCache.map(", " + _.toString).getOrElse("") + override def verboseStringWithSuffix(maxFields: Option[Int]): String = { + super.verboseString(maxFields) + statsCache.map(", " + _.toString).getOrElse("") } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 7ff83a9be3622..d5ec24a5b0830 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -468,7 +468,7 @@ case class View( override def newInstance(): LogicalPlan = copy(output = output.map(_.newInstance())) - override def simpleString: String = { + override def simpleString(maxFields: Option[Int]): String = { s"View (${desc.identifier}, ${output.mkString("[", ",", "]")})" } } @@ -484,8 +484,8 @@ case class View( case class With(child: LogicalPlan, cteRelations: Seq[(String, SubqueryAlias)]) extends UnaryNode { override def output: Seq[Attribute] = child.output - override def simpleString: String = { - val cteAliases = Utils.truncatedString(cteRelations.map(_._1), "[", ", ", "]") + override def simpleString(maxFields: Option[Int]): String = { + val cteAliases = Utils.truncatedString(cteRelations.map(_._1), "[", ", ", "]", maxFields) s"CTE $cteAliases" } @@ -557,7 +557,7 @@ case class Range( override def newInstance(): Range = copy(output = output.map(_.newInstance())) - override def simpleString: String = { + override def simpleString(maxFields: Option[Int]): String = { s"Range ($start, $end, step=$step, splits=$numSlices)" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 501b8d7662a70..404dfa9f3025b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -433,11 +433,11 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { private lazy val allChildren: Set[TreeNode[_]] = (children ++ innerChildren).toSet[TreeNode[_]] /** Returns a string representing the arguments to this node, minus any children */ - def argString: String = stringArgs.flatMap { + def argString(maxFields: Option[Int]): String = stringArgs.flatMap { case tn: TreeNode[_] if allChildren.contains(tn) => Nil case Some(tn: TreeNode[_]) if allChildren.contains(tn) => Nil - case Some(tn: TreeNode[_]) => tn.simpleString :: Nil - case tn: TreeNode[_] => tn.simpleString :: Nil + case Some(tn: TreeNode[_]) => tn.simpleString() :: Nil + case tn: TreeNode[_] => tn.simpleString() :: Nil case seq: Seq[Any] if seq.toSet.subsetOf(allChildren.asInstanceOf[Set[Any]]) => Nil case iter: Iterable[_] if iter.isEmpty => Nil case seq: Seq[_] => Utils.truncatedString(seq, "[", ", ", "]") :: Nil @@ -457,13 +457,15 @@ 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] = None): String = { + s"$nodeName ${argString(maxFields)}".trim + } /** ONE line description of this node with more information */ - def verboseString: String + def verboseString(maxFields: Option[Int]): String /** ONE line description of this node with some suffix information */ - def verboseStringWithSuffix: String = verboseString + def verboseStringWithSuffix(maxFields: Option[Int]): String = verboseString(maxFields) override def toString: String = treeString @@ -473,15 +475,19 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { def treeString(verbose: Boolean, addSuffix: Boolean = false): String = { val writer = new StringBuilderWriter() try { - treeString(writer, verbose, addSuffix) + treeString(writer, verbose, addSuffix, None) writer.toString } finally { writer.close() } } - def treeString(writer: Writer, verbose: Boolean, addSuffix: Boolean): Unit = { - generateTreeString(0, Nil, writer, verbose, "", addSuffix) + def treeString( + writer: Writer, + verbose: Boolean, + addSuffix: Boolean, + maxFields: Option[Int]): Unit = { + generateTreeString(0, Nil, writer, verbose, "", addSuffix, maxFields) } /** @@ -547,7 +553,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { writer: Writer, verbose: Boolean, prefix: String = "", - addSuffix: Boolean = false): Unit = { + addSuffix: Boolean = false, + maxFields: Option[Int]): Unit = { if (depth > 0) { lastChildren.init.foreach { isLast => @@ -557,9 +564,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } val str = if (verbose) { - if (addSuffix) verboseStringWithSuffix else verboseString + if (addSuffix) verboseStringWithSuffix(maxFields) else verboseString(maxFields) } else { - simpleString + simpleString() } writer.write(prefix) writer.write(str) @@ -568,17 +575,17 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { if (innerChildren.nonEmpty) { innerChildren.init.foreach(_.generateTreeString( depth + 2, lastChildren :+ children.isEmpty :+ false, writer, verbose, - addSuffix = addSuffix)) + addSuffix = addSuffix, maxFields = maxFields)) innerChildren.last.generateTreeString( depth + 2, lastChildren :+ children.isEmpty :+ true, writer, verbose, - addSuffix = addSuffix) + addSuffix = addSuffix, maxFields = maxFields) } if (children.nonEmpty) { children.init.foreach(_.generateTreeString( - depth + 1, lastChildren :+ false, writer, verbose, prefix, addSuffix)) + depth + 1, lastChildren :+ false, writer, verbose, prefix, addSuffix, maxFields)) children.last.generateTreeString( - depth + 1, lastChildren :+ true, writer, verbose, prefix, addSuffix) + depth + 1, lastChildren :+ true, writer, verbose, prefix, addSuffix, maxFields) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala index 2420ba513f287..40cabc3d3a01a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala @@ -215,7 +215,7 @@ class PercentileSuite extends SparkFunSuite { val percentile2 = new Percentile(child, percentage) assertEqual(percentile2.checkInputDataTypes(), TypeCheckFailure(s"Percentage(s) must be between 0.0 and 1.0, " + - s"but got ${percentage.simpleString}")) + s"but got ${percentage.simpleString()}")) } val nonFoldablePercentage = Seq(NonFoldableLiteral(0.5), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 36ed016773b67..ce1e8a36731f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -51,16 +51,19 @@ trait DataSourceScanExec extends LeafExecNode with CodegenSupport { // Metadata that describes more details of this scan. protected def metadata: Map[String, String] - override def simpleString: String = { + override def simpleString(maxFields: Option[Int]): String = { val metadataEntries = metadata.toSeq.sorted.map { case (key, value) => key + ": " + StringUtils.abbreviate(redact(value), 100) } - val metadataStr = Utils.truncatedString(metadataEntries, " ", ", ", "") - s"$nodeNamePrefix$nodeName${Utils.truncatedString(output, "[", ",", "]")}$metadataStr" + val metadataStr = Utils.truncatedString(metadataEntries, " ", ", ", "", maxFields) + val outputStr = Utils.truncatedString(output, "[", ",", "]", maxFields) + s"$nodeNamePrefix$nodeName${outputStr}$metadataStr" } - override def verboseString: String = redact(super.verboseString) + override def verboseString(maxFields: Option[Int]): String = { + redact(super.verboseString(maxFields)) + } override def treeString(verbose: Boolean, addSuffix: Boolean): String = { redact(super.treeString(verbose, addSuffix)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 2962becb64e88..5153bf0b0305a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -119,7 +119,7 @@ case class ExternalRDDScanExec[T]( } } - override def simpleString: String = { + override def simpleString(maxFields: Option[Int]): String = { s"$nodeName${output.mkString("[", ",", "]")}" } } @@ -196,7 +196,7 @@ case class RDDScanExec( } } - override def simpleString: String = { + override def simpleString(maxFields: Option[Int]): String = { s"$nodeName${Utils.truncatedString(output, "[", ",", "]")}" } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index ccf75e1eedc5a..61f68ffe18822 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -196,27 +196,27 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { 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 = { + private def writePlans(writer: Writer, maxFields: Option[Int]): Unit = { val (verbose, addSuffix) = (true, false) writer.write("== Parsed Logical Plan ==\n") - writeOrError(writer)(logical.treeString(_, verbose, addSuffix)) + writeOrError(writer)(logical.treeString(_, verbose, addSuffix, maxFields)) writer.write("\n== Analyzed Logical Plan ==\n") val analyzedOutput = stringOrError(Utils.truncatedString( - analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}"), ", ")) + analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}"), ", ", maxFields)) writer.write(analyzedOutput) writer.write("\n") - writeOrError(writer)(analyzed.treeString(_, verbose, addSuffix)) + writeOrError(writer)(analyzed.treeString(_, verbose, addSuffix, maxFields)) writer.write("\n== Optimized Logical Plan ==\n") - writeOrError(writer)(optimizedPlan.treeString(_, verbose, addSuffix)) + writeOrError(writer)(optimizedPlan.treeString(_, verbose, addSuffix, maxFields)) writer.write("\n== Physical Plan ==\n") - writeOrError(writer)(executedPlan.treeString(_, verbose, addSuffix)) + writeOrError(writer)(executedPlan.treeString(_, verbose, addSuffix, maxFields)) } override def toString: String = withRedaction { val writer = new StringBuilderWriter() try { - writePlans(writer) + writePlans(writer, None) writer.toString } finally { writer.close() @@ -275,7 +275,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { val writer = new OutputStreamWriter(fs.create(filePath)) try { - writePlans(writer) + writePlans(writer, Some(Int.MaxValue)) writer.write("\n== Whole Stage Codegen ==\n") org.apache.spark.sql.execution.debug.writeCodegen(writer, executedPlan) } finally { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index 59ffd16381116..53df9b5b19746 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -62,7 +62,7 @@ private[execution] object SparkPlanInfo { case fileScan: FileSourceScanExec => fileScan.metadata case _ => Map[String, String]() } - new SparkPlanInfo(plan.nodeName, plan.simpleString, children.map(fromSparkPlan), + new SparkPlanInfo(plan.nodeName, plan.simpleString(), children.map(fromSparkPlan), metadata, metrics) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 9cb1c9e5ffeb2..d40008fd984da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -87,7 +87,7 @@ trait CodegenSupport extends SparkPlan { this.parent = parent ctx.freshNamePrefix = variablePrefix s""" - |${ctx.registerComment(s"PRODUCE: ${this.simpleString}")} + |${ctx.registerComment(s"PRODUCE: ${this.simpleString()}")} |${doProduce(ctx)} """.stripMargin } @@ -185,7 +185,7 @@ trait CodegenSupport extends SparkPlan { parent.doConsume(ctx, inputVars, rowVar) } s""" - |${ctx.registerComment(s"CONSUME: ${parent.simpleString}")} + |${ctx.registerComment(s"CONSUME: ${parent.simpleString()}")} |$evaluated |$consumeFunc """.stripMargin @@ -396,8 +396,9 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp writer: Writer, verbose: Boolean, prefix: String = "", - addSuffix: Boolean = false): Unit = { - child.generateTreeString(depth, lastChildren, writer, verbose, "") + addSuffix: Boolean = false, + maxFields: Option[Int]): Unit = { + child.generateTreeString(depth, lastChildren, writer, verbose, "", false, maxFields) } override def needCopyResult: Boolean = false @@ -672,8 +673,16 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) writer: Writer, verbose: Boolean, prefix: String = "", - addSuffix: Boolean = false): Unit = { - child.generateTreeString(depth, lastChildren, writer, verbose, s"*($codegenStageId) ") + addSuffix: Boolean = false, + maxFields: Option[Int]): Unit = { + child.generateTreeString( + depth, + lastChildren, + writer, + verbose, + s"*($codegenStageId) ", + false, + maxFields) } override def needStopCheck: Boolean = true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 98adba50b2973..a3397fb5557fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -915,18 +915,19 @@ case class HashAggregateExec( """ } - override def verboseString: String = toString(verbose = true) + override def verboseString(maxFields: Option[Int]): String = toString(verbose = true, maxFields) - override def simpleString: String = toString(verbose = false) + override def simpleString(maxFields: Option[Int]): String = toString(verbose = false, maxFields) - private def toString(verbose: Boolean): String = { + private def toString(verbose: Boolean, maxFields: Option[Int]): String = { val allAggregateExpressions = aggregateExpressions testFallbackStartsAt match { case None => - val keyString = Utils.truncatedString(groupingExpressions, "[", ", ", "]") - val functionString = Utils.truncatedString(allAggregateExpressions, "[", ", ", "]") - val outputString = Utils.truncatedString(output, "[", ", ", "]") + val keyString = Utils.truncatedString(groupingExpressions, "[", ", ", "]", maxFields) + val functionString = + Utils.truncatedString(allAggregateExpressions, "[", ", ", "]", maxFields) + val outputString = Utils.truncatedString(output, "[", ", ", "]", maxFields) if (verbose) { s"HashAggregate(keys=$keyString, functions=$functionString, output=$outputString)" } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala index 66955b8ef723c..83b0fe8f8f110 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala @@ -137,15 +137,15 @@ case class ObjectHashAggregateExec( } } - override def verboseString: String = toString(verbose = true) + override def verboseString(maxFields: Option[Int]): String = toString(verbose = true, maxFields) - override def simpleString: String = toString(verbose = false) + override def simpleString(maxFields: Option[Int]): String = toString(verbose = false, maxFields) - private def toString(verbose: Boolean): String = { + private def toString(verbose: Boolean, maxFields: Option[Int]): String = { val allAggregateExpressions = aggregateExpressions - val keyString = Utils.truncatedString(groupingExpressions, "[", ", ", "]") - val functionString = Utils.truncatedString(allAggregateExpressions, "[", ", ", "]") - val outputString = Utils.truncatedString(output, "[", ", ", "]") + val keyString = Utils.truncatedString(groupingExpressions, "[", ", ", "]", maxFields) + val functionString = Utils.truncatedString(allAggregateExpressions, "[", ", ", "]", maxFields) + val outputString = Utils.truncatedString(output, "[", ", ", "]", maxFields) if (verbose) { s"ObjectHashAggregate(keys=$keyString, functions=$functionString, output=$outputString)" } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala index fc87de2c52e41..8de14cee7e94f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala @@ -107,16 +107,16 @@ case class SortAggregateExec( } } - override def simpleString: String = toString(verbose = false) + override def simpleString(maxFields: Option[Int]): String = toString(verbose = false, maxFields) - override def verboseString: String = toString(verbose = true) + override def verboseString(maxFields: Option[Int]): String = toString(verbose = true, maxFields) - private def toString(verbose: Boolean): String = { + private def toString(verbose: Boolean, maxFields: Option[Int]): String = { val allAggregateExpressions = aggregateExpressions - val keyString = Utils.truncatedString(groupingExpressions, "[", ", ", "]") - val functionString = Utils.truncatedString(allAggregateExpressions, "[", ", ", "]") - val outputString = Utils.truncatedString(output, "[", ", ", "]") + val keyString = Utils.truncatedString(groupingExpressions, "[", ", ", "]", maxFields) + val functionString = Utils.truncatedString(allAggregateExpressions, "[", ", ", "]", maxFields) + val outputString = Utils.truncatedString(output, "[", ", ", "]", maxFields) if (verbose) { s"SortAggregate(key=$keyString, functions=$functionString, output=$outputString)" } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index 222a1b8bc7301..cab6ca2594143 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -552,7 +552,9 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) } } - override def simpleString: String = s"Range ($start, $end, step=$step, splits=$numSlices)" + override def simpleString(maxFields: Option[Int]): String = { + s"Range ($start, $end, step=$step, splits=$numSlices)" + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 1a8fbaca53f59..33d1a1ef5d55f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -208,6 +208,8 @@ case class InMemoryRelation( override protected def otherCopyArgs: Seq[AnyRef] = Seq(statsOfPlanToCache) - override def simpleString: String = - s"InMemoryRelation [${Utils.truncatedString(output, ", ")}], ${cacheBuilder.storageLevel}" + override def simpleString(maxFields: Option[Int]): String = { + val outputStr = Utils.truncatedString(output, ", ", maxFields) + s"InMemoryRelation [${outputStr}], ${cacheBuilder.storageLevel}" + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index 8d715f6342988..1527440f3cb6d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -63,7 +63,9 @@ case class LogicalRelation( case _ => // Do nothing. } - override def simpleString: String = s"Relation[${Utils.truncatedString(output, ",")}] $relation" + override def simpleString(maxFields: Option[Int]): String = { + s"Relation[${Utils.truncatedString(output, ",", maxFields)}] $relation" + } } object LogicalRelation { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala index 00b1b5dedb593..5c26a8c9f7575 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala @@ -48,7 +48,7 @@ case class SaveIntoDataSourceCommand( Seq.empty[Row] } - override def simpleString: String = { + override def simpleString(maxFields: Option[Int]): String = { val redacted = SQLConf.get.redactOptions(options) s"SaveIntoDataSourceCommand ${dataSource}, ${redacted}, ${mode}" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index fdc5e85f3c2ea..d9219c0a9c414 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -68,7 +68,7 @@ case class CreateTempViewUsing( s"Temporary view '$tableIdent' should not have specified a database") } - override def argString: String = { + override def argString(maxFields: Option[Int]): String = { s"[tableIdent:$tableIdent " + userSpecifiedSchema.map(_ + " ").getOrElse("") + s"replace:$replace " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index f7e29593a6353..1b9ae9e326d85 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -56,7 +56,7 @@ case class DataSourceV2Relation( override def pushedFilters: Seq[Expression] = Seq.empty - override def simpleString: String = "RelationV2 " + metadataString + override def simpleString(maxFields: Option[Int]): String = "RelationV2 " + metadataString def newWriteSupport(): BatchWriteSupport = source.createWriteSupport(options, schema) @@ -90,7 +90,9 @@ case class StreamingDataSourceV2Relation( override def isStreaming: Boolean = true - override def simpleString: String = "Streaming RelationV2 " + metadataString + override def simpleString(maxFields: Option[Int]): String = { + "Streaming RelationV2 " + metadataString + } override def pushedFilters: Seq[Expression] = Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala index 04a97735d024d..5474c11ee3aec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala @@ -40,7 +40,7 @@ case class DataSourceV2ScanExec( @transient scanConfig: ScanConfig) extends LeafExecNode with DataSourceV2StringFormat with ColumnarBatchScan { - override def simpleString: String = "ScanV2 " + metadataString + override def simpleString(maxFields: Option[Int]): String = "ScanV2 " + metadataString // TODO: unify the equal/hashCode implementation for all data source v2 query plans. override def equals(other: Any): Boolean = other match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index 59653cadb6cd5..0c5e5333e3c7f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -216,7 +216,7 @@ package object debug { val columnStats: Array[ColumnMetrics] = Array.fill(child.output.size)(new ColumnMetrics()) def dumpStats(): Unit = { - debugPrint(s"== ${child.simpleString} ==") + debugPrint(s"== ${child.simpleString()} ==") debugPrint(s"Tuples output: ${tupleCount.value}") child.output.zip(columnStats).foreach { case (attr, metric) => // This is called on driver. All accumulator updates have a fixed value. So it's safe to use diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index 1a09632f93ca1..e135cdb9eeef1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -243,9 +243,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 = { + val orderByString = Utils.truncatedString(sortOrder, "[", ",", "]", maxFields) + val outputString = Utils.truncatedString(output, "[", ",", "]", maxFields) s"TakeOrderedAndProject(limit=$limit, orderBy=$orderByString, output=$outputString)" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index d11045fb6ac8c..f39acd435175a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -51,7 +51,7 @@ case class ScalarSubquery( override def dataType: DataType = plan.schema.fields.head.dataType override def children: Seq[Expression] = Nil override def nullable: Boolean = true - override def toString: String = plan.simpleString + override def toString: String = plan.simpleString() override def withNewPlan(query: SubqueryExec): ScalarSubquery = copy(plan = query) override def semanticEquals(other: Expression): Boolean = other match { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index aa573b54a2b62..ac73bbdcae5ff 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -95,7 +95,7 @@ case class CreateHiveTableAsSelectCommand( Seq.empty[Row] } - override def argString: String = { + override def argString(maxFields: Option[Int]): String = { s"[Database:${tableDesc.database}}, " + s"TableName: ${tableDesc.identifier.table}, " + s"InsertIntoHiveTable]" From 33249274cd9b2fa8ff5e934fe572123e562bf105 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 18 Sep 2018 15:00:22 +0200 Subject: [PATCH 21/41] Bug fix + test --- .../spark/sql/catalyst/trees/TreeNode.scala | 6 ++--- .../sql/execution/QueryExecutionSuite.scala | 22 +++++++++++++++++++ 2 files changed, 25 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 404dfa9f3025b..4edd646eecacb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -440,10 +440,10 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { case tn: TreeNode[_] => tn.simpleString() :: Nil case seq: Seq[Any] if seq.toSet.subsetOf(allChildren.asInstanceOf[Set[Any]]) => Nil case iter: Iterable[_] if iter.isEmpty => Nil - case seq: Seq[_] => Utils.truncatedString(seq, "[", ", ", "]") :: Nil - case set: Set[_] => Utils.truncatedString(set.toSeq, "{", ", ", "}") :: Nil + case seq: Seq[_] => Utils.truncatedString(seq, "[", ", ", "]", maxFields) :: Nil + case set: Set[_] => Utils.truncatedString(set.toSeq, "{", ", ", "}", maxFields) :: Nil case array: Array[_] if array.isEmpty => Nil - case array: Array[_] => Utils.truncatedString(array, "[", ", ", "]") :: Nil + case array: Array[_] => Utils.truncatedString(array, "[", ", ", "]", maxFields) :: Nil case null => Nil case None => Nil case Some(null) => Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index a5922d7c825db..ae865231dd466 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -22,7 +22,16 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} import org.apache.spark.sql.test.SharedSQLContext +case class QueryExecutionTestRecord( + c0: Int, c1: Int, c2: Int, c3: Int, c4: Int, + c5: Int, c6: Int, c7: Int, c8: Int, c9: Int, + c10: Int, c11: Int, c12: Int, c13: Int, c14: Int, + c15: Int, c16: Int, c17: Int, c18: Int, c19: Int, + c20: Int, c21: Int, c22: Int, c23: Int, c24: Int, + c25: Int, c26: Int) + class QueryExecutionSuite extends SharedSQLContext { + import testImplicits._ def checkDumpedPlans(path: String, expected: Int): Unit = { assert(Source.fromFile(path).getLines.toList .takeWhile(_ != "== Whole Stage Codegen ==") == List( @@ -80,6 +89,19 @@ class QueryExecutionSuite extends SharedSQLContext { assert(exception.getMessage.contains("Illegal character in scheme name")) } + test("check maximum fields restriction") { + withTempDir { dir => + val path = dir.getCanonicalPath + "/plans.txt" + val ds = spark.createDataset(Seq(QueryExecutionTestRecord( + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, + 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26))) + ds.queryExecution.debug.toFile(path) + val localRelations = Source.fromFile(path).getLines().filter(_.contains("LocalRelation")) + + assert(!localRelations.exists(_.contains("more fields"))) + } + } + test("toString() exception/error handling") { spark.experimental.extraStrategies = Seq( new SparkStrategy { From d63f862b74213509b0e432cba2ea8c514f2718b9 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 18 Sep 2018 15:06:57 +0200 Subject: [PATCH 22/41] Bug fix: passing maxFields to simpleString --- .../scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 4edd646eecacb..1a64ee1b9501b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -436,8 +436,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { def argString(maxFields: Option[Int]): String = stringArgs.flatMap { case tn: TreeNode[_] if allChildren.contains(tn) => Nil case Some(tn: TreeNode[_]) if allChildren.contains(tn) => Nil - case Some(tn: TreeNode[_]) => tn.simpleString() :: Nil - case tn: TreeNode[_] => tn.simpleString() :: Nil + case Some(tn: TreeNode[_]) => tn.simpleString(maxFields) :: Nil + case tn: TreeNode[_] => tn.simpleString(maxFields) :: Nil case seq: Seq[Any] if seq.toSet.subsetOf(allChildren.asInstanceOf[Set[Any]]) => Nil case iter: Iterable[_] if iter.isEmpty => Nil case seq: Seq[_] => Utils.truncatedString(seq, "[", ", ", "]", maxFields) :: Nil From 7fd88d3367b04ee8b9187db361094b6fae8c7729 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 24 Sep 2018 22:53:30 +0200 Subject: [PATCH 23/41] Passing parameters by names --- .../spark/sql/execution/WholeStageCodegenExec.scala | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index d40008fd984da..b3a918278e1d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -398,7 +398,14 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp prefix: String = "", addSuffix: Boolean = false, maxFields: Option[Int]): Unit = { - child.generateTreeString(depth, lastChildren, writer, verbose, "", false, maxFields) + child.generateTreeString( + depth, + lastChildren, + writer, + verbose, + prefix = "", + addSuffix = false, + maxFields) } override def needCopyResult: Boolean = false @@ -680,8 +687,8 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) lastChildren, writer, verbose, - s"*($codegenStageId) ", - false, + prefix = s"*($codegenStageId) ", + addSuffix = false, maxFields) } From 732707a9d1fc7b44515243f2122c0ceb78ca6db5 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 24 Sep 2018 22:56:35 +0200 Subject: [PATCH 24/41] Getting file system from file path --- .../scala/org/apache/spark/sql/execution/QueryExecution.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 61f68ffe18822..2e4458c0a4712 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -271,7 +271,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { */ def toFile(path: String): Unit = { val filePath = new Path(path) - val fs = FileSystem.get(filePath.toUri, sparkSession.sessionState.newHadoopConf()) + val fs = filePath.getFileSystem(sparkSession.sessionState.newHadoopConf()) val writer = new OutputStreamWriter(fs.create(filePath)) try { From 3a133aeeb05eb32bb04fc7ffeef2ef372aa3814c Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 24 Sep 2018 23:04:20 +0200 Subject: [PATCH 25/41] Using the buffered writer --- .../org/apache/spark/sql/execution/QueryExecution.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 2e4458c0a4712..96e411b8abbe4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -17,14 +17,13 @@ package org.apache.spark.sql.execution -import java.io.{OutputStreamWriter, Writer} +import java.io.{BufferedWriter, OutputStreamWriter, Writer} import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} import org.apache.commons.io.output.StringBuilderWriter -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.Path -import org.apache.spark.SparkEnv import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow @@ -272,7 +271,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { def toFile(path: String): Unit = { val filePath = new Path(path) val fs = filePath.getFileSystem(sparkSession.sessionState.newHadoopConf()) - val writer = new OutputStreamWriter(fs.create(filePath)) + val writer = new BufferedWriter(new OutputStreamWriter(fs.create(filePath))) try { writePlans(writer, Some(Int.MaxValue)) From 7452b8270e65b7f2ae82ed8f38845ea3ec4e5e9a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 24 Sep 2018 23:26:41 +0200 Subject: [PATCH 26/41] Removing default value for maxFields in simpleString --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 4 ++-- .../apache/spark/sql/catalyst/analysis/CheckAnalysis.scala | 7 ++++--- .../apache/spark/sql/catalyst/analysis/TypeCoercion.scala | 3 ++- .../spark/sql/catalyst/encoders/ExpressionEncoder.scala | 6 ++++-- .../apache/spark/sql/catalyst/expressions/generators.scala | 2 +- .../org/apache/spark/sql/catalyst/expressions/misc.scala | 4 ++-- .../org/apache/spark/sql/catalyst/trees/TreeNode.scala | 4 ++-- .../catalyst/expressions/aggregate/PercentileSuite.scala | 2 +- .../org/apache/spark/sql/execution/SparkPlanInfo.scala | 5 ++++- .../apache/spark/sql/execution/WholeStageCodegenExec.scala | 4 ++-- .../org/apache/spark/sql/execution/debug/package.scala | 2 +- .../scala/org/apache/spark/sql/execution/subquery.scala | 2 +- 12 files changed, 26 insertions(+), 19 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index db38298a8ba9d..5208b7fc04139 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -951,7 +951,7 @@ class Analyzer( case plan if containsDeserializer(plan.expressions) => plan case q: LogicalPlan => - logTrace(s"Attempting to resolve ${q.simpleString()}") + logTrace(s"Attempting to resolve ${q.simpleString(maxFields = None)}") q.mapExpressions(resolve(_, q)) } @@ -1745,7 +1745,7 @@ class Analyzer( case p if p.expressions.exists(hasGenerator) => throw new AnalysisException("Generators are not supported outside the SELECT clause, but " + - "got: " + p.simpleString()) + "got: " + p.simpleString(maxFields = None)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index a2fea116ab2c8..703c035a8624e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -308,7 +308,7 @@ trait CheckAnalysis extends PredicateHelper { val missingAttributes = o.missingInput.mkString(",") val input = o.inputSet.mkString(",") val msgForMissingAttributes = s"Resolved attribute(s) $missingAttributes missing " + - s"from $input in operator ${operator.simpleString()}." + s"from $input in operator ${operator.simpleString(maxFields = None)}." val resolver = plan.conf.resolver val attrsWithSameName = o.missingInput.filter { missing => @@ -373,7 +373,7 @@ trait CheckAnalysis extends PredicateHelper { s"""nondeterministic expressions are only allowed in |Project, Filter, Aggregate or Window, found: | ${o.expressions.map(_.sql).mkString(",")} - |in operator ${operator.simpleString()} + |in operator ${operator.simpleString(maxFields = None)} """.stripMargin) case _: UnresolvedHint => @@ -385,7 +385,8 @@ trait CheckAnalysis extends PredicateHelper { } extendedCheckRules.foreach(_(plan)) plan.foreachUp { - case o if !o.resolved => failAnalysis(s"unresolved operator ${o.simpleString()}") + case o if !o.resolved => + failAnalysis(s"unresolved operator ${o.simpleString(maxFields = None)}") case _ => } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 5186d39f87462..9dc485ea70906 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -1019,7 +1019,8 @@ trait TypeCoercionRule extends Rule[LogicalPlan] with Logging { case Some(newType) if a.dataType == newType.dataType => a case Some(newType) => logDebug( - s"Promoting $a from ${a.dataType} to ${newType.dataType} in ${q.simpleString()}") + s"Promoting $a from ${a.dataType} to ${newType.dataType} in " + + q.simpleString(maxFields = None)) newType } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index 7b328ff28782c..b18ec2fc2698b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -289,8 +289,9 @@ case class ExpressionEncoder[T]( extractProjection(inputRow) } catch { case e: Exception => + val encoded = serializer.map(_.simpleString(maxFields = None)).mkString("\n") throw new RuntimeException( - s"Error while encoding: $e\n${serializer.map(_.simpleString()).mkString("\n")}", e) + s"Error while encoding: $e\n${encoded}", e) } /** @@ -302,7 +303,8 @@ case class ExpressionEncoder[T]( constructProjection(row).get(0, ObjectType(clsTag.runtimeClass)).asInstanceOf[T] } catch { case e: Exception => - throw new RuntimeException(s"Error while decoding: $e\n${deserializer.simpleString()}", e) + val decoded = deserializer.simpleString(maxFields = None) + throw new RuntimeException(s"Error while decoding: $e\n${decoded}", e) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 9899a91c90285..a14c417b7e7f3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -101,7 +101,7 @@ case class UserDefinedGenerator( inputRow = new InterpretedProjection(children) convertToScala = { val inputSchema = StructType(children.map { e => - StructField(e.simpleString(), e.dataType, nullable = true) + StructField(e.simpleString(maxFields = None), e.dataType, nullable = true) }) CatalystTypeConverters.createToScalaConverter(inputSchema) }.asInstanceOf[InternalRow => Row] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index 2a650dccacb83..5b46f4b937cab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -40,7 +40,7 @@ case class PrintToStderr(child: Expression) extends UnaryExpression { input } - private val outputPrefix = s"Result of ${child.simpleString()} is " + private val outputPrefix = s"Result of ${child.simpleString(maxFields = None)} is " override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val outputPrefixField = ctx.addReferenceObj("outputPrefix", outputPrefix) @@ -72,7 +72,7 @@ case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCa override def prettyName: String = "assert_true" - private val errMsg = s"'${child.simpleString()}' is not true!" + private val errMsg = s"'${child.simpleString(maxFields = None)}' is not true!" override def eval(input: InternalRow) : Any = { val v = child.eval(input) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 1a64ee1b9501b..8f8b157f02c3e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -457,7 +457,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { }.mkString(", ") /** ONE line description of this node. */ - def simpleString(maxFields: Option[Int] = None): String = { + def simpleString(maxFields: Option[Int]): String = { s"$nodeName ${argString(maxFields)}".trim } @@ -566,7 +566,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { val str = if (verbose) { if (addSuffix) verboseStringWithSuffix(maxFields) else verboseString(maxFields) } else { - simpleString() + simpleString(maxFields) } writer.write(prefix) writer.write(str) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala index 40cabc3d3a01a..57e783f42f7b4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala @@ -215,7 +215,7 @@ class PercentileSuite extends SparkFunSuite { val percentile2 = new Percentile(child, percentage) assertEqual(percentile2.checkInputDataTypes(), TypeCheckFailure(s"Percentage(s) must be between 0.0 and 1.0, " + - s"but got ${percentage.simpleString()}")) + s"but got ${percentage.simpleString(None)}")) } val nonFoldablePercentage = Seq(NonFoldableLiteral(0.5), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index 53df9b5b19746..ad2c023bd32ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -62,7 +62,10 @@ private[execution] object SparkPlanInfo { case fileScan: FileSourceScanExec => fileScan.metadata case _ => Map[String, String]() } - new SparkPlanInfo(plan.nodeName, plan.simpleString(), children.map(fromSparkPlan), + new SparkPlanInfo( + plan.nodeName, + plan.simpleString(maxFields = None), + children.map(fromSparkPlan), metadata, metrics) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index b3a918278e1d6..9a121e03521d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -87,7 +87,7 @@ trait CodegenSupport extends SparkPlan { this.parent = parent ctx.freshNamePrefix = variablePrefix s""" - |${ctx.registerComment(s"PRODUCE: ${this.simpleString()}")} + |${ctx.registerComment(s"PRODUCE: ${this.simpleString(maxFields = None)}")} |${doProduce(ctx)} """.stripMargin } @@ -185,7 +185,7 @@ trait CodegenSupport extends SparkPlan { parent.doConsume(ctx, inputVars, rowVar) } s""" - |${ctx.registerComment(s"CONSUME: ${parent.simpleString()}")} + |${ctx.registerComment(s"CONSUME: ${parent.simpleString(maxFields = None)}")} |$evaluated |$consumeFunc """.stripMargin diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index 0c5e5333e3c7f..905f89efe2c9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -216,7 +216,7 @@ package object debug { val columnStats: Array[ColumnMetrics] = Array.fill(child.output.size)(new ColumnMetrics()) def dumpStats(): Unit = { - debugPrint(s"== ${child.simpleString()} ==") + debugPrint(s"== ${child.simpleString(maxFields = None)} ==") debugPrint(s"Tuples output: ${tupleCount.value}") child.output.zip(columnStats).foreach { case (attr, metric) => // This is called on driver. All accumulator updates have a fixed value. So it's safe to use diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index f39acd435175a..d80e97b644c8d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -51,7 +51,7 @@ case class ScalarSubquery( override def dataType: DataType = plan.schema.fields.head.dataType override def children: Seq[Expression] = Nil override def nullable: Boolean = true - override def toString: String = plan.simpleString() + override def toString: String = plan.simpleString(maxFields = None) override def withNewPlan(query: SubqueryExec): ScalarSubquery = copy(plan = query) override def semanticEquals(other: Expression): Boolean = other match { From 4ec57326612adb2d650e425d03eae70366480242 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 24 Sep 2018 23:37:55 +0200 Subject: [PATCH 27/41] Removing unnecessary signature of truncatedString --- .../scala/org/apache/spark/util/Utils.scala | 17 +++-------------- .../org/apache/spark/util/UtilsSuite.scala | 10 +++++----- .../sql/catalyst/expressions/Expression.scala | 2 +- .../spark/sql/catalyst/trees/TreeNode.scala | 2 +- .../org/apache/spark/sql/types/StructType.scala | 2 +- .../spark/sql/execution/ExistingRDD.scala | 2 +- .../datasources/v2/DataSourceV2Relation.scala | 6 ++++-- .../datasources/v2/DataSourceV2ScanExec.scala | 4 +++- .../v2/DataSourceV2StringFormat.scala | 6 +++--- .../streaming/MicroBatchExecution.scala | 4 ++-- .../continuous/ContinuousExecution.scala | 4 ++-- .../spark/sql/execution/streaming/memory.scala | 4 +++- 12 files changed, 29 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 6d453019d4782..c8efcd47a2bf1 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -122,12 +122,13 @@ private[spark] object Utils extends Logging { start: String, sep: String, end: String, - maxNumFields: Int = maxNumToStringFields): String = { + maxFields: Option[Int]): String = { + val maxNumFields = maxFields.getOrElse(maxNumToStringFields) if (seq.length > maxNumFields) { if (truncationWarningPrinted.compareAndSet(false, true)) { logWarning( "Truncated the string representation of a plan since it was too large. This " + - "behavior can be adjusted by setting 'spark.debug.maxToStringFields' in SparkEnv.conf.") + "behavior can be adjusted by setting 'spark.debug.maxToStringFields' in SparkEnv.conf.") } val numFields = math.max(0, maxNumFields - 1) seq.take(numFields).mkString( @@ -137,19 +138,7 @@ private[spark] object Utils extends Logging { } } - def truncatedString[T]( - seq: Seq[T], - start: String, - sep: String, - end: String, - maxFields: Option[Int]): String = { - val maxNumFields = maxFields.getOrElse(maxNumToStringFields) - truncatedString(seq, start, sep, end, maxNumFields) - } - /** Shorthand for calling truncatedString() without start or end strings. */ - def truncatedString[T](seq: Seq[T], sep: String): String = truncatedString(seq, "", sep, "") - def truncatedString[T](seq: Seq[T], sep: String, maxFields: Option[Int]): String = { truncatedString(seq, "", sep, "", maxFields) } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 39f4fba78583f..3a7e347d2edd7 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -46,11 +46,11 @@ import org.apache.spark.scheduler.SparkListener class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { test("truncatedString") { - assert(Utils.truncatedString(Nil, "[", ", ", "]", 2) == "[]") - assert(Utils.truncatedString(Seq(1, 2), "[", ", ", "]", 2) == "[1, 2]") - assert(Utils.truncatedString(Seq(1, 2, 3), "[", ", ", "]", 2) == "[1, ... 2 more fields]") - assert(Utils.truncatedString(Seq(1, 2, 3), "[", ", ", "]", -5) == "[, ... 3 more fields]") - assert(Utils.truncatedString(Seq(1, 2, 3), ", ") == "1, 2, 3") + assert(Utils.truncatedString(Nil, "[", ", ", "]", Some(2)) == "[]") + assert(Utils.truncatedString(Seq(1, 2), "[", ", ", "]", Some(2)) == "[1, 2]") + assert(Utils.truncatedString(Seq(1, 2, 3), "[", ", ", "]", Some(2)) == "[1, ... 2 more fields]") + assert(Utils.truncatedString(Seq(1, 2, 3), "[", ", ", "]", Some(-5)) == "[, ... 3 more fields]") + assert(Utils.truncatedString(Seq(1, 2, 3), ", ", maxFields = None) == "1, 2, 3") } test("timeConversion") { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index ba2b0fc8ad9b7..edf0464c58e69 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -235,7 +235,7 @@ abstract class Expression extends TreeNode[Expression] { override def simpleString(maxFields: Option[Int]): String = toString override def toString: String = prettyName + Utils.truncatedString( - flatArguments.toSeq, "(", ", ", ")") + flatArguments.toSeq, "(", ", ", ")", maxFields = None) /** * Returns SQL representation of this expression. For expressions extending [[NonSQLExpression]], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 8f8b157f02c3e..738a8c0b8e5b1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -668,7 +668,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { t.forall(_.isInstanceOf[Partitioning]) || t.forall(_.isInstanceOf[DataType]) => JArray(t.map(parseToJson).toList) case t: Seq[_] if t.length > 0 && t.head.isInstanceOf[String] => - JString(Utils.truncatedString(t, "[", ", ", "]")) + JString(Utils.truncatedString(t, "[", ", ", "]", maxFields = None)) case t: Seq[_] => JNull case m: Map[_, _] => JNull // if it's a scala object, we can simply keep the full class path. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index 06289b1483203..a26ec1046768e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -346,7 +346,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru override def simpleString: String = { val fieldTypes = fields.view.map(field => s"${field.name}:${field.dataType.simpleString}") - Utils.truncatedString(fieldTypes, "struct<", ",", ">") + Utils.truncatedString(fieldTypes, "struct<", ",", ">", maxFields = None) } override def catalogString: String = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 5153bf0b0305a..180b860b710e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -197,6 +197,6 @@ case class RDDScanExec( } override def simpleString(maxFields: Option[Int]): String = { - s"$nodeName${Utils.truncatedString(output, "[", ",", "]")}" + s"$nodeName${Utils.truncatedString(output, "[", ",", "]", maxFields)}" } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index 1b9ae9e326d85..1026225c64920 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -56,7 +56,9 @@ case class DataSourceV2Relation( override def pushedFilters: Seq[Expression] = Seq.empty - override def simpleString(maxFields: Option[Int]): String = "RelationV2 " + metadataString + override def simpleString(maxFields: Option[Int]): String = { + "RelationV2 " + metadataString(maxFields) + } def newWriteSupport(): BatchWriteSupport = source.createWriteSupport(options, schema) @@ -91,7 +93,7 @@ case class StreamingDataSourceV2Relation( override def isStreaming: Boolean = true override def simpleString(maxFields: Option[Int]): String = { - "Streaming RelationV2 " + metadataString + "Streaming RelationV2 " + metadataString(maxFields) } override def pushedFilters: Seq[Expression] = Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala index 5474c11ee3aec..598a0b7926b2d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala @@ -40,7 +40,9 @@ case class DataSourceV2ScanExec( @transient scanConfig: ScanConfig) extends LeafExecNode with DataSourceV2StringFormat with ColumnarBatchScan { - override def simpleString(maxFields: Option[Int]): String = "ScanV2 " + metadataString + override def simpleString(maxFields: Option[Int]): String = { + "ScanV2 " + metadataString(maxFields) + } // TODO: unify the equal/hashCode implementation for all data source v2 query plans. override def equals(other: Any): Boolean = other match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StringFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StringFormat.scala index 97e6c6d702acb..8328b3d75c069 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StringFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StringFormat.scala @@ -58,7 +58,7 @@ trait DataSourceV2StringFormat { case _ => Utils.getSimpleName(source.getClass) } - def metadataString: String = { + def metadataString(maxFields: Option[Int]): String = { val entries = scala.collection.mutable.ArrayBuffer.empty[(String, String)] if (pushedFilters.nonEmpty) { @@ -72,12 +72,12 @@ trait DataSourceV2StringFormat { }.mkString("[", ",", "]") } - val outputStr = Utils.truncatedString(output, "[", ", ", "]") + val outputStr = Utils.truncatedString(output, "[", ", ", "]", maxFields) val entriesStr = if (entries.nonEmpty) { Utils.truncatedString(entries.map { case (key, value) => key + ": " + StringUtils.abbreviate(value, 100) - }, " (", ", ", ")") + }, " (", ", ", ")", maxFields) } else { "" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index 2cac86599ef19..1db8224f4eb02 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -475,8 +475,8 @@ class MicroBatchExecution( case StreamingExecutionRelation(source, output) => newData.get(source).map { dataPlan => assert(output.size == dataPlan.output.size, - s"Invalid batch: ${Utils.truncatedString(output, ",")} != " + - s"${Utils.truncatedString(dataPlan.output, ",")}") + s"Invalid batch: ${Utils.truncatedString(output, ",", maxFields = None)} != " + + s"${Utils.truncatedString(dataPlan.output, ",", maxFields = None)}") val aliases = output.zip(dataPlan.output).map { case (to, from) => Alias(from, to.name)(exprId = to.exprId, explicitMetadata = Some(from.metadata)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index f009c52449adc..57587d16f93f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -164,8 +164,8 @@ class ContinuousExecution( val newOutput = readSupport.fullSchema().toAttributes assert(output.size == newOutput.size, - s"Invalid reader: ${Utils.truncatedString(output, ",")} != " + - s"${Utils.truncatedString(newOutput, ",")}") + s"Invalid reader: ${Utils.truncatedString(output, ",", maxFields = None)} != " + + s"${Utils.truncatedString(newOutput, ",", maxFields = None)}") replacements ++= output.zip(newOutput) val loggedOffset = offsets.offsets(0) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index adf52aba21a04..304d6557fbf28 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -117,7 +117,9 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) } } - override def toString: String = s"MemoryStream[${Utils.truncatedString(output, ",")}]" + override def toString: String = { + s"MemoryStream[${Utils.truncatedString(output, ",", maxFields = None)}]" + } override def deserializeOffset(json: String): OffsetV2 = LongOffset(json.toLong) From be161756a4d85cecc7bb8d81940e371e45cac151 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 25 Sep 2018 11:47:44 +0200 Subject: [PATCH 28/41] Minor improvement - passing maxFields by name --- .../sql/catalyst/expressions/aggregate/PercentileSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala index 57e783f42f7b4..d16e5aa92c745 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala @@ -215,7 +215,7 @@ class PercentileSuite extends SparkFunSuite { val percentile2 = new Percentile(child, percentage) assertEqual(percentile2.checkInputDataTypes(), TypeCheckFailure(s"Percentage(s) must be between 0.0 and 1.0, " + - s"but got ${percentage.simpleString(None)}")) + s"but got ${percentage.simpleString(maxFields = None)}")) } val nonFoldablePercentage = Seq(NonFoldableLiteral(0.5), From 90ff7b543967d8f98eaa35dd456ef43d3866097f Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 25 Sep 2018 13:31:02 +0200 Subject: [PATCH 29/41] Moving truncatedString out of core --- .../scala/org/apache/spark/util/Utils.scala | 51 ----------------- .../org/apache/spark/util/UtilsSuite.scala | 8 --- .../sql/catalyst/expressions/Expression.scala | 4 +- .../plans/logical/basicLogicalOperators.scala | 4 +- .../spark/sql/catalyst/trees/TreeNode.scala | 10 ++-- .../spark/sql/catalyst/util/package.scala | 55 ++++++++++++++++++- .../apache/spark/sql/types/StructType.scala | 5 +- .../apache/spark/sql/util/UtilsSuite.scala | 32 +++++++++++ .../sql/execution/DataSourceScanExec.scala | 5 +- .../spark/sql/execution/ExistingRDD.scala | 4 +- .../spark/sql/execution/QueryExecution.scala | 4 +- .../aggregate/HashAggregateExec.scala | 7 ++- .../aggregate/ObjectHashAggregateExec.scala | 8 +-- .../aggregate/SortAggregateExec.scala | 8 +-- .../execution/columnar/InMemoryRelation.scala | 5 +- .../datasources/LogicalRelation.scala | 3 +- .../datasources/jdbc/JDBCRelation.scala | 4 +- .../v2/DataSourceV2StringFormat.scala | 5 +- .../apache/spark/sql/execution/limit.scala | 6 +- .../streaming/MicroBatchExecution.scala | 7 ++- .../continuous/ContinuousExecution.scala | 7 ++- .../sql/execution/streaming/memory.scala | 4 +- 22 files changed, 139 insertions(+), 107 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/util/UtilsSuite.scala diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index c8efcd47a2bf1..68819bbf5828f 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -93,57 +93,6 @@ private[spark] object Utils extends Logging { private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 @volatile private var localRootDirs: Array[String] = null - /** - * 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) - } else { - DEFAULT_MAX_TO_STRING_FIELDS - } - } - - /** 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) - if (seq.length > maxNumFields) { - if (truncationWarningPrinted.compareAndSet(false, true)) { - logWarning( - "Truncated the string representation of a plan since it was too large. This " + - "behavior can be adjusted by setting 'spark.debug.maxToStringFields' in SparkEnv.conf.") - } - val numFields = math.max(0, maxNumFields - 1) - seq.take(numFields).mkString( - start, sep, sep + "... " + (seq.length - numFields) + " more fields" + end) - } else { - seq.mkString(start, sep, end) - } - } - - /** Shorthand for calling truncatedString() without start or end strings. */ - def truncatedString[T](seq: Seq[T], sep: String, maxFields: Option[Int]): String = { - truncatedString(seq, "", sep, "", maxFields) - } - - /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 3a7e347d2edd7..534b33be7efc7 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -45,14 +45,6 @@ import org.apache.spark.scheduler.SparkListener class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { - test("truncatedString") { - assert(Utils.truncatedString(Nil, "[", ", ", "]", Some(2)) == "[]") - assert(Utils.truncatedString(Seq(1, 2), "[", ", ", "]", Some(2)) == "[1, 2]") - assert(Utils.truncatedString(Seq(1, 2, 3), "[", ", ", "]", Some(2)) == "[1, ... 2 more fields]") - assert(Utils.truncatedString(Seq(1, 2, 3), "[", ", ", "]", Some(-5)) == "[, ... 3 more fields]") - assert(Utils.truncatedString(Seq(1, 2, 3), ", ", maxFields = None) == "1, 2, 3") - } - test("timeConversion") { // Test -1 assert(Utils.timeStringAsSeconds("-1") === -1) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index edf0464c58e69..82b481428ec5d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -24,8 +24,8 @@ import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.types._ -import org.apache.spark.util.Utils //////////////////////////////////////////////////////////////////////////////////////////////////// // This file defines the basic expression abstract classes in Catalyst. @@ -234,7 +234,7 @@ abstract class Expression extends TreeNode[Expression] { override def simpleString(maxFields: Option[Int]): String = toString - override def toString: String = prettyName + Utils.truncatedString( + override def toString: String = prettyName + truncatedString( flatArguments.toSeq, "(", ", ", ")", maxFields = None) /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index d5ec24a5b0830..70342f1ce764f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -24,8 +24,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.types._ -import org.apache.spark.util.Utils import org.apache.spark.util.random.RandomSampler /** @@ -485,7 +485,7 @@ case class With(child: LogicalPlan, cteRelations: Seq[(String, SubqueryAlias)]) override def output: Seq[Attribute] = child.output override def simpleString(maxFields: Option[Int]): String = { - val cteAliases = Utils.truncatedString(cteRelations.map(_._1), "[", ", ", "]", maxFields) + val cteAliases = truncatedString(cteRelations.map(_._1), "[", ", ", "]", maxFields) s"CTE $cteAliases" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 738a8c0b8e5b1..7b2b8d55d3543 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -37,9 +37,9 @@ import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning} +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.Utils /** Used by [[TreeNode.getNodeNumbered]] when traversing the tree for a given number */ private class MutableInt(var i: Int) @@ -440,10 +440,10 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { case tn: TreeNode[_] => tn.simpleString(maxFields) :: Nil case seq: Seq[Any] if seq.toSet.subsetOf(allChildren.asInstanceOf[Set[Any]]) => Nil case iter: Iterable[_] if iter.isEmpty => Nil - case seq: Seq[_] => Utils.truncatedString(seq, "[", ", ", "]", maxFields) :: Nil - case set: Set[_] => Utils.truncatedString(set.toSeq, "{", ", ", "}", maxFields) :: Nil + case seq: Seq[_] => truncatedString(seq, "[", ", ", "]", maxFields) :: Nil + case set: Set[_] => truncatedString(set.toSeq, "{", ", ", "}", maxFields) :: Nil case array: Array[_] if array.isEmpty => Nil - case array: Array[_] => Utils.truncatedString(array, "[", ", ", "]", maxFields) :: Nil + case array: Array[_] => truncatedString(array, "[", ", ", "]", maxFields) :: Nil case null => Nil case None => Nil case Some(null) => Nil @@ -668,7 +668,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { t.forall(_.isInstanceOf[Partitioning]) || t.forall(_.isInstanceOf[DataType]) => JArray(t.map(parseToJson).toList) case t: Seq[_] if t.length > 0 && t.head.isInstanceOf[String] => - JString(Utils.truncatedString(t, "[", ", ", "]", maxFields = None)) + JString(truncatedString(t, "[", ", ", "]", maxFields = None)) case t: Seq[_] => JNull case m: Map[_, _] => JNull // if it's a scala object, we can simply keep the full class path. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index 0978e92dd4f72..315a2d74c92ce 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -19,13 +19,16 @@ package org.apache.spark.sql.catalyst import java.io._ import java.nio.charset.StandardCharsets +import java.util.concurrent.atomic.AtomicBoolean +import org.apache.spark.SparkEnv +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{NumericType, StringType} import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils -package object util { +package object util extends Logging { /** Silences output to stderr or stdout for the duration of f */ def quietly[A](f: => A): A = { @@ -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) + } else { + DEFAULT_MAX_TO_STRING_FIELDS + } + } + + /** 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) + if (seq.length > maxNumFields) { + if (truncationWarningPrinted.compareAndSet(false, true)) { + logWarning( + "Truncated the string representation of a plan since it was too large. This " + + "behavior can be adjusted by setting 'spark.debug.maxToStringFields' in SparkEnv.conf.") + } + val numFields = math.max(0, maxNumFields - 1) + seq.take(numFields).mkString( + start, sep, sep + "... " + (seq.length - numFields) + " more fields" + end) + } else { + seq.mkString(start, sep, end) + } + } + + /** Shorthand for calling truncatedString() without start or end strings. */ + def truncatedString[T](seq: Seq[T], sep: String, maxFields: Option[Int]): String = { + truncatedString(seq, "", sep, "", maxFields) + } + /* FIX ME implicit class debugLogging(a: Any) { def debugLogging() { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index a26ec1046768e..d3312914d7539 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -27,8 +27,7 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, LegacyTypeStringParser} -import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIdentifier} -import org.apache.spark.util.Utils +import org.apache.spark.sql.catalyst.util.{quoteIdentifier, truncatedString} /** * A [[StructType]] object can be constructed by @@ -346,7 +345,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru override def simpleString: String = { val fieldTypes = fields.view.map(field => s"${field.name}:${field.dataType.simpleString}") - Utils.truncatedString(fieldTypes, "struct<", ",", ">", maxFields = None) + truncatedString(fieldTypes, "struct<", ",", ">", maxFields = None) } override def catalogString: String = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/UtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/UtilsSuite.scala new file mode 100644 index 0000000000000..58cd20387ebe3 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/UtilsSuite.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.util + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.util._ + +class UtilsSuite extends SparkFunSuite { + + test("truncatedString") { + assert(truncatedString(Nil, "[", ", ", "]", Some(2)) == "[]") + assert(truncatedString(Seq(1, 2), "[", ", ", "]", Some(2)) == "[1, 2]") + assert(truncatedString(Seq(1, 2, 3), "[", ", ", "]", Some(2)) == "[1, ... 2 more fields]") + assert(truncatedString(Seq(1, 2, 3), "[", ", ", "]", Some(-5)) == "[, ... 3 more fields]") + assert(truncatedString(Seq(1, 2, 3), ", ", maxFields = None) == "1, 2, 3") + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 70c66d0e1f270..5b5e92383794a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource} import org.apache.spark.sql.execution.metric.SQLMetrics @@ -56,8 +57,8 @@ trait DataSourceScanExec extends LeafExecNode with CodegenSupport { case (key, value) => key + ": " + StringUtils.abbreviate(redact(value), 100) } - val metadataStr = Utils.truncatedString(metadataEntries, " ", ", ", "", maxFields) - val outputStr = Utils.truncatedString(output, "[", ",", "]", maxFields) + val metadataStr = truncatedString(metadataEntries, " ", ", ", "", maxFields) + val outputStr = truncatedString(output, "[", ",", "]", maxFields) s"$nodeNamePrefix$nodeName${outputStr}$metadataStr" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 180b860b710e8..c4e735f08430d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -24,9 +24,9 @@ import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.DataType -import org.apache.spark.util.Utils object RDDConversions { def productToRowRdd[A <: Product](data: RDD[A], outputTypes: Seq[DataType]): RDD[InternalRow] = { @@ -197,6 +197,6 @@ case class RDDScanExec( } override def simpleString(maxFields: Option[Int]): String = { - s"$nodeName${Utils.truncatedString(output, "[", ",", "]", maxFields)}" + s"$nodeName${truncatedString(output, "[", ",", "]", maxFields)}" } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 96e411b8abbe4..2f5311cc3f126 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{truncatedString, DateTimeUtils} import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExecutedCommandExec, ShowTablesCommand} import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} import org.apache.spark.sql.types.{BinaryType, DateType, DecimalType, TimestampType, _} @@ -201,7 +201,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { writer.write("== Parsed Logical Plan ==\n") writeOrError(writer)(logical.treeString(_, verbose, addSuffix, maxFields)) writer.write("\n== Analyzed Logical Plan ==\n") - val analyzedOutput = stringOrError(Utils.truncatedString( + val analyzedOutput = stringOrError(truncatedString( analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}"), ", ", maxFields)) writer.write(analyzedOutput) writer.write("\n") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index a3397fb5557fb..f16e51d4097c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.vectorized.MutableColumnarRow @@ -924,10 +925,10 @@ case class HashAggregateExec( testFallbackStartsAt match { case None => - val keyString = Utils.truncatedString(groupingExpressions, "[", ", ", "]", maxFields) + val keyString = truncatedString(groupingExpressions, "[", ", ", "]", maxFields) val functionString = - Utils.truncatedString(allAggregateExpressions, "[", ", ", "]", maxFields) - val outputString = Utils.truncatedString(output, "[", ", ", "]", maxFields) + truncatedString(allAggregateExpressions, "[", ", ", "]", maxFields) + val outputString = truncatedString(output, "[", ", ", "]", maxFields) if (verbose) { s"HashAggregate(keys=$keyString, functions=$functionString, output=$outputString)" } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala index 83b0fe8f8f110..00b89554f57df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala @@ -23,9 +23,9 @@ import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.util.Utils /** * A hash-based aggregate operator that supports [[TypedImperativeAggregate]] functions that may @@ -143,9 +143,9 @@ case class ObjectHashAggregateExec( private def toString(verbose: Boolean, maxFields: Option[Int]): String = { val allAggregateExpressions = aggregateExpressions - val keyString = Utils.truncatedString(groupingExpressions, "[", ", ", "]", maxFields) - val functionString = Utils.truncatedString(allAggregateExpressions, "[", ", ", "]", maxFields) - val outputString = Utils.truncatedString(output, "[", ", ", "]", maxFields) + val keyString = truncatedString(groupingExpressions, "[", ", ", "]", maxFields) + val functionString = truncatedString(allAggregateExpressions, "[", ", ", "]", maxFields) + val outputString = truncatedString(output, "[", ", ", "]", maxFields) if (verbose) { s"ObjectHashAggregate(keys=$keyString, functions=$functionString, output=$outputString)" } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala index 8de14cee7e94f..02c477b76299a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala @@ -23,9 +23,9 @@ import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.util.Utils /** * Sort-based aggregate operator. @@ -114,9 +114,9 @@ case class SortAggregateExec( private def toString(verbose: Boolean, maxFields: Option[Int]): String = { val allAggregateExpressions = aggregateExpressions - val keyString = Utils.truncatedString(groupingExpressions, "[", ", ", "]", maxFields) - val functionString = Utils.truncatedString(allAggregateExpressions, "[", ", ", "]", maxFields) - val outputString = Utils.truncatedString(output, "[", ", ", "]", maxFields) + val keyString = truncatedString(groupingExpressions, "[", ", ", "]", maxFields) + val functionString = truncatedString(allAggregateExpressions, "[", ", ", "]", maxFields) + val outputString = truncatedString(output, "[", ", ", "]", maxFields) if (verbose) { s"SortAggregate(key=$keyString, functions=$functionString, output=$outputString)" } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 33d1a1ef5d55f..566168a24b65a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -27,9 +27,10 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.{HintInfo, LogicalPlan, Statistics} +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{LongAccumulator, Utils} +import org.apache.spark.util.LongAccumulator /** @@ -209,7 +210,7 @@ case class InMemoryRelation( override protected def otherCopyArgs: Seq[AnyRef] = Seq(statsOfPlanToCache) override def simpleString(maxFields: Option[Int]): String = { - val outputStr = Utils.truncatedString(output, ", ", maxFields) + val outputStr = truncatedString(output, ", ", maxFields) s"InMemoryRelation [${outputStr}], ${cacheBuilder.storageLevel}" } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index 1527440f3cb6d..0a20f23b758fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.{AttributeMap, AttributeReference} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.util.Utils @@ -64,7 +65,7 @@ case class LogicalRelation( } override def simpleString(maxFields: Option[Int]): String = { - s"Relation[${Utils.truncatedString(output, ",", maxFields)}] $relation" + s"Relation[${truncatedString(output, ",", maxFields)}] $relation" } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index f15014442e3fb..93746de45ae5a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -27,10 +27,10 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SaveMode, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.maxNumToStringFields import org.apache.spark.sql.jdbc.JdbcDialects import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, DateType, NumericType, StructType, TimestampType} -import org.apache.spark.util.Utils /** * Instructions on how to partition the table among workers. @@ -160,7 +160,7 @@ private[sql] object JDBCRelation extends Logging { resolver(f.name, columnName) || resolver(dialect.quoteIdentifier(f.name), columnName) }.getOrElse { throw new AnalysisException(s"User-defined partition column $columnName not " + - s"found in the JDBC relation: ${schema.simpleString(Utils.maxNumToStringFields)}") + s"found in the JDBC relation: ${schema.simpleString(maxNumToStringFields)}") } column.dataType match { case _: NumericType | DateType | TimestampType => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StringFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StringFormat.scala index 8328b3d75c069..d2ea08f7ee9e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StringFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StringFormat.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.commons.lang3.StringUtils import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.sources.v2.DataSourceV2 import org.apache.spark.util.Utils @@ -72,10 +73,10 @@ trait DataSourceV2StringFormat { }.mkString("[", ",", "]") } - val outputStr = Utils.truncatedString(output, "[", ", ", "]", maxFields) + val outputStr = truncatedString(output, "[", ", ", "]", maxFields) val entriesStr = if (entries.nonEmpty) { - Utils.truncatedString(entries.map { + truncatedString(entries.map { case (key, value) => key + ": " + StringUtils.abbreviate(value, 100) }, " (", ", ", ")", maxFields) } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index 57d9439707cef..02b3a56701901 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -23,8 +23,8 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, LazilyGeneratedOrdering} import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec -import org.apache.spark.util.Utils /** * Take the first `limit` elements and collect them to a single partition. @@ -170,8 +170,8 @@ case class TakeOrderedAndProjectExec( override def outputPartitioning: Partitioning = SinglePartition override def simpleString(maxFields: Option[Int]): String = { - val orderByString = Utils.truncatedString(sortOrder, "[", ",", "]", maxFields) - val outputString = Utils.truncatedString(output, "[", ",", "]", maxFields) + val orderByString = truncatedString(sortOrder, "[", ",", "]", maxFields) + val outputString = truncatedString(output, "[", ",", "]", maxFields) s"TakeOrderedAndProject(limit=$limit, orderBy=$orderByString, output=$outputString)" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index 1db8224f4eb02..42e96a60ad17c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -24,13 +24,14 @@ import org.apache.spark.sql.{Dataset, SparkSession} import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Alias, CurrentBatchTimestamp, CurrentDate, CurrentTimestamp} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relation, WriteToDataSourceV2} import org.apache.spark.sql.execution.streaming.sources.{MicroBatchWritSupport, RateControlMicroBatchReadSupport} import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset => OffsetV2} import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger} -import org.apache.spark.util.{Clock, Utils} +import org.apache.spark.util.Clock class MicroBatchExecution( sparkSession: SparkSession, @@ -475,8 +476,8 @@ class MicroBatchExecution( case StreamingExecutionRelation(source, output) => newData.get(source).map { dataPlan => assert(output.size == dataPlan.output.size, - s"Invalid batch: ${Utils.truncatedString(output, ",", maxFields = None)} != " + - s"${Utils.truncatedString(dataPlan.output, ",", maxFields = None)}") + s"Invalid batch: ${truncatedString(output, ",", maxFields = None)} != " + + s"${truncatedString(dataPlan.output, ",", maxFields = None)}") val aliases = output.zip(dataPlan.output).map { case (to, from) => Alias(from, to.name)(exprId = to.exprId, explicitMetadata = Some(from.metadata)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index 57587d16f93f9..4c073777b1137 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -28,6 +28,7 @@ import org.apache.spark.SparkEnv import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, CurrentDate, CurrentTimestamp} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanExec, StreamingDataSourceV2Relation} import org.apache.spark.sql.execution.streaming.{ContinuousExecutionRelation, StreamingRelationV2, _} @@ -35,7 +36,7 @@ import org.apache.spark.sql.sources.v2 import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceOptions, StreamingWriteSupportProvider} import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, PartitionOffset} import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger} -import org.apache.spark.util.{Clock, Utils} +import org.apache.spark.util.Clock class ContinuousExecution( sparkSession: SparkSession, @@ -164,8 +165,8 @@ class ContinuousExecution( val newOutput = readSupport.fullSchema().toAttributes assert(output.size == newOutput.size, - s"Invalid reader: ${Utils.truncatedString(output, ",", maxFields = None)} != " + - s"${Utils.truncatedString(newOutput, ",", maxFields = None)}") + s"Invalid reader: ${truncatedString(output, ",", maxFields = None)} != " + + s"${truncatedString(newOutput, ",", maxFields = None)}") replacements ++= output.zip(newOutput) val loggedOffset = offsets.offsets(0) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index 304d6557fbf28..79918379fb700 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -31,11 +31,11 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeRow} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset => OffsetV2} import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType -import org.apache.spark.util.Utils object MemoryStream { protected val currentBlockId = new AtomicInteger(0) @@ -118,7 +118,7 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) } override def toString: String = { - s"MemoryStream[${Utils.truncatedString(output, ",", maxFields = None)}]" + s"MemoryStream[${truncatedString(output, ",", maxFields = None)}]" } override def deserializeOffset(json: String): OffsetV2 = LongOffset(json.toLong) From 1fcfc23e308d2488e7d300486fca9d97758e7337 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 27 Sep 2018 22:13:07 +0200 Subject: [PATCH 30/41] Adding SQL config to control maximum number of fields --- .../spark/sql/catalyst/util/package.scala | 6 +- .../apache/spark/sql/internal/SQLConf.scala | 124 ++++++++++-------- .../sql/execution/QueryExecutionSuite.scala | 16 +++ 3 files changed, 88 insertions(+), 58 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index 315a2d74c92ce..1520c3c9c2e0f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicBoolean import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{NumericType, StringType} import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils @@ -178,11 +179,14 @@ package object util extends Logging { val DEFAULT_MAX_TO_STRING_FIELDS = 25 private[spark] def maxNumToStringFields = { - if (SparkEnv.get != null) { + 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) } /** Whether we have warned about plan string truncation yet. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index e7c9a83798907..a47951880fb4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -331,16 +331,16 @@ object SQLConf { val PARQUET_SCHEMA_MERGING_ENABLED = buildConf("spark.sql.parquet.mergeSchema") .doc("When true, the Parquet data source merges schemas collected from all data files, " + - "otherwise the schema is picked from the summary file or a random data file " + - "if no summary file is available.") + "otherwise the schema is picked from the summary file or a random data file " + + "if no summary file is available.") .booleanConf .createWithDefault(false) val PARQUET_SCHEMA_RESPECT_SUMMARIES = buildConf("spark.sql.parquet.respectSummaryFiles") .doc("When true, we make assumption that all part-files of Parquet are consistent with " + - "summary files and we will ignore them when merging schema. Otherwise, if this is " + - "false, which is the default, we will merge all part-files. This should be considered " + - "as expert-only option, and shouldn't be enabled before knowing what it means exactly.") + "summary files and we will ignore them when merging schema. Otherwise, if this is " + + "false, which is the default, we will merge all part-files. This should be considered " + + "as expert-only option, and shouldn't be enabled before knowing what it means exactly.") .booleanConf .createWithDefault(false) @@ -418,9 +418,9 @@ object SQLConf { .doc("If true, enables Parquet filter push-down optimization for Timestamp. " + "This configuration only has an effect when 'spark.sql.parquet.filterPushdown' is " + "enabled and Timestamp stored as TIMESTAMP_MICROS or TIMESTAMP_MILLIS type.") - .internal() - .booleanConf - .createWithDefault(true) + .internal() + .booleanConf + .createWithDefault(true) val PARQUET_FILTER_PUSHDOWN_DECIMAL_ENABLED = buildConf("spark.sql.parquet.filterPushdown.decimal") @@ -432,11 +432,11 @@ object SQLConf { val PARQUET_FILTER_PUSHDOWN_STRING_STARTSWITH_ENABLED = buildConf("spark.sql.parquet.filterPushdown.string.startsWith") - .doc("If true, enables Parquet filter push-down optimization for string startsWith function. " + - "This configuration only has an effect when 'spark.sql.parquet.filterPushdown' is enabled.") - .internal() - .booleanConf - .createWithDefault(true) + .doc("If true, enables Parquet filter push-down optimization for string startsWith function. " + + "This configuration only has an effect when 'spark.sql.parquet.filterPushdown' is enabled.") + .internal() + .booleanConf + .createWithDefault(true) val PARQUET_FILTER_PUSHDOWN_INFILTERTHRESHOLD = buildConf("spark.sql.parquet.pushdown.inFilterThreshold") @@ -527,34 +527,34 @@ object SQLConf { val HIVE_VERIFY_PARTITION_PATH = buildConf("spark.sql.hive.verifyPartitionPath") .doc("When true, check all the partition paths under the table\'s root directory " + - "when reading data stored in HDFS. This configuration will be deprecated in the future " + - "releases and replaced by spark.files.ignoreMissingFiles.") + "when reading data stored in HDFS. This configuration will be deprecated in the future " + + "releases and replaced by spark.files.ignoreMissingFiles.") .booleanConf .createWithDefault(false) val HIVE_METASTORE_PARTITION_PRUNING = buildConf("spark.sql.hive.metastorePartitionPruning") .doc("When true, some predicates will be pushed down into the Hive metastore so that " + - "unmatching partitions can be eliminated earlier. This only affects Hive tables " + - "not converted to filesource relations (see HiveUtils.CONVERT_METASTORE_PARQUET and " + - "HiveUtils.CONVERT_METASTORE_ORC for more information).") + "unmatching partitions can be eliminated earlier. This only affects Hive tables " + + "not converted to filesource relations (see HiveUtils.CONVERT_METASTORE_PARQUET and " + + "HiveUtils.CONVERT_METASTORE_ORC for more information).") .booleanConf .createWithDefault(true) val HIVE_MANAGE_FILESOURCE_PARTITIONS = buildConf("spark.sql.hive.manageFilesourcePartitions") .doc("When true, enable metastore partition management for file source tables as well. " + - "This includes both datasource and converted Hive tables. When partition management " + - "is enabled, datasource tables store partition in the Hive metastore, and use the " + - "metastore to prune partitions during query planning.") + "This includes both datasource and converted Hive tables. When partition management " + + "is enabled, datasource tables store partition in the Hive metastore, and use the " + + "metastore to prune partitions during query planning.") .booleanConf .createWithDefault(true) val HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE = buildConf("spark.sql.hive.filesourcePartitionFileCacheSize") .doc("When nonzero, enable caching of partition file metadata in memory. All tables share " + - "a cache that can use up to specified num bytes for file metadata. This conf only " + - "has an effect when hive filesource partition management is enabled.") + "a cache that can use up to specified num bytes for file metadata. This conf only " + + "has an effect when hive filesource partition management is enabled.") .longConf .createWithDefault(250 * 1024 * 1024) @@ -643,12 +643,12 @@ object SQLConf { .createWithDefault(false) val GATHER_FASTSTAT = buildConf("spark.sql.hive.gatherFastStats") - .internal() - .doc("When true, fast stats (number of files and total size of all files) will be gathered" + - " in parallel while repairing table partitions to avoid the sequential listing in Hive" + - " metastore.") - .booleanConf - .createWithDefault(true) + .internal() + .doc("When true, fast stats (number of files and total size of all files) will be gathered" + + " in parallel while repairing table partitions to avoid the sequential listing in Hive" + + " metastore.") + .booleanConf + .createWithDefault(true) val PARTITION_COLUMN_TYPE_INFERENCE = buildConf("spark.sql.sources.partitionColumnTypeInference.enabled") @@ -669,13 +669,13 @@ object SQLConf { val CROSS_JOINS_ENABLED = buildConf("spark.sql.crossJoin.enabled") .doc("When false, we will throw an error if a query contains a cartesian product without " + - "explicit CROSS JOIN syntax.") + "explicit CROSS JOIN syntax.") .booleanConf .createWithDefault(false) val ORDER_BY_ORDINAL = buildConf("spark.sql.orderByOrdinal") .doc("When true, the ordinal numbers are treated as the position in the select list. " + - "When false, the ordinal numbers in order/sort by clause are ignored.") + "When false, the ordinal numbers in order/sort by clause are ignored.") .booleanConf .createWithDefault(true) @@ -727,10 +727,10 @@ object SQLConf { // Whether to automatically resolve ambiguity in join conditions for self-joins. // See SPARK-6231. val DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY = - buildConf("spark.sql.selfJoinAutoResolveAmbiguity") - .internal() - .booleanConf - .createWithDefault(true) + buildConf("spark.sql.selfJoinAutoResolveAmbiguity") + .internal() + .booleanConf + .createWithDefault(true) // Whether to retain group by columns or not in GroupedData.agg. val DATAFRAME_RETAIN_GROUP_COLUMNS = buildConf("spark.sql.retainGroupColumns") @@ -759,11 +759,11 @@ object SQLConf { val WHOLESTAGE_CODEGEN_USE_ID_IN_CLASS_NAME = buildConf("spark.sql.codegen.useIdInClassName") - .internal() - .doc("When true, embed the (whole-stage) codegen stage ID into " + - "the class name of the generated class as a suffix") - .booleanConf - .createWithDefault(true) + .internal() + .doc("When true, embed the (whole-stage) codegen stage ID into " + + "the class name of the generated class as a suffix") + .booleanConf + .createWithDefault(true) val WHOLESTAGE_MAX_NUM_FIELDS = buildConf("spark.sql.codegen.maxFields") .internal() @@ -1258,7 +1258,7 @@ object SQLConf { buildConf("spark.sql.execution.rangeExchange.sampleSizePerPartition") .internal() .doc("Number of points to sample per partition in order to determine the range boundaries" + - " for range partitioning, typically used in global sorting (without limit).") + " for range partitioning, typically used in global sorting (without limit).") .intConf .createWithDefault(100) @@ -1334,8 +1334,8 @@ object SQLConf { "information. The values of options whose names that match this regex will be redacted " + "in the explain output. This redaction is applied on top of the global redaction " + s"configuration defined by ${SECRET_REDACTION_PATTERN.key}.") - .regexConf - .createWithDefault("(?i)url".r) + .regexConf + .createWithDefault("(?i)url".r) val SQL_STRING_REDACTION_PATTERN = buildConf("spark.sql.redaction.string.regex") @@ -1359,19 +1359,19 @@ object SQLConf { val ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION = buildConf("spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation") - .internal() - .doc("When this option is set to true, creating managed tables with nonempty location " + - "is allowed. Otherwise, an analysis exception is thrown. ") - .booleanConf - .createWithDefault(false) + .internal() + .doc("When this option is set to true, creating managed tables with nonempty location " + + "is allowed. Otherwise, an analysis exception is thrown. ") + .booleanConf + .createWithDefault(false) val CONTINUOUS_STREAMING_EXECUTOR_QUEUE_SIZE = buildConf("spark.sql.streaming.continuous.executorQueueSize") - .internal() - .doc("The size (measured in number of rows) of the queue used in continuous execution to" + - " buffer the results of a ContinuousDataReader.") - .intConf - .createWithDefault(1024) + .internal() + .doc("The size (measured in number of rows) of the queue used in continuous execution to" + + " buffer the results of a ContinuousDataReader.") + .intConf + .createWithDefault(1024) val CONTINUOUS_STREAMING_EXECUTOR_POLL_INTERVAL_MS = buildConf("spark.sql.streaming.continuous.executorPollIntervalMs") @@ -1429,8 +1429,8 @@ object SQLConf { "issues. Turn on this config to insert a local sort before actually doing repartition " + "to generate consistent repartition results. The performance of repartition() may go " + "down since we insert extra local sort before it.") - .booleanConf - .createWithDefault(true) + .booleanConf + .createWithDefault(true) val NESTED_SCHEMA_PRUNING_ENABLED = buildConf("spark.sql.optimizer.nestedSchemaPruning.enabled") @@ -1446,8 +1446,8 @@ object SQLConf { buildConf("spark.sql.execution.topKSortFallbackThreshold") .internal() .doc("In SQL queries with a SORT followed by a LIMIT like " + - "'SELECT x FROM t ORDER BY y LIMIT m', if m is under this threshold, do a top-K sort" + - " in memory, otherwise do a global sort which spills to disk if necessary.") + "'SELECT x FROM t ORDER BY y LIMIT m', if m is under this threshold, do a top-K sort" + + " in memory, otherwise do a global sort which spills to disk if necessary.") .intConf .createWithDefault(Int.MaxValue) @@ -1554,6 +1554,14 @@ object SQLConf { .internal() .booleanConf .createWithDefault(false) + + val MAX_TO_STRING_FIELDS = buildConf("spark.sql.debug.maxToStringFields") + .internal() + .doc("Maximum number of fields of sequence-like entries that can be converted to strings " + + "in debug output. Any elements beyond the limit will be dropped and replaced by a" + + """ "... N more fields" placeholder.""") + .intConf + .createWithDefault(25) } /** @@ -1965,6 +1973,8 @@ class SQLConf extends Serializable with Logging { def integralDivideReturnLong: Boolean = getConf(SQLConf.LEGACY_INTEGRALDIVIDE_RETURN_LONG) + def maxToStringFields: Int = getConf(SQLConf.MAX_TO_STRING_FIELDS) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index ae865231dd466..5c09d0d18cb7f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -20,6 +20,7 @@ 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.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext case class QueryExecutionTestRecord( @@ -102,6 +103,21 @@ class QueryExecutionSuite extends SharedSQLContext { } } + test("limit number of fields by sql config") { + def relationPlans: String = { + val ds = spark.createDataset(Seq(QueryExecutionTestRecord( + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, + 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26))) + ds.queryExecution.toString + } + withSQLConf(SQLConf.MAX_TO_STRING_FIELDS.key -> "26") { + assert(relationPlans.contains("more fields")) + } + withSQLConf(SQLConf.MAX_TO_STRING_FIELDS.key -> "27") { + assert(!relationPlans.contains("more fields")) + } + } + test("toString() exception/error handling") { spark.experimental.extraStrategies = Seq( new SparkStrategy { From 2bf11fcb1c22d7118c2bcb24cc279494ea953482 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 27 Sep 2018 22:30:35 +0200 Subject: [PATCH 31/41] Adding Spark Core config to control maximum number of fields --- .../org/apache/spark/internal/config/package.scala | 10 ++++++++++ .../org/apache/spark/sql/catalyst/util/package.scala | 12 ++++++------ 2 files changed, 16 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 9891b6a2196de..c82174c16b4b5 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/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 " + + "in debug output. Any elements beyond the limit will be dropped and replaced by a" + + """ "... N more fields" placeholder. The config will be removed in Spark 3.0.""") + .intConf + .checkValue(v => v > 0, "The value should be a positive integer.") + .createWithDefault(25) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index 1520c3c9c2e0f..23518f869a246 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicBoolean import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging +import org.apache.spark.internal.config import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{NumericType, StringType} @@ -174,15 +175,14 @@ package object util extends Logging { /** * 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. + * by setting the 'spark.debug.maxToStringFields' conf in SparkEnv or by settings the SQL config + * `spark.sql.debug.maxToStringFields`. */ - val DEFAULT_MAX_TO_STRING_FIELDS = 25 - - private[spark] def maxNumToStringFields = { + private[spark] def maxNumToStringFields: Int = { val legacyLimit = if (SparkEnv.get != null) { - SparkEnv.get.conf.getInt("spark.debug.maxToStringFields", DEFAULT_MAX_TO_STRING_FIELDS) + SparkEnv.get.conf.get(config.MAX_TO_STRING_FIELDS) } else { - DEFAULT_MAX_TO_STRING_FIELDS + config.MAX_TO_STRING_FIELDS.defaultValue.get } val sqlConfLimit = SQLConf.get.maxToStringFields From bd331c5f8b2e3a0642cc9f818f400af9cbcf37ef Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 27 Sep 2018 22:47:57 +0200 Subject: [PATCH 32/41] Revert indentations --- .../apache/spark/sql/internal/SQLConf.scala | 114 +++++++++--------- 1 file changed, 57 insertions(+), 57 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index cdcba785a5dd1..89788bf8fb346 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -331,16 +331,16 @@ object SQLConf { val PARQUET_SCHEMA_MERGING_ENABLED = buildConf("spark.sql.parquet.mergeSchema") .doc("When true, the Parquet data source merges schemas collected from all data files, " + - "otherwise the schema is picked from the summary file or a random data file " + - "if no summary file is available.") + "otherwise the schema is picked from the summary file or a random data file " + + "if no summary file is available.") .booleanConf .createWithDefault(false) val PARQUET_SCHEMA_RESPECT_SUMMARIES = buildConf("spark.sql.parquet.respectSummaryFiles") .doc("When true, we make assumption that all part-files of Parquet are consistent with " + - "summary files and we will ignore them when merging schema. Otherwise, if this is " + - "false, which is the default, we will merge all part-files. This should be considered " + - "as expert-only option, and shouldn't be enabled before knowing what it means exactly.") + "summary files and we will ignore them when merging schema. Otherwise, if this is " + + "false, which is the default, we will merge all part-files. This should be considered " + + "as expert-only option, and shouldn't be enabled before knowing what it means exactly.") .booleanConf .createWithDefault(false) @@ -418,9 +418,9 @@ object SQLConf { .doc("If true, enables Parquet filter push-down optimization for Timestamp. " + "This configuration only has an effect when 'spark.sql.parquet.filterPushdown' is " + "enabled and Timestamp stored as TIMESTAMP_MICROS or TIMESTAMP_MILLIS type.") - .internal() - .booleanConf - .createWithDefault(true) + .internal() + .booleanConf + .createWithDefault(true) val PARQUET_FILTER_PUSHDOWN_DECIMAL_ENABLED = buildConf("spark.sql.parquet.filterPushdown.decimal") @@ -432,11 +432,11 @@ object SQLConf { val PARQUET_FILTER_PUSHDOWN_STRING_STARTSWITH_ENABLED = buildConf("spark.sql.parquet.filterPushdown.string.startsWith") - .doc("If true, enables Parquet filter push-down optimization for string startsWith function. " + - "This configuration only has an effect when 'spark.sql.parquet.filterPushdown' is enabled.") - .internal() - .booleanConf - .createWithDefault(true) + .doc("If true, enables Parquet filter push-down optimization for string startsWith function. " + + "This configuration only has an effect when 'spark.sql.parquet.filterPushdown' is enabled.") + .internal() + .booleanConf + .createWithDefault(true) val PARQUET_FILTER_PUSHDOWN_INFILTERTHRESHOLD = buildConf("spark.sql.parquet.pushdown.inFilterThreshold") @@ -530,34 +530,34 @@ object SQLConf { val HIVE_VERIFY_PARTITION_PATH = buildConf("spark.sql.hive.verifyPartitionPath") .doc("When true, check all the partition paths under the table\'s root directory " + - "when reading data stored in HDFS. This configuration will be deprecated in the future " + - "releases and replaced by spark.files.ignoreMissingFiles.") + "when reading data stored in HDFS. This configuration will be deprecated in the future " + + "releases and replaced by spark.files.ignoreMissingFiles.") .booleanConf .createWithDefault(false) val HIVE_METASTORE_PARTITION_PRUNING = buildConf("spark.sql.hive.metastorePartitionPruning") .doc("When true, some predicates will be pushed down into the Hive metastore so that " + - "unmatching partitions can be eliminated earlier. This only affects Hive tables " + - "not converted to filesource relations (see HiveUtils.CONVERT_METASTORE_PARQUET and " + - "HiveUtils.CONVERT_METASTORE_ORC for more information).") + "unmatching partitions can be eliminated earlier. This only affects Hive tables " + + "not converted to filesource relations (see HiveUtils.CONVERT_METASTORE_PARQUET and " + + "HiveUtils.CONVERT_METASTORE_ORC for more information).") .booleanConf .createWithDefault(true) val HIVE_MANAGE_FILESOURCE_PARTITIONS = buildConf("spark.sql.hive.manageFilesourcePartitions") .doc("When true, enable metastore partition management for file source tables as well. " + - "This includes both datasource and converted Hive tables. When partition management " + - "is enabled, datasource tables store partition in the Hive metastore, and use the " + - "metastore to prune partitions during query planning.") + "This includes both datasource and converted Hive tables. When partition management " + + "is enabled, datasource tables store partition in the Hive metastore, and use the " + + "metastore to prune partitions during query planning.") .booleanConf .createWithDefault(true) val HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE = buildConf("spark.sql.hive.filesourcePartitionFileCacheSize") .doc("When nonzero, enable caching of partition file metadata in memory. All tables share " + - "a cache that can use up to specified num bytes for file metadata. This conf only " + - "has an effect when hive filesource partition management is enabled.") + "a cache that can use up to specified num bytes for file metadata. This conf only " + + "has an effect when hive filesource partition management is enabled.") .longConf .createWithDefault(250 * 1024 * 1024) @@ -646,12 +646,12 @@ object SQLConf { .createWithDefault(false) val GATHER_FASTSTAT = buildConf("spark.sql.hive.gatherFastStats") - .internal() - .doc("When true, fast stats (number of files and total size of all files) will be gathered" + - " in parallel while repairing table partitions to avoid the sequential listing in Hive" + - " metastore.") - .booleanConf - .createWithDefault(true) + .internal() + .doc("When true, fast stats (number of files and total size of all files) will be gathered" + + " in parallel while repairing table partitions to avoid the sequential listing in Hive" + + " metastore.") + .booleanConf + .createWithDefault(true) val PARTITION_COLUMN_TYPE_INFERENCE = buildConf("spark.sql.sources.partitionColumnTypeInference.enabled") @@ -672,13 +672,13 @@ object SQLConf { val CROSS_JOINS_ENABLED = buildConf("spark.sql.crossJoin.enabled") .doc("When false, we will throw an error if a query contains a cartesian product without " + - "explicit CROSS JOIN syntax.") + "explicit CROSS JOIN syntax.") .booleanConf .createWithDefault(false) val ORDER_BY_ORDINAL = buildConf("spark.sql.orderByOrdinal") .doc("When true, the ordinal numbers are treated as the position in the select list. " + - "When false, the ordinal numbers in order/sort by clause are ignored.") + "When false, the ordinal numbers in order/sort by clause are ignored.") .booleanConf .createWithDefault(true) @@ -730,10 +730,10 @@ object SQLConf { // Whether to automatically resolve ambiguity in join conditions for self-joins. // See SPARK-6231. val DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY = - buildConf("spark.sql.selfJoinAutoResolveAmbiguity") - .internal() - .booleanConf - .createWithDefault(true) + buildConf("spark.sql.selfJoinAutoResolveAmbiguity") + .internal() + .booleanConf + .createWithDefault(true) // Whether to retain group by columns or not in GroupedData.agg. val DATAFRAME_RETAIN_GROUP_COLUMNS = buildConf("spark.sql.retainGroupColumns") @@ -762,11 +762,11 @@ object SQLConf { val WHOLESTAGE_CODEGEN_USE_ID_IN_CLASS_NAME = buildConf("spark.sql.codegen.useIdInClassName") - .internal() - .doc("When true, embed the (whole-stage) codegen stage ID into " + - "the class name of the generated class as a suffix") - .booleanConf - .createWithDefault(true) + .internal() + .doc("When true, embed the (whole-stage) codegen stage ID into " + + "the class name of the generated class as a suffix") + .booleanConf + .createWithDefault(true) val WHOLESTAGE_MAX_NUM_FIELDS = buildConf("spark.sql.codegen.maxFields") .internal() @@ -1261,7 +1261,7 @@ object SQLConf { buildConf("spark.sql.execution.rangeExchange.sampleSizePerPartition") .internal() .doc("Number of points to sample per partition in order to determine the range boundaries" + - " for range partitioning, typically used in global sorting (without limit).") + " for range partitioning, typically used in global sorting (without limit).") .intConf .createWithDefault(100) @@ -1346,8 +1346,8 @@ object SQLConf { "information. The values of options whose names that match this regex will be redacted " + "in the explain output. This redaction is applied on top of the global redaction " + s"configuration defined by ${SECRET_REDACTION_PATTERN.key}.") - .regexConf - .createWithDefault("(?i)url".r) + .regexConf + .createWithDefault("(?i)url".r) val SQL_STRING_REDACTION_PATTERN = buildConf("spark.sql.redaction.string.regex") @@ -1371,19 +1371,19 @@ object SQLConf { val ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION = buildConf("spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation") - .internal() - .doc("When this option is set to true, creating managed tables with nonempty location " + - "is allowed. Otherwise, an analysis exception is thrown. ") - .booleanConf - .createWithDefault(false) + .internal() + .doc("When this option is set to true, creating managed tables with nonempty location " + + "is allowed. Otherwise, an analysis exception is thrown. ") + .booleanConf + .createWithDefault(false) val CONTINUOUS_STREAMING_EXECUTOR_QUEUE_SIZE = buildConf("spark.sql.streaming.continuous.executorQueueSize") - .internal() - .doc("The size (measured in number of rows) of the queue used in continuous execution to" + - " buffer the results of a ContinuousDataReader.") - .intConf - .createWithDefault(1024) + .internal() + .doc("The size (measured in number of rows) of the queue used in continuous execution to" + + " buffer the results of a ContinuousDataReader.") + .intConf + .createWithDefault(1024) val CONTINUOUS_STREAMING_EXECUTOR_POLL_INTERVAL_MS = buildConf("spark.sql.streaming.continuous.executorPollIntervalMs") @@ -1441,8 +1441,8 @@ object SQLConf { "issues. Turn on this config to insert a local sort before actually doing repartition " + "to generate consistent repartition results. The performance of repartition() may go " + "down since we insert extra local sort before it.") - .booleanConf - .createWithDefault(true) + .booleanConf + .createWithDefault(true) val NESTED_SCHEMA_PRUNING_ENABLED = buildConf("spark.sql.optimizer.nestedSchemaPruning.enabled") @@ -1458,8 +1458,8 @@ object SQLConf { buildConf("spark.sql.execution.topKSortFallbackThreshold") .internal() .doc("In SQL queries with a SORT followed by a LIMIT like " + - "'SELECT x FROM t ORDER BY y LIMIT m', if m is under this threshold, do a top-K sort" + - " in memory, otherwise do a global sort which spills to disk if necessary.") + "'SELECT x FROM t ORDER BY y LIMIT m', if m is under this threshold, do a top-K sort" + + " in memory, otherwise do a global sort which spills to disk if necessary.") .intConf .createWithDefault(Int.MaxValue) From 2375064390bf3f673aa4453ac39521130fcdd7ad Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 11 Oct 2018 12:17:04 +0200 Subject: [PATCH 33/41] Making writeOrError multi-line --- .../org/apache/spark/sql/execution/QueryExecution.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 2f5311cc3f126..39dad51ba8515 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -192,8 +192,12 @@ 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 writeOrError(writer: Writer)(f: Writer => Unit): Unit = { + try f(writer) + catch { + case e: AnalysisException => writer.write(e.toString) + } + } private def writePlans(writer: Writer, maxFields: Option[Int]): Unit = { val (verbose, addSuffix) = (true, false) From 8befa13baf4bfeeee429ae6315614518d2e392ff Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 11 Oct 2018 13:19:54 +0200 Subject: [PATCH 34/41] Removing core config: spark.debug.maxToStringFields --- .../spark/internal/config/package.scala | 10 ---------- .../spark/sql/catalyst/util/package.scala | 19 +------------------ .../datasources/jdbc/JDBCRelation.scala | 4 ++-- 3 files changed, 3 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index f4fe4d1442d14..e8b1d8859cc44 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -647,14 +647,4 @@ 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 " + - "in debug output. Any elements beyond the limit will be dropped and replaced by a" + - """ "... N more fields" placeholder. The config will be removed in Spark 3.0.""") - .intConf - .checkValue(v => v > 0, "The value should be a positive integer.") - .createWithDefault(25) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index 23518f869a246..5c55111f83010 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -172,23 +172,6 @@ package object util extends Logging { 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) @@ -204,7 +187,7 @@ package object util extends Logging { sep: String, end: String, maxFields: Option[Int]): String = { - val maxNumFields = maxFields.getOrElse(maxNumToStringFields) + val maxNumFields = maxFields.getOrElse(SQLConf.get.maxToStringFields) if (seq.length > maxNumFields) { if (truncationWarningPrinted.compareAndSet(false, true)) { logWarning( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index 93746de45ae5a..5c48ed18f8673 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -27,7 +27,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SaveMode, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.catalyst.util.maxNumToStringFields +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.jdbc.JdbcDialects import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, DateType, NumericType, StructType, TimestampType} @@ -160,7 +160,7 @@ private[sql] object JDBCRelation extends Logging { resolver(f.name, columnName) || resolver(dialect.quoteIdentifier(f.name), columnName) }.getOrElse { throw new AnalysisException(s"User-defined partition column $columnName not " + - s"found in the JDBC relation: ${schema.simpleString(maxNumToStringFields)}") + s"found in the JDBC relation: ${schema.simpleString(SQLConf.get.maxToStringFields)}") } column.dataType match { case _: NumericType | DateType | TimestampType => From 28795c7fe5dd9a04467b4767fcb179196432bbc6 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 11 Oct 2018 13:23:37 +0200 Subject: [PATCH 35/41] Improving description of spark.sql.debug.maxToStringFields --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index ff7e20ed1194a..b416a6d8a14cf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1570,7 +1570,7 @@ object SQLConf { val MAX_TO_STRING_FIELDS = buildConf("spark.sql.debug.maxToStringFields") .internal() - .doc("Maximum number of fields of sequence-like entries that can be converted to strings " + + .doc("Maximum number of fields of a tree node that can be converted to strings " + "in debug output. Any elements beyond the limit will be dropped and replaced by a" + """ "... N more fields" placeholder.""") .intConf From a246db4120cba955d48aa1e3bd17c5e7ba9e3181 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 11 Oct 2018 13:50:41 +0200 Subject: [PATCH 36/41] Limit number of fields in structs too --- .../scala/org/apache/spark/sql/types/DataType.scala | 10 ++++++++-- .../scala/org/apache/spark/sql/types/StructType.scala | 6 ++++-- .../src/main/scala/org/apache/spark/sql/Dataset.scala | 2 +- .../org/apache/spark/sql/KeyValueGroupedDataset.scala | 4 ++-- .../apache/spark/sql/RelationalGroupedDataset.scala | 2 +- .../sql/execution/datasources/FileSourceStrategy.scala | 2 +- .../sql/execution/datasources/jdbc/JDBCRelation.scala | 2 +- 7 files changed, 18 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala index e53628d11ccf3..317899a947a2b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -75,8 +75,14 @@ abstract class DataType extends AbstractDataType { /** String representation for the type saved in external catalogs. */ def catalogString: String = simpleString - /** Readable string representation for the type with truncation */ - private[sql] def simpleString(maxNumberFields: Int): String = simpleString + /** + * Readable string representation for the type with truncation. + * @param maxFields Maximum number of fields that will be converted to strings. + * Any elements beyond the limit will be dropped. + * `None` means the limit is defined by the SQL config + * `spark.sql.debug.maxToStringFields`. + */ + private[sql] def simpleString(maxFields: Option[Int]): String = simpleString def sql: String = simpleString.toUpperCase(Locale.ROOT) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index d3312914d7539..67c95e6445ef8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -28,6 +28,7 @@ import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, LegacyTypeStringParser} import org.apache.spark.sql.catalyst.util.{quoteIdentifier, truncatedString} +import org.apache.spark.sql.internal.SQLConf /** * A [[StructType]] object can be constructed by @@ -369,10 +370,11 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru */ def toDDL: String = fields.map(_.toDDL).mkString(",") - private[sql] override def simpleString(maxNumberFields: Int): String = { + private[sql] override def simpleString(maxFields: Option[Int]): String = { val builder = new StringBuilder + val maxNumberFields = maxFields.getOrElse(SQLConf.get.maxToStringFields) val fieldTypes = fields.take(maxNumberFields).map { - f => s"${f.name}: ${f.dataType.simpleString(maxNumberFields)}" + f => s"${f.name}: ${f.dataType.simpleString(maxFields)}" } builder.append("struct<") builder.append(fieldTypes.mkString(", ")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index fa14aa14ee968..f2a3b54fc6186 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -375,7 +375,7 @@ class Dataset[T] private[sql]( try { val builder = new StringBuilder val fields = schema.take(2).map { - case f => s"${f.name}: ${f.dataType.simpleString(2)}" + case f => s"${f.name}: ${f.dataType.simpleString(Some(2))}" } builder.append("[") builder.append(fields.mkString(", ")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index 6bab21dca0cbd..490f223ce537d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -567,10 +567,10 @@ class KeyValueGroupedDataset[K, V] private[sql]( override def toString: String = { val builder = new StringBuilder val kFields = kExprEnc.schema.map { - case f => s"${f.name}: ${f.dataType.simpleString(2)}" + case f => s"${f.name}: ${f.dataType.simpleString(Some(2))}" } val vFields = vExprEnc.schema.map { - case f => s"${f.name}: ${f.dataType.simpleString(2)}" + case f => s"${f.name}: ${f.dataType.simpleString(Some(2))}" } builder.append("KeyValueGroupedDataset: [key: [") builder.append(kFields.take(2).mkString(", ")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index d4e75b5ebd405..cee27a013be51 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -528,7 +528,7 @@ class RelationalGroupedDataset protected[sql]( builder.append("RelationalGroupedDataset: [grouping expressions: [") val kFields = groupingExprs.collect { case expr: NamedExpression if expr.resolved => - s"${expr.name}: ${expr.dataType.simpleString(2)}" + s"${expr.name}: ${expr.dataType.simpleString(Some(2))}" case expr: NamedExpression => expr.name case o => o.toString } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index fe27b78bf3360..9546c70928917 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -194,7 +194,7 @@ object FileSourceStrategy extends Strategy with Logging { .filter(requiredAttributes.contains) .filterNot(partitionColumns.contains) val outputSchema = readDataColumns.toStructType - logInfo(s"Output Data Schema: ${outputSchema.simpleString(5)}") + logInfo(s"Output Data Schema: ${outputSchema.simpleString(Some(5))}") val outputAttributes = readDataColumns ++ partitionColumns diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index 5c48ed18f8673..59ba993f1d086 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -160,7 +160,7 @@ private[sql] object JDBCRelation extends Logging { resolver(f.name, columnName) || resolver(dialect.quoteIdentifier(f.name), columnName) }.getOrElse { throw new AnalysisException(s"User-defined partition column $columnName not " + - s"found in the JDBC relation: ${schema.simpleString(SQLConf.get.maxToStringFields)}") + s"found in the JDBC relation: ${schema.simpleString(maxFields = None)}") } column.dataType match { case _: NumericType | DateType | TimestampType => From d4da29bcc3f44356e969ef545fe541b9e7b6e807 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 11 Oct 2018 13:53:01 +0200 Subject: [PATCH 37/41] Description of simpleString of TreeNode. --- .../org/apache/spark/sql/catalyst/trees/TreeNode.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 7b2b8d55d3543..17529c997513e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -456,7 +456,13 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { case other => other :: Nil }.mkString(", ") - /** ONE line description of this node. */ + /** + * ONE line description of this node. + * @param maxFields Maximum number of fields that will be converted to strings. + * Any elements beyond the limit will be dropped. + * `None` means the limit is defined by the SQL config + * `spark.sql.debug.maxToStringFields`. + */ def simpleString(maxFields: Option[Int]): String = { s"$nodeName ${argString(maxFields)}".trim } From 41b57bc7928b717089658276254f63383172450d Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 11 Oct 2018 13:55:55 +0200 Subject: [PATCH 38/41] Added description of maxFields param of truncatedString --- .../scala/org/apache/spark/sql/catalyst/util/package.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index 5c55111f83010..036a826913ed2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -177,7 +177,9 @@ package object util extends Logging { /** * Format a sequence with semantics similar to calling .mkString(). Any elements beyond - * maxNumToStringFields will be dropped and replaced by a "... N more fields" placeholder. + * maxFields will be dropped and replaced by a "... N more fields" placeholder. + * If maxFields is set to `None`, maximum number of fields is defined by the SQL config + * `spark.sql.debug.maxToStringFields` * * @return the trimmed and formatted string. */ From 28cce2e810ba84d60b066fc8fe8fba38aa7ecc1a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 11 Oct 2018 13:58:21 +0200 Subject: [PATCH 39/41] Fix typo --- .../main/scala/org/apache/spark/sql/catalyst/util/package.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index 036a826913ed2..0cff70381bd68 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -179,7 +179,7 @@ package object util extends Logging { * Format a sequence with semantics similar to calling .mkString(). Any elements beyond * maxFields will be dropped and replaced by a "... N more fields" placeholder. * If maxFields is set to `None`, maximum number of fields is defined by the SQL config - * `spark.sql.debug.maxToStringFields` + * `spark.sql.debug.maxToStringFields`. * * @return the trimmed and formatted string. */ From e4567cbc01d58a17a1b4ba8a618463b8872a69a2 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 11 Oct 2018 13:59:55 +0200 Subject: [PATCH 40/41] Passing maxField --- .../scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala index 43a1825f5434b..adaf7bd6eaffb 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala @@ -53,7 +53,7 @@ case class CatalystDataToAvro(child: Expression) extends UnaryExpression { } override def simpleString(maxFields: Option[Int]): String = { - s"to_avro(${child.sql}, ${child.dataType.simpleString})" + s"to_avro(${child.sql}, ${child.dataType.simpleString(maxFields)})" } override def sql: String = { From 9b721046c103f55d0abd4f65fd3e2e4b60be5ace Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 11 Oct 2018 14:09:05 +0200 Subject: [PATCH 41/41] Fix for the warning --- .../main/scala/org/apache/spark/sql/catalyst/util/package.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index 0cff70381bd68..3c8727727abd0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -194,7 +194,7 @@ package object util extends Logging { if (truncationWarningPrinted.compareAndSet(false, true)) { logWarning( "Truncated the string representation of a plan since it was too large. This " + - "behavior can be adjusted by setting 'spark.debug.maxToStringFields' in SparkEnv.conf.") + s"behavior can be adjusted by setting '${SQLConf.MAX_TO_STRING_FIELDS.key}'") } val numFields = math.max(0, maxNumFields - 1) seq.take(numFields).mkString(