This is an automated email from the ASF dual-hosted git repository.

yao pushed a commit to branch branch-3.5
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.5 by this push:
     new 0272916697f [SPARK-44737][SQL][UI] Should not display json format 
errors on SQL page for non-SparkThrowables on SQL Tab
0272916697f is described below

commit 0272916697f56dd26df2bd3e27b5b68ef72d099c
Author: Kent Yao <[email protected]>
AuthorDate: Fri Aug 11 14:58:46 2023 +0800

    [SPARK-44737][SQL][UI] Should not display json format errors on SQL page 
for non-SparkThrowables on SQL Tab
    
    ### What changes were proposed in this pull request?
    
    This pull request addresses the issue of inconsistent display for 
SparkThrowables and others. Currently, SparkThrowables appear as plain text 
while others are displayed in JSON format on SQL Tab. This fix aims to make the 
display consistent across all items.
    
    ### Why are the changes needed?
    
    To fix the inconsistent behavior, and the JSON format is not correctly 
displayed.
    
    For those non-SparkThrowables thrown by SQLExecution may contain errors 
either that we are not ready covered by the new error handling framework or 
some critical path(what I have met is OOM in RDD.getPartitions), we shall 
supply the stacktraces
    
    ### Does this PR introduce _any_ user-facing change?
    
    no
    
    ### How was this patch tested?
    
    add a unit test
    
    Closes #42407 from yaooqinn/SPARK-44737.
    
    Authored-by: Kent Yao <[email protected]>
    Signed-off-by: Kent Yao <[email protected]>
    (cherry picked from commit 5bb9cd8e55ffdbe88d82f6140583ba4cef219d9a)
    Signed-off-by: Kent Yao <[email protected]>
---
 .../org/apache/spark/SparkThrowableHelper.scala    | 14 ----
 .../apache/spark/sql/execution/SQLExecution.scala  |  3 +-
 .../spark/sql/execution/SQLJsonProtocolSuite.scala | 15 +++--
 .../execution/ui/SQLAppStatusListenerSuite.scala   |  9 +--
 .../spark/sql/execution/ui/UISeleniumSuite.scala   | 74 ++++++++++++++++++++++
 5 files changed, 87 insertions(+), 28 deletions(-)

diff --git 
a/common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala 
b/common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala
index 5106460d145..0f329b5655b 100644
--- a/common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala
+++ b/common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala
@@ -19,8 +19,6 @@ package org.apache.spark
 
 import scala.collection.JavaConverters._
 
-import com.fasterxml.jackson.core.util.MinimalPrettyPrinter
-
 import org.apache.spark.util.JsonUtils.toJsonString
 import org.apache.spark.util.SparkClassUtils
 
@@ -121,16 +119,4 @@ private[spark] object SparkThrowableHelper {
         }
     }
   }
-
-  def getMessage(throwable: Throwable): String = {
-    toJsonString { generator =>
-      val g = generator.setPrettyPrinter(new MinimalPrettyPrinter)
-      g.writeStartObject()
-      g.writeStringField("errorClass", throwable.getClass.getCanonicalName)
-      g.writeObjectFieldStart("messageParameters")
-      g.writeStringField("message", throwable.getMessage)
-      g.writeEndObject()
-      g.writeEndObject()
-    }
-  }
 }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala
index 66de9d37db0..daeac699c27 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala
@@ -133,8 +133,7 @@ object SQLExecution {
             case e: SparkThrowable =>
               SparkThrowableHelper.getMessage(e, ErrorMessageFormat.PRETTY)
             case e =>
-              // unexpected behavior
-              SparkThrowableHelper.getMessage(e)
+              Utils.exceptionString(e)
           }
           val event = SparkListenerSQLExecutionEnd(
             executionId,
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLJsonProtocolSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLJsonProtocolSuite.scala
index 49cca666d1d..8f7e68632bf 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLJsonProtocolSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLJsonProtocolSuite.scala
@@ -17,14 +17,15 @@
 
 package org.apache.spark.sql.execution
 
+import com.fasterxml.jackson.databind.ObjectMapper
 import org.json4s.jackson.JsonMethods._
 
-import org.apache.spark.{SparkFunSuite, SparkThrowableHelper}
+import org.apache.spark.SparkFunSuite
 import org.apache.spark.scheduler.SparkListenerEvent
 import org.apache.spark.sql.LocalSparkSession
 import org.apache.spark.sql.execution.ui.{SparkListenerSQLExecutionEnd, 
SparkListenerSQLExecutionStart}
 import org.apache.spark.sql.test.TestSparkSession
-import org.apache.spark.util.JsonProtocol
+import org.apache.spark.util.{JsonProtocol, Utils}
 
 class SQLJsonProtocolSuite extends SparkFunSuite with LocalSparkSession {
 
@@ -84,21 +85,23 @@ class SQLJsonProtocolSuite extends SparkFunSuite with 
LocalSparkSession {
   test("SparkListenerSQLExecutionEnd backward compatibility") {
     spark = new TestSparkSession()
     val qe = spark.sql("select 1").queryExecution
-    val errorMessage = SparkThrowableHelper.getMessage(new Exception("test"))
+    val exception = new Exception("test")
+    val errorMessage = Utils.exceptionString(exception)
+    val errorMessageJson = new ObjectMapper().writeValueAsString(errorMessage)
     val event = SparkListenerSQLExecutionEnd(1, 10, Some(errorMessage))
     event.duration = 1000
     event.executionName = Some("test")
     event.qe = qe
-    event.executionFailure = Some(new Exception("test"))
+    event.executionFailure = Some(exception)
     val json = JsonProtocol.sparkEventToJsonString(event)
     // scalastyle:off line.size.limit
     assert(parse(json) == parse(
-      """
+      s"""
         |{
         |  "Event" : 
"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd",
         |  "executionId" : 1,
         |  "time" : 10,
-        |  "errorMessage" : 
"{\"errorClass\":\"java.lang.Exception\",\"messageParameters\":{\"message\":\"test\"}}"
+        |  "errorMessage" : $errorMessageJson
         |}
       """.stripMargin))
     // scalastyle:on
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala
index 72f8de3e9cc..67206e9c655 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala
@@ -21,7 +21,6 @@ import java.util.Properties
 
 import scala.collection.mutable.{ArrayBuffer, ListBuffer}
 
-import com.fasterxml.jackson.databind.ObjectMapper
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{FileSystem, Path}
 import org.json4s.jackson.JsonMethods._
@@ -990,14 +989,12 @@ abstract class SQLAppStatusListenerSuite extends 
SharedSparkSession with JsonTes
 
   test("SPARK-40834: Use SparkListenerSQLExecutionEnd to track final SQL 
status in UI") {
     var received = false
+    val e = new Exception("test")
     spark.sparkContext.addSparkListener(new SparkListener {
       override def onOtherEvent(event: SparkListenerEvent): Unit = {
         event match {
           case SparkListenerSQLExecutionEnd(_, _, Some(errorMessage)) =>
-            val error = new ObjectMapper().readTree(errorMessage)
-            assert(error.get("errorClass").toPrettyString === 
"\"java.lang.Exception\"")
-            
assert(error.path("messageParameters").get("message").toPrettyString === 
"\"test\"")
-            received = true
+            received = errorMessage == Utils.exceptionString(e)
           case _ =>
         }
       }
@@ -1005,7 +1002,7 @@ abstract class SQLAppStatusListenerSuite extends 
SharedSparkSession with JsonTes
 
     intercept[Exception] {
       SQLExecution.withNewExecutionId(spark.range(1).queryExecution) {
-        throw new Exception("test")
+        throw e
       }
     }
     spark.sparkContext.listenerBus.waitUntilEmpty(10000)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/UISeleniumSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/UISeleniumSuite.scala
new file mode 100644
index 00000000000..eadafa9a754
--- /dev/null
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/UISeleniumSuite.scala
@@ -0,0 +1,74 @@
+/*
+ * 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.execution.ui
+
+import scala.concurrent.duration.DurationInt
+
+import org.openqa.selenium.htmlunit.HtmlUnitDriver
+import org.scalatest.concurrent.Eventually.eventually
+import org.scalatest.concurrent.Futures.{interval, timeout}
+import org.scalatestplus.selenium.WebBrowser
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.ui.SparkUICssErrorHandler
+
+class UISeleniumSuite extends SparkFunSuite with WebBrowser {
+
+  private var spark: SparkSession = _
+
+  implicit val webDriver: HtmlUnitDriver = new HtmlUnitDriver {
+    getWebClient.setCssErrorHandler(new SparkUICssErrorHandler)
+  }
+
+  override def afterAll(): Unit = {
+    try {
+      webDriver.quit()
+    } finally {
+      super.afterAll()
+    }
+  }
+
+  override def afterEach(): Unit = {
+    SparkSession.clearActiveSession()
+    SparkSession.clearDefaultSession()
+    if (spark != null) {
+      spark.stop()
+      spark = null
+    }
+  }
+
+  test("SPARK-44737: Should not display json format errors on SQL page for 
non-SparkThrowables") {
+    spark = SparkSession.builder()
+      .master("local[1,1]")
+      .appName("sql ui test")
+      .config("spark.ui.enabled", "true")
+      .config("spark.ui.port", "0")
+      .getOrCreate()
+
+    intercept[Exception](spark.sql("SET mapreduce.job.reduces = 0").isEmpty)
+    eventually(timeout(10.seconds), interval(100.milliseconds)) {
+      val webUrl = spark.sparkContext.uiWebUrl
+      assert(webUrl.isDefined, "please turn on spark.ui.enabled")
+      go to s"${webUrl.get}/SQL"
+      val sd = findAll(cssSelector("""#failed-table td 
.stacktrace-details""")).map(_.text).toList
+      assert(sd.size === 1, "SET mapreduce.job.reduces = 0 shall fail")
+      assert(sd.head.startsWith("java.lang.IllegalArgumentException:"))
+    }
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to