Skip to content
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

[SPARK-48716] Add jobGroupId to SparkListenerSQLExecutionStart #47092

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicLong
import scala.jdk.CollectionConverters._
import scala.util.control.NonFatal

import org.apache.spark.{ErrorMessageFormat, JobArtifactSet, SparkEnv, SparkException, SparkThrowable, SparkThrowableHelper}
import org.apache.spark.{ErrorMessageFormat, JobArtifactSet, SparkContext, SparkEnv, SparkException, SparkThrowable, SparkThrowableHelper}
import org.apache.spark.SparkContext.{SPARK_JOB_DESCRIPTION, SPARK_JOB_INTERRUPT_ON_CANCEL}
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config.{SPARK_DRIVER_PREFIX, SPARK_EXECUTOR_PREFIX}
Expand Down Expand Up @@ -128,7 +128,8 @@ object SQLExecution extends Logging {
sparkPlanInfo = SparkPlanInfo.EMPTY,
time = System.currentTimeMillis(),
modifiedConfigs = redactedConfigs,
jobTags = sc.getJobTags()
jobTags = sc.getJobTags(),
jobGroupId = Some(sc.getLocalProperty(SparkContext.SPARK_JOB_GROUP_ID))
)
try {
body match {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -343,7 +343,7 @@ class SQLAppStatusListener(

private def onExecutionStart(event: SparkListenerSQLExecutionStart): Unit = {
val SparkListenerSQLExecutionStart(executionId, rootExecutionId, description, details,
physicalPlanDescription, sparkPlanInfo, time, modifiedConfigs, _) = event
physicalPlanDescription, sparkPlanInfo, time, modifiedConfigs, _, _) = event

val planGraph = SparkPlanGraph(sparkPlanInfo)
val sqlPlanMetrics = planGraph.allNodes.flatMap { node =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,8 @@ case class SparkListenerSQLExecutionStart(
sparkPlanInfo: SparkPlanInfo,
time: Long,
modifiedConfigs: Map[String, String] = Map.empty,
jobTags: Set[String] = Set.empty)
jobTags: Set[String] = Set.empty,
jobGroupId: Option[String] = None)
extends SparkListenerEvent

@DeveloperApi
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -234,6 +234,38 @@ class SQLExecutionSuite extends SparkFunSuite with SQLConfHelper {
spark.stop()
}
}

test("jobGroupId property") {
val spark = SparkSession.builder().master("local[*]").appName("test").getOrCreate()
// Disable photon. The job below launches many tasks, which can cause Photon OOMs
val JobGroupId = "test-JobGroupId"
try {
spark.sparkContext.setJobGroup(JobGroupId, "job Group id")

var jobGroupIdOpt: Option[String] = None
var sqlJobGroupIdOpt: Option[String] = None
spark.sparkContext.addSparkListener(new SparkListener {
override def onJobStart(jobStart: SparkListenerJobStart): Unit = {
jobGroupIdOpt = Some(jobStart.properties.getProperty(SparkContext.SPARK_JOB_GROUP_ID))
}

override def onOtherEvent(event: SparkListenerEvent): Unit = {
event match {
case e: SparkListenerSQLExecutionStart =>
sqlJobGroupIdOpt = e.jobGroupId
}
}
})

spark.range(1).collect()

assert(jobGroupIdOpt.contains(JobGroupId))
assert(sqlJobGroupIdOpt.contains(JobGroupId))
} finally {
spark.sparkContext.clearJobGroup()
spark.stop()
}
}
}

object SQLExecutionSuite {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -344,7 +344,7 @@ abstract class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTes
val listener = new SparkListener {
override def onOtherEvent(event: SparkListenerEvent): Unit = {
event match {
case SparkListenerSQLExecutionStart(_, _, _, _, planDescription, _, _, _, _) =>
case SparkListenerSQLExecutionStart(_, _, _, _, planDescription, _, _, _, _, _) =>
assert(expected.forall(planDescription.contains))
checkDone = true
case _ => // ignore other events
Expand Down