You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by hv...@apache.org on 2016/10/17 18:33:10 UTC

spark git commit: [SPARK-17751][SQL] Remove spark.sql.eagerAnalysis and Output the Plan if Existed in AnalysisException

Repository: spark
Updated Branches:
  refs/heads/master c7ac027d5 -> d88a1bae6


[SPARK-17751][SQL] Remove spark.sql.eagerAnalysis and Output the Plan if Existed in AnalysisException

### What changes were proposed in this pull request?
Dataset always does eager analysis now. Thus, `spark.sql.eagerAnalysis` is not used any more. Thus, we need to remove it.

This PR also outputs the plan. Without the fix, the analysis error is like
```
cannot resolve '`k1`' given input columns: [k, v]; line 1 pos 12
```

After the fix, the analysis error becomes:
```
org.apache.spark.sql.AnalysisException: cannot resolve '`k1`' given input columns: [k, v]; line 1 pos 12;
'Project [unresolvedalias(CASE WHEN ('k1 = 2) THEN 22 WHEN ('k1 = 4) THEN 44 ELSE 0 END, None), v#6]
+- SubqueryAlias t
   +- Project [_1#2 AS k#5, _2#3 AS v#6]
      +- LocalRelation [_1#2, _2#3]
```

### How was this patch tested?
N/A

Author: gatorsmile <ga...@gmail.com>

Closes #15316 from gatorsmile/eagerAnalysis.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d88a1bae
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d88a1bae
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d88a1bae

Branch: refs/heads/master
Commit: d88a1bae6a9c975c39549ec2326d839ea93949b2
Parents: c7ac027
Author: gatorsmile <ga...@gmail.com>
Authored: Mon Oct 17 11:33:06 2016 -0700
Committer: Herman van Hovell <hv...@databricks.com>
Committed: Mon Oct 17 11:33:06 2016 -0700

----------------------------------------------------------------------
 .../scala/org/apache/spark/sql/AnalysisException.scala    |  7 +++++++
 .../org/apache/spark/sql/execution/debug/package.scala    |  9 ---------
 .../scala/org/apache/spark/sql/internal/SQLConf.scala     | 10 ----------
 .../scala/org/apache/spark/sql/SQLQueryTestSuite.scala    |  3 +++
 4 files changed, 10 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/d88a1bae/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala
index f300330..7defb9d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala
@@ -42,6 +42,13 @@ class AnalysisException protected[sql] (
   }
 
   override def getMessage: String = {
+    val planAnnotation = plan.map(p => s";\n$p").getOrElse("")
+    getSimpleMessage + planAnnotation
+  }
+
+  // Outputs an exception without the logical plan.
+  // For testing only
+  def getSimpleMessage: String = {
     val lineAnnotation = line.map(l => s" line $l").getOrElse("")
     val positionAnnotation = startPosition.map(p => s" pos $p").getOrElse("")
     s"$message;$lineAnnotation$positionAnnotation"

http://git-wip-us.apache.org/repos/asf/spark/blob/d88a1bae/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala
----------------------------------------------------------------------
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 d321f4c..dd9d837 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,15 +70,6 @@ package object debug {
   }
 
   /**
-   * Augments [[SparkSession]] with debug methods.
-   */
-  implicit class DebugSQLContext(sparkSession: SparkSession) {
-    def debug(): Unit = {
-      sparkSession.conf.set(SQLConf.DATAFRAME_EAGER_ANALYSIS.key, false)
-    }
-  }
-
-  /**
    * Augments [[Dataset]]s with debug methods.
    */
   implicit class DebugQuery(query: Dataset[_]) extends Logging {

http://git-wip-us.apache.org/repos/asf/spark/blob/d88a1bae/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index a055e01..8afd39d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -388,14 +388,6 @@ object SQLConf {
       .intConf
       .createWithDefault(32)
 
-  // Whether to perform eager analysis when constructing a dataframe.
-  // Set to false when debugging requires the ability to look at invalid query plans.
-  val DATAFRAME_EAGER_ANALYSIS = SQLConfigBuilder("spark.sql.eagerAnalysis")
-    .internal()
-    .doc("When true, eagerly applies query analysis on DataFrame operations.")
-    .booleanConf
-    .createWithDefault(true)
-
   // Whether to automatically resolve ambiguity in join conditions for self-joins.
   // See SPARK-6231.
   val DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY =
@@ -748,8 +740,6 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging {
 
   def bucketingEnabled: Boolean = getConf(SQLConf.BUCKETING_ENABLED)
 
-  def dataFrameEagerAnalysis: Boolean = getConf(DATAFRAME_EAGER_ANALYSIS)
-
   def dataFrameSelfJoinAutoResolveAmbiguity: Boolean =
     getConf(DATAFRAME_SELF_JOIN_AUTO_RESOLVE_AMBIGUITY)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/d88a1bae/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala
index 55d5a56..02841d7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala
@@ -220,6 +220,9 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext {
       if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted)
 
     } catch {
+      case a: AnalysisException if a.plan.nonEmpty =>
+        // Do not output the logical plan tree which contains expression IDs.
+        (StructType(Seq.empty), Seq(a.getClass.getName, a.getSimpleMessage))
       case NonFatal(e) =>
         // If there is an exception, put the exception class followed by the message.
         (StructType(Seq.empty), Seq(e.getClass.getName, e.getMessage))


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org