You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "yaooqinn (via GitHub)" <gi...@apache.org> on 2023/08/14 06:33:05 UTC

[GitHub] [spark] yaooqinn opened a new pull request, #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

yaooqinn opened a new pull request, #42481:
URL: https://github.com/apache/spark/pull/42481

   
   
   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'core/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   This PR wraps the catch-block with a new execution id to QueryExecution.assertAnalyzed. It will reuse `SQLExecution.withNewExecutionId` to produce execution events to the listener and UI.
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   The listener and UI are not able to track analyzing failed queries
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   Yes. UI improvements.
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   
   new tests
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1316655772


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -124,7 +136,7 @@ object SQLExecution {
             physicalPlanDescription = queryExecution.explainString(planDescriptionMode),
             // `queryExecution.executedPlan` triggers query planning. If it fails, the exception
             // will be caught and reported in the `SparkListenerSQLExecutionEnd`

Review Comment:
   Yeah, I'll make a followup



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on PR #42481:
URL: https://github.com/apache/spark/pull/42481#issuecomment-1676912484

   updated


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1316776270


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -116,6 +119,15 @@ object SQLExecution {
         var ex: Option[Throwable] = None
         val startTime = System.nanoTime()
         try {
+          val planInfo = try {
+            SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan)
+          } catch {
+            case NonFatal(e) =>
+              logDebug("Failed to generate SparkPlanInfo", e)

Review Comment:
   one more followup: I think we should add `ex = Some(e)` here and skip running the `body` if `ex.isDefined`. The reason is to avoid triggering the failed analysis twice, which may introduce repeated error logging.
   ```
   ...
   if (ex.isEmpty) body else throw ex.get
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on PR #42481:
URL: https://github.com/apache/spark/pull/42481#issuecomment-1691907710

   thanks @cloud-fan @dongjoon-hyun @LuciferYang @MaxGekk @HyukjinKwon 
   
   Merged to master


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1294130406


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -65,7 +65,8 @@ object SQLExecution {
    */
   def withNewExecutionId[T](
       queryExecution: QueryExecution,
-      name: Option[String] = None)(body: => T): T = queryExecution.sparkSession.withActive {
+      name: Option[String] = None,
+      error: Option[Throwable] = None)(body: => T): T = queryExecution.sparkSession.withActive {

Review Comment:
   hmm, I think `queryExecution.executedPlan` will fail and the start event won't be sent. It seems a simpler fix is to update `SQLAppStatusListener` to accept `SparkListenerSQLExecutionEnd` without the start event, which means the query failed at analysis/planning time.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1294420509


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala:
##########
@@ -63,7 +63,14 @@ class QueryExecution(
   // TODO: Move the planner an optimizer into here from SessionState.
   protected def planner = sparkSession.sessionState.planner
 
-  def assertAnalyzed(): Unit = analyzed
+  def assertAnalyzed(): Unit = {
+    try {
+      analyzed
+    } catch {
+      case e: AnalysisException =>
+        SQLExecution.withNewExecutionId(this, Some("analyze"))(throw e)

Review Comment:
   is this change necessary?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on PR #42481:
URL: https://github.com/apache/spark/pull/42481#issuecomment-1676810662

   cc @sarutak @cloud-fan @HyukjinKwon @dongjoon-hyun , thanks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1294149812


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -65,7 +65,8 @@ object SQLExecution {
    */
   def withNewExecutionId[T](
       queryExecution: QueryExecution,
-      name: Option[String] = None)(body: => T): T = queryExecution.sparkSession.withActive {
+      name: Option[String] = None,
+      error: Option[Throwable] = None)(body: => T): T = queryExecution.sparkSession.withActive {

Review Comment:
   I was contemplating the option of sending the end event directly. I don't because the start event could contain the necessary information for the analysis failure reason. e.g., a modified configuration



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1316792164


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -116,6 +119,15 @@ object SQLExecution {
         var ex: Option[Throwable] = None
         val startTime = System.nanoTime()
         try {
+          val planInfo = try {
+            SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan)
+          } catch {
+            case NonFatal(e) =>
+              logDebug("Failed to generate SparkPlanInfo", e)

Review Comment:
   This doesn't seem right. If the `boby` is from an analysis error, `ex = Some(e)` would be a deterministic step that prevents execute `body` 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1297942500


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala:
##########
@@ -63,7 +63,14 @@ class QueryExecution(
   // TODO: Move the planner an optimizer into here from SessionState.
   protected def planner = sparkSession.sessionState.planner
 
-  def assertAnalyzed(): Unit = analyzed
+  def assertAnalyzed(): Unit = {
+    try {
+      analyzed
+    } catch {
+      case e: AnalysisException =>
+        SQLExecution.withNewExecutionId(this, Some("analyze"))(throw e)

Review Comment:
   oh I see what you mean, for simple `df.select(...)`, no `SQLExecution` is created. We need to specifically create an executio here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1294161922


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -65,7 +65,8 @@ object SQLExecution {
    */
   def withNewExecutionId[T](
       queryExecution: QueryExecution,
-      name: Option[String] = None)(body: => T): T = queryExecution.sparkSession.withActive {
+      name: Option[String] = None,
+      error: Option[Throwable] = None)(body: => T): T = queryExecution.sparkSession.withActive {

Review Comment:
   OK, then shall we update the `finally` branch to send the start event as well if it was not sent before? I feel it's awkward to ask the caller side to indicate if it will fail the analysis or not. `def withNewExecutionId` should be smarter about it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1294369640


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -65,7 +65,8 @@ object SQLExecution {
    */
   def withNewExecutionId[T](
       queryExecution: QueryExecution,
-      name: Option[String] = None)(body: => T): T = queryExecution.sparkSession.withActive {
+      name: Option[String] = None,
+      error: Option[Throwable] = None)(body: => T): T = queryExecution.sparkSession.withActive {

Review Comment:
   Updated to wrap planInfo with `try-catch`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1316842538


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -124,7 +136,7 @@ object SQLExecution {
             physicalPlanDescription = queryExecution.explainString(planDescriptionMode),

Review Comment:
   shall we use empty string here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] cloud-fan commented on pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #42481:
URL: https://github.com/apache/spark/pull/42481#issuecomment-1676891006

   can we have some screenshots to show this new feature?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] LuciferYang commented on pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on PR #42481:
URL: https://github.com/apache/spark/pull/42481#issuecomment-1677338653

   <img width="982" alt="image" src="https://github.com/apache/spark/assets/1475305/697cde84-4b93-4646-855c-d441cb05fb7f">
   @yaooqinn 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] dongjoon-hyun commented on pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on PR #42481:
URL: https://github.com/apache/spark/pull/42481#issuecomment-1687059412

   The revert PR landed at https://github.com/apache/spark/commit/b9c4fa4bc3129a352c443be04ef88f3ea71ae99d


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn closed pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn closed pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI
URL: https://github.com/apache/spark/pull/42481


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1316829618


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -116,6 +119,15 @@ object SQLExecution {
         var ex: Option[Throwable] = None
         val startTime = System.nanoTime()
         try {
+          val planInfo = try {
+            SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan)
+          } catch {
+            case NonFatal(e) =>
+              logDebug("Failed to generate SparkPlanInfo", e)

Review Comment:
   To avoid 2x analysis for failed ones seems that we need to revert to the first commit of PR which passes the `e` in to completely avoid call `SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan)`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1293146020


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -65,7 +65,8 @@ object SQLExecution {
    */
   def withNewExecutionId[T](
       queryExecution: QueryExecution,
-      name: Option[String] = None)(body: => T): T = queryExecution.sparkSession.withActive {
+      name: Option[String] = None,
+      error: Option[Throwable] = None)(body: => T): T = queryExecution.sparkSession.withActive {

Review Comment:
   To skip getting physical plan info



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1316817958


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -116,6 +119,15 @@ object SQLExecution {
         var ex: Option[Throwable] = None
         val startTime = System.nanoTime()
         try {
+          val planInfo = try {
+            SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan)
+          } catch {
+            case NonFatal(e) =>
+              logDebug("Failed to generate SparkPlanInfo", e)

Review Comment:
   I think I know where the problem is: We already triggered the analysis and failed, and then invoke `withNewExecutionId`. However, within `withNewExecutionId`, we don't know that the analysis has failed, and called `queryExecution.executedPlan` which triggers the analysis again.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1316776270


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -116,6 +119,15 @@ object SQLExecution {
         var ex: Option[Throwable] = None
         val startTime = System.nanoTime()
         try {
+          val planInfo = try {
+            SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan)
+          } catch {
+            case NonFatal(e) =>
+              logDebug("Failed to generate SparkPlanInfo", e)

Review Comment:
   one more followup: I think we should add `ex = Some(e)` here and skip running the `body` if `ex.isDefined`. The reason is to avoid triggering the failed analysis twice, which may introduce repeated error logging.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -116,6 +119,15 @@ object SQLExecution {
         var ex: Option[Throwable] = None
         val startTime = System.nanoTime()
         try {
+          val planInfo = try {
+            SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan)
+          } catch {
+            case NonFatal(e) =>
+              logDebug("Failed to generate SparkPlanInfo", e)

Review Comment:
   cc @yaooqinn 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] dongjoon-hyun commented on pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on PR #42481:
URL: https://github.com/apache/spark/pull/42481#issuecomment-1687028634

   Ya, it seems that the last commit didn't pass the CIs due to the same failure. Thanks, @MaxGekk .


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on PR #42481:
URL: https://github.com/apache/spark/pull/42481#issuecomment-1687474449

   ```scala
   scala> val df =spark.sql("select regexp_replace('', '[a\\\\d]{0, 2}', 'x')")
   df: org.apache.spark.sql.DataFrame = [regexp_replace(, [a\d]{0, 2}, x, 1): string]
   
   scala> val qe = df.queryExecution
   org.apache.spark.SparkRuntimeException: [INVALID_PARAMETER_VALUE.PATTERN] The value of parameter(s) `regexp` in `regexp_replace` is invalid: '[a\\d]{0, 2}'.
     at org.apache.spark.sql.errors.QueryExecutionErrors$.invalidPatternError(QueryExecutionErrors.scala:2754)
     at org.apache.spark.sql.catalyst.expressions.RegExpReplace.nullSafeEval(regexpExpressions.scala:646)
     at org.apache.spark.sql.catalyst.expressions.QuaternaryExpression.eval(Expression.scala:920)
     at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:80)
     at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.$anonfun$constantFolding$4(expressions.scala:90)
     at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1249)
     at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1248)
     at org.apache.spark.sql.catalyst.expressions.UnaryExpression.mapChildren(Expression.scala:532)
     at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:90)
     at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.$anonfun$applyOrElse$1(expressions.scala:94)
     at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$1(QueryPlan.scala:207)
     at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:104)
     at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:207)
     at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:218)
     at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$3(QueryPlan.scala:223)
     at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:286)
     at scala.collection.immutable.List.foreach(List.scala:431)
     at scala.collection.TraversableLike.map(TraversableLike.scala:286)
     at scala.collection.TraversableLike.map$(TraversableLike.scala:279)
     at scala.collection.immutable.List.map(List.scala:305)
     at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:223)
     at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$4(QueryPlan.scala:228)
     at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:355)
     at org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:228)
     at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:94)
     at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:93)
     at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:512)
     at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:104)
     at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:512)
     at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:31)
     at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
     at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
     at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:31)
     at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:31)
     at org.apache.spark.sql.catalyst.trees.TreeNode.transformWithPruning(TreeNode.scala:478)
     at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:93)
     at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:46)
     at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:222)
     at scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126)
     at scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122)
     at scala.collection.immutable.List.foldLeft(List.scala:91)
     at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:219)
     at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:211)
     at scala.collection.immutable.List.foreach(List.scala:431)
     at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:211)
     at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:182)
     at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88)
     at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:182)
     at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:143)
     at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
     at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:202)
     at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:526)
     at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:202)
     at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:827)
     at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:201)
     at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:139)
     at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:135)
     at org.apache.spark.sql.execution.QueryExecution.$anonfun$writePlans$4(QueryExecution.scala:285)
     at org.apache.spark.sql.catalyst.plans.QueryPlan$.append(QueryPlan.scala:671)
     at org.apache.spark.sql.execution.QueryExecution.writePlans(QueryExecution.scala:285)
     at org.apache.spark.sql.execution.QueryExecution.toString(QueryExecution.scala:302)
     at org.apache.spark.sql.execution.QueryExecution.toString(QueryExecution.scala:295)
     at scala.runtime.ScalaRunTime$.inner$1(ScalaRunTime.scala:272)
     at scala.runtime.ScalaRunTime$.stringOf(ScalaRunTime.scala:277)
     at scala.runtime.ScalaRunTime$.replStringOf(ScalaRunTime.scala:285)
     at .lzycompute(<console>:9)
     at .$print(<console>:6)
     at $print(<console>)
     at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
     at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
     at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
     at java.lang.reflect.Method.invoke(Method.java:498)
     at scala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:747)
     at scala.tools.nsc.interpreter.IMain$Request.loadAndRun(IMain.scala:1020)
     at scala.tools.nsc.interpreter.IMain.$anonfun$interpret$1(IMain.scala:568)
     at scala.reflect.internal.util.ScalaClassLoader.asContext(ScalaClassLoader.scala:36)
     at scala.reflect.internal.util.ScalaClassLoader.asContext$(ScalaClassLoader.scala:116)
     at scala.reflect.internal.util.AbstractFileClassLoader.asContext(AbstractFileClassLoader.scala:41)
     at scala.tools.nsc.interpreter.IMain.loadAndRunReq$1(IMain.scala:567)
     at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:594)
     at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:564)
     at scala.tools.nsc.interpreter.ILoop.interpretStartingWith(ILoop.scala:865)
     at scala.tools.nsc.interpreter.ILoop.command(ILoop.scala:733)
     at scala.tools.nsc.interpreter.ILoop.processLine(ILoop.scala:435)
     at scala.tools.nsc.interpreter.ILoop.loop(ILoop.scala:456)
     at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:239)
     at org.apache.spark.repl.Main$.doMain(Main.scala:78)
     at org.apache.spark.repl.Main$.main(Main.scala:58)
     at org.apache.spark.repl.Main.main(Main.scala)
     at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
     at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
     at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
     at java.lang.reflect.Method.invoke(Method.java:498)
     at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
     at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:1020)
     at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:192)
     at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:215)
     at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:91)
     at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1111)
     at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1120)
     at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
   Caused by: java.util.regex.PatternSyntaxException: Unclosed counted closure near index 8
   [a\d]{0, 2}
           ^
     at java.util.regex.Pattern.error(Pattern.java:1969)
     at java.util.regex.Pattern.closure(Pattern.java:3155)
     at java.util.regex.Pattern.sequence(Pattern.java:2148)
     at java.util.regex.Pattern.expr(Pattern.java:2010)
     at java.util.regex.Pattern.compile(Pattern.java:1702)
     at java.util.regex.Pattern.<init>(Pattern.java:1352)
     at java.util.regex.Pattern.compile(Pattern.java:1028)
     at org.apache.spark.sql.catalyst.expressions.RegExpReplace.nullSafeEval(regexpExpressions.scala:643)
     ... 99 more
   
   scala> val qe = df.queryExecution.e
   ensuring   eq   equals   executedPlan   explainString
   
   scala> val qe = df.queryExecution.explainString(org.apache.spark.sql.execution.FormattedMode)
   org.apache.spark.SparkException: [INTERNAL_ERROR] The Spark SQL phase optimization failed with an internal error. You hit a bug in Spark or the Spark plugins you use. Please, report this bug to the corresponding communities or vendors, and provide the full stack trace.
     at org.apache.spark.SparkException$.internalError(SparkException.scala:88)
     at org.apache.spark.sql.execution.QueryExecution$.toInternalError(QueryExecution.scala:516)
     at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:528)
     at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:202)
     at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:827)
     at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:201)
     at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:139)
     at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:135)
     at org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:153)
     at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:171)
     at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:168)
     at org.apache.spark.sql.execution.QueryExecution.simpleString(QueryExecution.scala:221)
     at org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$explainString(QueryExecution.scala:266)
     at org.apache.spark.sql.execution.QueryExecution.explainString(QueryExecution.scala:235)
     ... 47 elided
   Caused by: java.lang.NullPointerException
     at org.apache.spark.sql.catalyst.expressions.RegExpReplace.nullSafeEval(regexpExpressions.scala:657)
     at org.apache.spark.sql.catalyst.expressions.QuaternaryExpression.eval(Expression.scala:920)
     at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:80)
     at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.$anonfun$constantFolding$4(expressions.scala:90)
     at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1249)
     at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1248)
     at org.apache.spark.sql.catalyst.expressions.UnaryExpression.mapChildren(Expression.scala:532)
     at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.org$apache$spark$sql$catalyst$optimizer$ConstantFolding$$constantFolding(expressions.scala:90)
     at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.$anonfun$applyOrElse$1(expressions.scala:94)
     at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$1(QueryPlan.scala:207)
     at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:104)
     at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpression$1(QueryPlan.scala:207)
     at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:218)
     at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$3(QueryPlan.scala:223)
     at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:286)
     at scala.collection.immutable.List.foreach(List.scala:431)
     at scala.collection.TraversableLike.map(TraversableLike.scala:286)
     at scala.collection.TraversableLike.map$(TraversableLike.scala:279)
     at scala.collection.immutable.List.map(List.scala:305)
     at org.apache.spark.sql.catalyst.plans.QueryPlan.recursiveTransform$1(QueryPlan.scala:223)
     at org.apache.spark.sql.catalyst.plans.QueryPlan.$anonfun$mapExpressions$4(QueryPlan.scala:228)
     at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:355)
     at org.apache.spark.sql.catalyst.plans.QueryPlan.mapExpressions(QueryPlan.scala:228)
     at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:94)
     at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1.applyOrElse(expressions.scala:93)
     at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:512)
     at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:104)
     at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:512)
     at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:31)
     at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
     at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
     at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:31)
     at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:31)
     at org.apache.spark.sql.catalyst.trees.TreeNode.transformWithPruning(TreeNode.scala:478)
     at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:93)
     at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$.apply(expressions.scala:46)
     at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:222)
     at scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126)
     at scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122)
     at scala.collection.immutable.List.foldLeft(List.scala:91)
     at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:219)
     at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:211)
     at scala.collection.immutable.List.foreach(List.scala:431)
     at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:211)
     at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:182)
     at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:88)
     at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:182)
     at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:143)
     at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
     at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:202)
     at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:526)
     ... 58 more
   ```
   
   After some investigations with spark 3.4.1, this PR just hit a potential bug that we already have


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1293142773


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -65,7 +65,8 @@ object SQLExecution {
    */
   def withNewExecutionId[T](
       queryExecution: QueryExecution,
-      name: Option[String] = None)(body: => T): T = queryExecution.sparkSession.withActive {
+      name: Option[String] = None,
+      error: Option[Throwable] = None)(body: => T): T = queryExecution.sparkSession.withActive {

Review Comment:
   why is it useful to have this new parameter?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1293527428


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -65,7 +65,8 @@ object SQLExecution {
    */
   def withNewExecutionId[T](
       queryExecution: QueryExecution,
-      name: Option[String] = None)(body: => T): T = queryExecution.sparkSession.withActive {
+      name: Option[String] = None,
+      error: Option[Throwable] = None)(body: => T): T = queryExecution.sparkSession.withActive {

Review Comment:
   `body` executes later than the start event. Or we can use a boolen here instead



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on PR #42481:
URL: https://github.com/apache/spark/pull/42481#issuecomment-1689179057

   The related issue that causes CI failure has been fixed at https://github.com/apache/spark/pull/42601.
   
   As the code change stays AS-IS, I propose that this PR will be merged after all tests pass with sufficient obtained approvals. Please let me know if you guys have any concerns.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1316797688


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -116,6 +119,15 @@ object SQLExecution {
         var ex: Option[Throwable] = None
         val startTime = System.nanoTime()
         try {
+          val planInfo = try {
+            SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan)
+          } catch {
+            case NonFatal(e) =>
+              logDebug("Failed to generate SparkPlanInfo", e)

Review Comment:
   nvm, `body` is just a `throw ...` statement so no need to save the execution.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1316845213


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -116,6 +119,15 @@ object SQLExecution {
         var ex: Option[Throwable] = None
         val startTime = System.nanoTime()
         try {
+          val planInfo = try {
+            SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan)
+          } catch {
+            case NonFatal(e) =>
+              logDebug("Failed to generate SparkPlanInfo", e)

Review Comment:
   Yea I agree. Sorry for the back and forth! One idea to make the API better: the `body` can be `Either[() => T, Throwable]`, as these two are mutually exclusive



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1294429218


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala:
##########
@@ -63,7 +63,14 @@ class QueryExecution(
   // TODO: Move the planner an optimizer into here from SessionState.
   protected def planner = sparkSession.sessionState.planner
 
-  def assertAnalyzed(): Unit = analyzed
+  def assertAnalyzed(): Unit = {
+    try {
+      analyzed
+    } catch {
+      case e: AnalysisException =>
+        SQLExecution.withNewExecutionId(this, Some("analyze"))(throw e)

Review Comment:
   This is the key change of this PR to capture failures from analysis



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1293527428


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -65,7 +65,8 @@ object SQLExecution {
    */
   def withNewExecutionId[T](
       queryExecution: QueryExecution,
-      name: Option[String] = None)(body: => T): T = queryExecution.sparkSession.withActive {
+      name: Option[String] = None,
+      error: Option[Throwable] = None)(body: => T): T = queryExecution.sparkSession.withActive {

Review Comment:
   `body` executes later than the start event



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] cloud-fan commented on pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on PR #42481:
URL: https://github.com/apache/spark/pull/42481#issuecomment-1683758931

   cc @rednaxelafx 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] MaxGekk commented on pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on PR #42481:
URL: https://github.com/apache/spark/pull/42481#issuecomment-1686939506

   Seems like the changes cause an internal error on the example:
   ```
   sql("select regexp_replace('', '[a\\\\d]{0, 2}', 'x')").collect()
   ```
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on PR #42481:
URL: https://github.com/apache/spark/pull/42481#issuecomment-1687341484

   thanks for recovering @MaxGekk @dongjoon-hyun, I will take a look at the failure


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1315972356


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -124,7 +136,7 @@ object SQLExecution {
             physicalPlanDescription = queryExecution.explainString(planDescriptionMode),
             // `queryExecution.executedPlan` triggers query planning. If it fails, the exception
             // will be caught and reported in the `SparkListenerSQLExecutionEnd`

Review Comment:
   shall we remove this comment now?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1316826857


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -116,6 +119,15 @@ object SQLExecution {
         var ex: Option[Throwable] = None
         val startTime = System.nanoTime()
         try {
+          val planInfo = try {
+            SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan)
+          } catch {
+            case NonFatal(e) =>
+              logDebug("Failed to generate SparkPlanInfo", e)

Review Comment:
   Ok, I get your point, but I still don't understand - 
   > The reason is to avoid triggering the failed analysis twice
   
   It seems that we just pick `e` from `body` or `e` from `SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan)`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1316854070


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -116,6 +119,15 @@ object SQLExecution {
         var ex: Option[Throwable] = None
         val startTime = System.nanoTime()
         try {
+          val planInfo = try {
+            SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan)
+          } catch {
+            case NonFatal(e) =>
+              logDebug("Failed to generate SparkPlanInfo", e)

Review Comment:
   OK, I will try `Either`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn closed pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn closed pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI
URL: https://github.com/apache/spark/pull/42481


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on PR #42481:
URL: https://github.com/apache/spark/pull/42481#issuecomment-1685688005

   thanks, merged to master


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1298312925


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala:
##########
@@ -63,7 +63,14 @@ class QueryExecution(
   // TODO: Move the planner an optimizer into here from SessionState.
   protected def planner = sparkSession.sessionState.planner
 
-  def assertAnalyzed(): Unit = analyzed
+  def assertAnalyzed(): Unit = {
+    try {
+      analyzed
+    } catch {
+      case e: AnalysisException =>
+        SQLExecution.withNewExecutionId(this, Some("analyze"))(throw e)

Review Comment:
   updated



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1297942642


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala:
##########
@@ -63,7 +63,14 @@ class QueryExecution(
   // TODO: Move the planner an optimizer into here from SessionState.
   protected def planner = sparkSession.sessionState.planner
 
-  def assertAnalyzed(): Unit = analyzed
+  def assertAnalyzed(): Unit = {
+    try {
+      analyzed
+    } catch {
+      case e: AnalysisException =>
+        SQLExecution.withNewExecutionId(this, Some("analyze"))(throw e)

Review Comment:
   The reason is we do eager analysis for dataframe. Can we add some comments to explain it?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1297942075


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala:
##########
@@ -63,7 +63,14 @@ class QueryExecution(
   // TODO: Move the planner an optimizer into here from SessionState.
   protected def planner = sparkSession.sessionState.planner
 
-  def assertAnalyzed(): Unit = analyzed
+  def assertAnalyzed(): Unit = {
+    try {
+      analyzed
+    } catch {
+      case e: AnalysisException =>
+        SQLExecution.withNewExecutionId(this, Some("analyze"))(throw e)

Review Comment:
   shouldn't `SQLExecution` be the only place to try-catch the errors? including analysis errors.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1316852917


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -124,7 +136,7 @@ object SQLExecution {
             physicalPlanDescription = queryExecution.explainString(planDescriptionMode),

Review Comment:
   Yeah, it's okay to use an empty string instead of `parsed + e` 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] yaooqinn commented on pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on PR #42481:
URL: https://github.com/apache/spark/pull/42481#issuecomment-1677388591

   It's the viz of SparkPlanInfo.EMPTY


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #42481: [SPARK-44801][SQL][UI] Capture analyzing failed queries in Listener and UI

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #42481:
URL: https://github.com/apache/spark/pull/42481#discussion_r1293484042


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala:
##########
@@ -65,7 +65,8 @@ object SQLExecution {
    */
   def withNewExecutionId[T](
       queryExecution: QueryExecution,
-      name: Option[String] = None)(body: => T): T = queryExecution.sparkSession.withActive {
+      name: Option[String] = None,
+      error: Option[Throwable] = None)(body: => T): T = queryExecution.sparkSession.withActive {

Review Comment:
   But the `body` will throw an exception and then be caught later. Why do we have this duplicated exception in the parameter?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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