-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-40834][SQL] Use SparkListenerSQLExecutionEnd to track final SQL status in UI #38302
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -56,7 +56,10 @@ case class SparkListenerSQLExecutionStart( | |
| } | ||
|
|
||
| @DeveloperApi | ||
| case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) | ||
| case class SparkListenerSQLExecutionEnd( | ||
| executionId: Long, | ||
| time: Long, | ||
| errorMessage: Option[String] = None) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thinking about this more, I think we should use
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I made a followup: #38747
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Doesn't this break binary compatibility? I think it should have overridden constructor.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is not a public class. It generates JSON string where backward compatibility matters, and this PR added tests for it. |
||
| extends SparkListenerEvent { | ||
|
|
||
| // The name of the execution, e.g. `df.collect` will trigger a SQL execution with name "collect". | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution | |
|
|
||
| import org.json4s.jackson.JsonMethods._ | ||
|
|
||
| import org.apache.spark.SparkFunSuite | ||
| import org.apache.spark.{SparkFunSuite, SparkThrowableHelper} | ||
| import org.apache.spark.scheduler.SparkListenerEvent | ||
| import org.apache.spark.sql.LocalSparkSession | ||
| import org.apache.spark.sql.execution.ui.{SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart} | ||
|
|
@@ -74,27 +74,60 @@ class SQLJsonProtocolSuite extends SparkFunSuite with LocalSparkSession { | |
| test("SparkListenerSQLExecutionEnd backward compatibility") { | ||
| spark = new TestSparkSession() | ||
| val qe = spark.sql("select 1").queryExecution | ||
| val event = SparkListenerSQLExecutionEnd(1, 10) | ||
| val errorMessage = SparkThrowableHelper.getMessage(new Exception("test")) | ||
| val event = SparkListenerSQLExecutionEnd(1, 10, Some(errorMessage)) | ||
| event.duration = 1000 | ||
| event.executionName = Some("test") | ||
| event.qe = qe | ||
| event.executionFailure = Some(new RuntimeException("test")) | ||
| event.executionFailure = Some(new Exception("test")) | ||
| val json = JsonProtocol.sparkEventToJsonString(event) | ||
| // scalastyle:off line.size.limit | ||
| assert(parse(json) == parse( | ||
| """ | ||
| |{ | ||
| | "Event" : "org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd", | ||
| | "executionId" : 1, | ||
| | "time" : 10 | ||
| | "time" : 10, | ||
| | "errorMessage" : "{\"errorClass\":\"java.lang.Exception\",\"messageParameters\":{\"message\":\"test\"}}" | ||
| |} | ||
| """.stripMargin)) | ||
| // scalastyle:on | ||
| val readBack = JsonProtocol.sparkEventFromJson(json) | ||
| event.duration = 0 | ||
| event.executionName = None | ||
| event.qe = null | ||
| event.executionFailure = None | ||
| assert(readBack == event) | ||
| } | ||
|
|
||
| test("SPARK-40834: Use SparkListenerSQLExecutionEnd to track final SQL status in UI") { | ||
| // parse old event log using new SparkListenerSQLExecutionEnd | ||
| val executionEnd = | ||
| """ | ||
| |{ | ||
| | "Event" : "org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd", | ||
| | "executionId" : 1, | ||
| | "time" : 10 | ||
| |} | ||
| """.stripMargin | ||
| val readBack = JsonProtocol.sparkEventFromJson(executionEnd) | ||
| assert(readBack == SparkListenerSQLExecutionEnd(1, 10)) | ||
|
|
||
| // parse new event using old SparkListenerSQLExecutionEnd | ||
| // scalastyle:off line.size.limit | ||
| val newExecutionEnd = | ||
| """ | ||
| |{ | ||
| | "Event" : "org.apache.spark.sql.execution.OldVersionSQLExecutionEnd", | ||
| | "executionId" : 1, | ||
| | "time" : 10, | ||
| | "errorMessage" : "{\"errorClass\":\"java.lang.Exception\",\"messageParameters\":{\"message\":\"test\"}}" | ||
|
||
| |} | ||
| """.stripMargin | ||
| // scalastyle:on | ||
| val readBack2 = JsonProtocol.sparkEventFromJson(newExecutionEnd) | ||
| assert(readBack2 == OldVersionSQLExecutionEnd(1, 10)) | ||
| } | ||
| } | ||
|
|
||
| private case class OldVersionSQLExecutionStart( | ||
|
|
@@ -105,3 +138,6 @@ private case class OldVersionSQLExecutionStart( | |
| sparkPlanInfo: SparkPlanInfo, | ||
| time: Long) | ||
| extends SparkListenerEvent | ||
|
|
||
| private case class OldVersionSQLExecutionEnd(executionId: Long, time: Long) | ||
| extends SparkListenerEvent | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@cloud-fan this follows the standard error msg json format