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-27963][core] Allow dynamic allocation without a shuffle service. #24817

Closed
wants to merge 11 commits into from
Closed
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 @@ -94,6 +94,7 @@ private[spark] class ExecutorAllocationManager(
client: ExecutorAllocationClient,
listenerBus: LiveListenerBus,
conf: SparkConf,
cleaner: Option[ContextCleaner] = None,
clock: Clock = new SystemClock())
extends Logging {

Expand Down Expand Up @@ -148,7 +149,7 @@ private[spark] class ExecutorAllocationManager(
// Listener for Spark events that impact the allocation policy
val listener = new ExecutorAllocationListener

val executorMonitor = new ExecutorMonitor(conf, client, clock)
val executorMonitor = new ExecutorMonitor(conf, client, listenerBus, clock)

// Executor that handles the scheduling task.
private val executor =
Expand Down Expand Up @@ -194,11 +195,13 @@ private[spark] class ExecutorAllocationManager(
throw new SparkException(
s"s${DYN_ALLOCATION_SUSTAINED_SCHEDULER_BACKLOG_TIMEOUT.key} must be > 0!")
}
// Require external shuffle service for dynamic allocation
// Otherwise, we may lose shuffle files when killing executors
if (!conf.get(config.SHUFFLE_SERVICE_ENABLED) && !testing) {
throw new SparkException("Dynamic allocation of executors requires the external " +
"shuffle service. You may enable this through spark.shuffle.service.enabled.")
if (!conf.get(config.SHUFFLE_SERVICE_ENABLED)) {
if (conf.get(config.DYN_ALLOCATION_SHUFFLE_TRACKING)) {
logWarning("Dynamic allocation without a shuffle service is an experimental feature.")
} else if (!testing) {
throw new SparkException("Dynamic allocation of executors requires the external " +
"shuffle service. You may enable this through spark.shuffle.service.enabled.")
}
}

if (executorAllocationRatio > 1.0 || executorAllocationRatio <= 0.0) {
Expand All @@ -214,6 +217,7 @@ private[spark] class ExecutorAllocationManager(
def start(): Unit = {
listenerBus.addToManagementQueue(listener)
listenerBus.addToManagementQueue(executorMonitor)
cleaner.foreach(_.attachListener(executorMonitor))

val scheduleTask = new Runnable() {
override def run(): Unit = {
Expand Down
20 changes: 10 additions & 10 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -553,14 +553,22 @@ class SparkContext(config: SparkConf) extends Logging {
None
}

// Optionally scale number of executors dynamically based on workload. Exposed for testing.
_cleaner =
if (_conf.get(CLEANER_REFERENCE_TRACKING)) {
Some(new ContextCleaner(this))
} else {
None
}
_cleaner.foreach(_.start())

val dynamicAllocationEnabled = Utils.isDynamicAllocationEnabled(_conf)
_executorAllocationManager =
if (dynamicAllocationEnabled) {
schedulerBackend match {
case b: ExecutorAllocationClient =>
Some(new ExecutorAllocationManager(
schedulerBackend.asInstanceOf[ExecutorAllocationClient], listenerBus, _conf))
schedulerBackend.asInstanceOf[ExecutorAllocationClient], listenerBus, _conf,
cleaner = cleaner))
case _ =>
None
}
Expand All @@ -569,14 +577,6 @@ class SparkContext(config: SparkConf) extends Logging {
}
_executorAllocationManager.foreach(_.start())

_cleaner =
if (_conf.get(CLEANER_REFERENCE_TRACKING)) {
Some(new ContextCleaner(this))
} else {
None
}
_cleaner.foreach(_.start())

setupAndStartListenerBus()
postEnvironmentUpdate()
postApplicationStart()
Expand Down
11 changes: 11 additions & 0 deletions core/src/main/scala/org/apache/spark/internal/config/package.scala
Original file line number Diff line number Diff line change
Expand Up @@ -369,6 +369,17 @@ package object config {
.checkValue(_ >= 0L, "Timeout must be >= 0.")
.createWithDefault(60)

private[spark] val DYN_ALLOCATION_SHUFFLE_TRACKING =
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

documenting these in configuration.md?

ConfigBuilder("spark.dynamicAllocation.shuffleTracking.enabled")
.booleanConf
.createWithDefault(false)

private[spark] val DYN_ALLOCATION_SHUFFLE_TIMEOUT =
ConfigBuilder("spark.dynamicAllocation.shuffleTimeout")
.timeConf(TimeUnit.MILLISECONDS)
.checkValue(_ >= 0L, "Timeout must be >= 0.")
.createWithDefault(Long.MaxValue)

private[spark] val DYN_ALLOCATION_SCHEDULER_BACKLOG_TIMEOUT =
ConfigBuilder("spark.dynamicAllocation.schedulerBacklogTimeout")
.timeConf(TimeUnit.SECONDS).createWithDefault(1)
Expand Down
10 changes: 8 additions & 2 deletions core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,8 @@ class StageInfo(
val parentIds: Seq[Int],
val details: String,
val taskMetrics: TaskMetrics = null,
private[spark] val taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty) {
private[spark] val taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty,
private[spark] val shuffleDepId: Option[Int] = None) {
/** When this stage was submitted from the DAGScheduler to a TaskScheduler. */
var submissionTime: Option[Long] = None
/** Time when all tasks in the stage completed or when the stage was cancelled. */
Expand Down Expand Up @@ -90,6 +91,10 @@ private[spark] object StageInfo {
): StageInfo = {
val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd)
val rddInfos = Seq(RDDInfo.fromRdd(stage.rdd)) ++ ancestorRddInfos
val shuffleDepId = stage match {
case sms: ShuffleMapStage => Option(sms.shuffleDep).map(_.shuffleId)
case _ => None
}
new StageInfo(
stage.id,
attemptId,
Expand All @@ -99,6 +104,7 @@ private[spark] object StageInfo {
stage.parents.map(_.id),
stage.details,
taskMetrics,
taskLocalityPreferences)
taskLocalityPreferences,
shuffleDepId)
}
}
Loading