Skip to content
Closed
Show file tree
Hide file tree
Changes from 3 commits
Commits
Show all changes
33 commits
Select commit Hold shift + click to select a range
e93cedd
tags api
xupefei Aug 20, 2024
694db1b
Merge branch 'master' of github.com:apache/spark into reverse-api-tag
xupefei Aug 20, 2024
40610a7
rename
xupefei Aug 20, 2024
a70d7d2
address comments
xupefei Aug 21, 2024
0656e25
.
xupefei Aug 21, 2024
6b6ca7f
.
xupefei Aug 21, 2024
d3cd5f5
new approach
xupefei Aug 23, 2024
0922dd2
address comments
xupefei Aug 26, 2024
2a6fcc6
return job IDs earlier
xupefei Aug 26, 2024
ef0fddf
doc
xupefei Aug 26, 2024
f2ad163
no mention of spark session in core
xupefei Aug 27, 2024
ab00685
re
xupefei Aug 27, 2024
dd10f46
fix test
xupefei Aug 27, 2024
bc9b76d
revert some changes
xupefei Aug 28, 2024
8656810
undo
xupefei Aug 28, 2024
1dfafad
wip
xupefei Aug 28, 2024
1d4d5cc
.
xupefei Aug 29, 2024
a35c4e5
Merge branch 'master' of github.com:apache/spark into reverse-api-tag
xupefei Aug 29, 2024
d1208c4
revert unnessesary changes and fix tests
xupefei Aug 29, 2024
13342cf
comment
xupefei Aug 29, 2024
3879989
oh no
xupefei Aug 29, 2024
cf6437f
remove internal tags
xupefei Aug 30, 2024
4d7da3b
Merge branch 'master' of github.com:apache/spark into reverse-api-tag
xupefei Aug 30, 2024
b3b7cbc
test
xupefei Aug 30, 2024
7c9294e
Merge branch 'master' of github.com:apache/spark into reverse-api-tag
xupefei Aug 30, 2024
7338b1d
move doc to api
xupefei Aug 30, 2024
905bf91
fix test
xupefei Sep 3, 2024
514b5e4
address mridulm's comments
xupefei Sep 10, 2024
c6fb41f
address herman's comments
xupefei Sep 10, 2024
2a0292c
address hyukjin's comment
xupefei Sep 10, 2024
2d059b3
Merge branch 'master' of github.com:apache/spark into reverse-api-tag
xupefei Sep 10, 2024
a55c47c
scalastyle
xupefei Sep 16, 2024
e66ba0a
fmt
xupefei Sep 17, 2024
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 @@ -741,7 +741,7 @@ class SparkSession private[sql] (
* Often, a unit of execution in an application consists of multiple Spark executions.
* Application programmers can use this method to group all those jobs together and give a group
* tag. The application can use `org.apache.spark.sql.SparkSession.interruptTag` to cancel all
* running running executions with this tag. For example:
* running executions with this tag. For example:
* {{{
* // In the main thread:
* spark.addTag("myjobs")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -408,21 +408,6 @@ object CheckConnectJvmClientCompatibility {
"org.apache.spark.sql.SparkSession.addArtifacts"),
ProblemFilters.exclude[DirectMissingMethodProblem](
"org.apache.spark.sql.SparkSession.registerClassFinder"),
// public
ProblemFilters.exclude[DirectMissingMethodProblem](
"org.apache.spark.sql.SparkSession.interruptAll"),
ProblemFilters.exclude[DirectMissingMethodProblem](
"org.apache.spark.sql.SparkSession.interruptTag"),
ProblemFilters.exclude[DirectMissingMethodProblem](
"org.apache.spark.sql.SparkSession.interruptOperation"),
ProblemFilters.exclude[DirectMissingMethodProblem](
"org.apache.spark.sql.SparkSession.addTag"),
ProblemFilters.exclude[DirectMissingMethodProblem](
"org.apache.spark.sql.SparkSession.removeTag"),
ProblemFilters.exclude[DirectMissingMethodProblem](
"org.apache.spark.sql.SparkSession.getTags"),
ProblemFilters.exclude[DirectMissingMethodProblem](
"org.apache.spark.sql.SparkSession.clearTags"),
// SparkSession#Builder
ProblemFilters.exclude[DirectMissingMethodProblem](
"org.apache.spark.sql.SparkSession#Builder.remote"),
Expand Down
22 changes: 22 additions & 0 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -2684,6 +2684,22 @@ class SparkContext(config: SparkConf) extends Logging {
dagScheduler.cancelJobGroup(groupId, cancelFutureJobs = true, None)
}

/**
* Cancel active jobs that have the specified tag. See `org.apache.spark.SparkContext.addJobTag`.
*
* @param tag The tag to be cancelled. Cannot contain ',' (comma) character.
* @param reason reason for cancellation
* @param jobIdCallback callback function to be called with the job ID of each job that is being
* cancelled.
*
* @since 4.0.0
*/
def cancelJobsWithTag(tag: String, reason: String, jobIdCallback: Int => Unit): Unit = {
SparkContext.throwIfInvalidTag(tag)
assertNotStopped()
dagScheduler.cancelJobsWithTag(tag, Option(reason))
}

/**
* Cancel active jobs that have the specified tag. See `org.apache.spark.SparkContext.addJobTag`.
*
Expand Down Expand Up @@ -2717,6 +2733,12 @@ class SparkContext(config: SparkConf) extends Logging {
dagScheduler.cancelAllJobs()
}

/** Cancel all jobs that have been scheduled or are running. */
def cancelAllJobs(jobIdCallback: Int => Unit): Unit = {
assertNotStopped()
dagScheduler.cancelAllJobs(Some(jobIdCallback))
}

/**
* Cancel a given job if it's scheduled or running.
*
Expand Down
55 changes: 38 additions & 17 deletions core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1117,23 +1117,37 @@ private[spark] class DAGScheduler(
/**
* Cancel all jobs with a given tag.
*/
def cancelJobsWithTag(tag: String, reason: Option[String]): Unit = {
def cancelJobsWithTag(tag: String, reason: Option[String]): Unit =
cancelJobsWithTag(tag, reason, jobIdCallback = None)

/**
* Cancel all jobs with a given tag.
*/
def cancelJobsWithTag(
tag: String,
reason: Option[String],
jobIdCallback: Option[Int => Unit]): Unit = {
SparkContext.throwIfInvalidTag(tag)
logInfo(log"Asked to cancel jobs with tag ${MDC(TAG, tag)}")
eventProcessLoop.post(JobTagCancelled(tag, reason))
eventProcessLoop.post(JobTagCancelled(tag, reason, jobIdCallback))
}

/**
* Cancel all jobs that are running or waiting in the queue.
*/
def cancelAllJobs(): Unit = {
eventProcessLoop.post(AllJobsCancelled)
def cancelAllJobs(): Unit = cancelAllJobs(jobIdCallback = None)

/**
* Cancel all jobs that are running or waiting in the queue.
*/
def cancelAllJobs(jobIdCallback: Option[Int => Unit]): Unit = {
eventProcessLoop.post(AllJobsCancelled(jobIdCallback))
}

private[scheduler] def doCancelAllJobs(): Unit = {
private[spark] def doCancelAllJobs(jobIdCallback: Option[Int => Unit]): Unit = {
// Cancel all running jobs.
runningStages.map(_.firstJobId).foreach(handleJobCancellation(_,
Option("as part of cancellation of all jobs")))
Option("as part of cancellation of all jobs"), jobIdCallback))
activeJobs.clear() // These should already be empty by this point,
jobIdToActiveJob.clear() // but just in case we lost track of some jobs...
}
Expand Down Expand Up @@ -1231,10 +1245,13 @@ private[spark] class DAGScheduler(
}
val jobIds = activeInGroup.map(_.jobId)
val updatedReason = reason.getOrElse("part of cancelled job group %s".format(groupId))
jobIds.foreach(handleJobCancellation(_, Option(updatedReason)))
jobIds.foreach(handleJobCancellation(_, Option(updatedReason), jobIdCallback = None))
}

private[scheduler] def handleJobTagCancelled(tag: String, reason: Option[String]): Unit = {
private[scheduler] def handleJobTagCancelled(
tag: String,
reason: Option[String],
jobIdCallback: Option[Int => Unit]): Unit = {
// Cancel all jobs belonging that have this tag.
// First finds all active jobs with this group id, and then kill stages for them.
val jobIds = activeJobs.filter { activeJob =>
Expand All @@ -1244,7 +1261,7 @@ private[spark] class DAGScheduler(
}
}.map(_.jobId)
val updatedReason = reason.getOrElse("part of cancelled job tag %s".format(tag))
jobIds.foreach(handleJobCancellation(_, Option(updatedReason)))
jobIds.foreach(handleJobCancellation(_, Option(updatedReason), jobIdCallback))
}

private[scheduler] def handleBeginEvent(task: Task[_], taskInfo: TaskInfo): Unit = {
Expand Down Expand Up @@ -2801,21 +2818,25 @@ private[spark] class DAGScheduler(
case None =>
s"because Stage $stageId was cancelled"
}
handleJobCancellation(jobId, Option(reasonStr))
handleJobCancellation(jobId, Option(reasonStr), jobIdCallback = None)
}
case None =>
logInfo(log"No active jobs to kill for Stage ${MDC(STAGE_ID, stageId)}")
}
}

private[scheduler] def handleJobCancellation(jobId: Int, reason: Option[String]): Unit = {
private[scheduler] def handleJobCancellation(
jobId: Int,
reason: Option[String],
jobIdCallback: Option[Int => Unit]): Unit = {
if (!jobIdToStageIds.contains(jobId)) {
logDebug("Trying to cancel unregistered job " + jobId)
} else {
failJobAndIndependentStages(
job = jobIdToActiveJob(jobId),
error = SparkCoreErrors.sparkJobCancelled(jobId, reason.getOrElse(""), null)
)
jobIdCallback.foreach(_(jobId))
}
}

Expand Down Expand Up @@ -3108,16 +3129,16 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler
dagScheduler.handleStageCancellation(stageId, reason)

case JobCancelled(jobId, reason) =>
dagScheduler.handleJobCancellation(jobId, reason)
dagScheduler.handleJobCancellation(jobId, reason, jobIdCallback = None)

case JobGroupCancelled(groupId, cancelFutureJobs, reason) =>
dagScheduler.handleJobGroupCancelled(groupId, cancelFutureJobs, reason)

case JobTagCancelled(tag, reason) =>
dagScheduler.handleJobTagCancelled(tag, reason)
case JobTagCancelled(tag, reason, callback) =>
dagScheduler.handleJobTagCancelled(tag, reason, callback)

case AllJobsCancelled =>
dagScheduler.doCancelAllJobs()
case AllJobsCancelled(callback) =>
dagScheduler.doCancelAllJobs(callback)

case ExecutorAdded(execId, host) =>
dagScheduler.handleExecutorAdded(execId, host)
Expand Down Expand Up @@ -3173,7 +3194,7 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler
override def onError(e: Throwable): Unit = {
logError("DAGSchedulerEventProcessLoop failed; shutting down SparkContext", e)
try {
dagScheduler.doCancelAllJobs()
dagScheduler.doCancelAllJobs(jobIdCallback = None)
} catch {
case t: Throwable => logError("DAGScheduler failed to cancel all jobs.", t)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,9 +71,11 @@ private[scheduler] case class JobGroupCancelled(

private[scheduler] case class JobTagCancelled(
tagName: String,
reason: Option[String]) extends DAGSchedulerEvent
reason: Option[String],
jobIdCallback: Option[Int => Unit]) extends DAGSchedulerEvent

private[scheduler] case object AllJobsCancelled extends DAGSchedulerEvent
private[scheduler] case class AllJobsCancelled(
jobIdCallback: Option[Int => Unit]) extends DAGSchedulerEvent

private[scheduler]
case class BeginEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent
Expand Down
103 changes: 103 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import java.util.{ServiceLoader, UUID}
import java.util.concurrent.TimeUnit._
import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference}

import scala.collection.mutable
import scala.jdk.CollectionConverters._
import scala.reflect.runtime.universe.TypeTag
import scala.util.control.NonFatal
Expand Down Expand Up @@ -794,6 +795,108 @@ class SparkSession private(
}
}


/**
* Add a tag to be assigned to all the operations started by this thread in this session.
*
* Often, a unit of execution in an application consists of multiple Spark executions.
* Application programmers can use this method to group all those jobs together and give a group
* tag. The application can use `org.apache.spark.sql.SparkSession.interruptTag` to cancel all
* running executions with this tag. For example:
* {{{
* // In the main thread:
* spark.addTag("myjobs")
* spark.range(10).map(i => { Thread.sleep(10); i }).collect()
*
* // In a separate thread:
* spark.interruptTag("myjobs")
* }}}
*
* There may be multiple tags present at the same time, so different parts of application may
* use different tags to perform cancellation at different levels of granularity.
*
* @param tag
* The tag to be added. Cannot contain ',' (comma) character or be an empty string.
*
* @since 4.0.0
*/
def addTag(tag: String): Unit = sparkContext.addJobTag(tag)

/**
* Remove a tag previously added to be assigned to all the operations started by this thread in
* this session. Noop if such a tag was not added earlier.
*
* @param tag
* The tag to be removed. Cannot contain ',' (comma) character or be an empty string.
*
* @since 4.0.0
*/
def removeTag(tag: String): Unit = sparkContext.removeJobTag(tag)

/**
* Get the tags that are currently set to be assigned to all the operations started by this
* thread.
*
* @since 4.0.0
*/
def getTags(): Set[String] = sparkContext.getJobTags()

/**
* Clear the current thread's operation tags.
*
* @since 4.0.0
*/
def clearTags(): Unit = sparkContext.clearJobTags()

/**
* Interrupt all operations of this session that are currently running.
*
* @return
* sequence of Job IDs of interrupted operations.
*
* @since 4.0.0
*/
def interruptAll(): Seq[String] = {
val jobIds = mutable.Set[Int]()
sparkContext.cancelAllJobs(jobIdCallback = (jobId: Int) => jobIds.add(jobId))
jobIds.toSeq.map(_.toString)
}

/**
* Interrupt all operations of this session with the given operation tag.
*
* @return
* sequence of Job IDs of interrupted operations.
*
* @since 4.0.0
*/
def interruptTag(tag: String): Seq[String] = {
val jobIds = mutable.Set[Int]()
sparkContext.cancelJobsWithTag(
tag,
"Interrupted by user",
jobIdCallback = (jobId: Int) => jobIds.add(jobId))
jobIds.toSeq.map(_.toString)
}

/**
* Interrupt an operation of this session with the given Job ID.
*
* @return
* sequence of Job IDs of interrupted operations.
*
* @since 4.0.0
*/
def interruptOperation(jobId: String): Seq[String] = {
scala.util.Try(jobId.toInt).toOption match {
case Some(id) =>
sparkContext.cancelJob(id, "Interrupted by user")
Seq(jobId)
case None =>
throw new IllegalArgumentException("jobId must be a number.")
}
}

/**
* Returns a [[DataFrameReader]] that can be used to read non-streaming data in as a
* `DataFrame`.
Expand Down