" +
"");
reselectCheckboxesBasedOnTaskTableState();
diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala
index 3d369802f3023..7ba3f6bba8ec1 100644
--- a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala
+++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala
@@ -216,6 +216,8 @@ class BarrierTaskContext private[spark] (
resources().asJava
}
+ override def resourceProfileId(): Int = taskContext.resourceProfileId()
+
override private[spark] def killTaskIfInterrupted(): Unit = taskContext.killTaskIfInterrupted()
override private[spark] def getKillReason(): Option[String] = taskContext.getKillReason()
diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
index cb965cb180207..9db52cdee80b6 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
@@ -37,24 +37,29 @@ private[spark] trait ExecutorAllocationClient {
/**
* Update the cluster manager on our scheduling needs. Three bits of information are included
* to help it make decisions.
- * @param numExecutors The total number of executors we'd like to have. The cluster manager
- * shouldn't kill any running executor to reach this number, but,
- * if all existing executors were to die, this is the number of executors
- * we'd want to be allocated.
- * @param localityAwareTasks The number of tasks in all active stages that have a locality
- * preferences. This includes running, pending, and completed tasks.
- * @param hostToLocalTaskCount A map of hosts to the number of tasks from all active stages
- * that would like to like to run on that host.
- * This includes running, pending, and completed tasks.
+ *
+ * @param numLocalityAwareTasksPerResourceProfileId The number of tasks in all active stages that
+ * have a locality preferences per
+ * ResourceProfile id. This includes running,
+ * pending, and completed tasks.
+ * @param hostToLocalTaskCount A map of ResourceProfile id to a map of hosts to the number of
+ * tasks from all active stages that would like to like to run on
+ * that host. This includes running, pending, and completed tasks.
+ * @param resourceProfileIdToNumExecutors The total number of executors we'd like to have per
+ * ResourceProfile id. The cluster manager shouldn't kill any
+ * running executor to reach this number, but, if all
+ * existing executors were to die, this is the number
+ * of executors we'd want to be allocated.
* @return whether the request is acknowledged by the cluster manager.
*/
private[spark] def requestTotalExecutors(
- numExecutors: Int,
- localityAwareTasks: Int,
- hostToLocalTaskCount: Map[String, Int]): Boolean
+ numLocalityAwareTasksPerResourceProfileId: Map[Int, Int],
+ hostToLocalTaskCount: Map[Int, Map[String, Int]],
+ resourceProfileIdToNumExecutors: Map[Int, Int]): Boolean
/**
- * Request an additional number of executors from the cluster manager.
+ * Request an additional number of executors from the cluster manager for the default
+ * ResourceProfile.
* @return whether the request is acknowledged by the cluster manager.
*/
def requestExecutors(numAdditionalExecutors: Int): Boolean
diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
index bff854a003572..3bad00c131661 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
@@ -29,6 +29,8 @@ import org.apache.spark.internal.{config, Logging}
import org.apache.spark.internal.config._
import org.apache.spark.internal.config.Tests.TEST_SCHEDULE_INTERVAL
import org.apache.spark.metrics.source.Source
+import org.apache.spark.resource.ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID
+import org.apache.spark.resource.ResourceProfileManager
import org.apache.spark.scheduler._
import org.apache.spark.scheduler.dynalloc.ExecutorMonitor
import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils}
@@ -57,8 +59,9 @@ import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils}
* quickly over time in case the maximum number of executors is very high. Otherwise, it will take
* a long time to ramp up under heavy workloads.
*
- * The remove policy is simpler: If an executor has been idle for K seconds, meaning it has not
- * been scheduled to run any tasks, then it is removed. Note that an executor caching any data
+ * The remove policy is simpler: If an executor has been idle for K seconds and the number of
+ * executors is more then what is needed, meaning there are not enough tasks that could use
+ * the executor, then it is removed. Note that an executor caching any data
* blocks will be removed if it has been idle for more than L seconds.
*
* There is no retry logic in either case because we make the assumption that the cluster manager
@@ -95,7 +98,8 @@ private[spark] class ExecutorAllocationManager(
listenerBus: LiveListenerBus,
conf: SparkConf,
cleaner: Option[ContextCleaner] = None,
- clock: Clock = new SystemClock())
+ clock: Clock = new SystemClock(),
+ resourceProfileManager: ResourceProfileManager)
extends Logging {
allocationManager =>
@@ -117,23 +121,21 @@ private[spark] class ExecutorAllocationManager(
// During testing, the methods to actually kill and add executors are mocked out
private val testing = conf.get(DYN_ALLOCATION_TESTING)
- // TODO: The default value of 1 for spark.executor.cores works right now because dynamic
- // allocation is only supported for YARN and the default number of cores per executor in YARN is
- // 1, but it might need to be attained differently for different cluster managers
- private val tasksPerExecutorForFullParallelism =
- conf.get(EXECUTOR_CORES) / conf.get(CPUS_PER_TASK)
-
private val executorAllocationRatio =
conf.get(DYN_ALLOCATION_EXECUTOR_ALLOCATION_RATIO)
+ private val defaultProfile = resourceProfileManager.defaultResourceProfile
+
validateSettings()
- // Number of executors to add in the next round
- private var numExecutorsToAdd = 1
+ // Number of executors to add for each ResourceProfile in the next round
+ private val numExecutorsToAddPerResourceProfileId = new mutable.HashMap[Int, Int]
+ numExecutorsToAddPerResourceProfileId(defaultProfile.id) = 1
// The desired number of executors at this moment in time. If all our executors were to die, this
// is the number of executors we would immediately want from the cluster manager.
- private var numExecutorsTarget = initialNumExecutors
+ private val numExecutorsTargetPerResourceProfileId = new mutable.HashMap[Int, Int]
+ numExecutorsTargetPerResourceProfileId(defaultProfile.id) = initialNumExecutors
// A timestamp of when an addition should be triggered, or NOT_SET if it is not set
// This is set when pending tasks are added but not scheduled yet
@@ -165,11 +167,12 @@ private[spark] class ExecutorAllocationManager(
// (2) an executor idle timeout has elapsed.
@volatile private var initializing: Boolean = true
- // Number of locality aware tasks, used for executor placement.
- private var localityAwareTasks = 0
+ // Number of locality aware tasks for each ResourceProfile, used for executor placement.
+ private var numLocalityAwareTasksPerResourceProfileId = new mutable.HashMap[Int, Int]
+ numLocalityAwareTasksPerResourceProfileId(defaultProfile.id) = 0
- // Host to possible task running on it, used for executor placement.
- private var hostToLocalTaskCount: Map[String, Int] = Map.empty
+ // ResourceProfile id to Host to possible task running on it, used for executor placement.
+ private var rpIdToHostToLocalTaskCount: Map[Int, Map[String, Int]] = Map.empty
/**
* Verify that the settings specified through the config are valid.
@@ -233,7 +236,8 @@ private[spark] class ExecutorAllocationManager(
}
executor.scheduleWithFixedDelay(scheduleTask, 0, intervalMillis, TimeUnit.MILLISECONDS)
- client.requestTotalExecutors(numExecutorsTarget, localityAwareTasks, hostToLocalTaskCount)
+ client.requestTotalExecutors(numLocalityAwareTasksPerResourceProfileId.toMap,
+ rpIdToHostToLocalTaskCount, numExecutorsTargetPerResourceProfileId.toMap)
}
/**
@@ -253,7 +257,11 @@ private[spark] class ExecutorAllocationManager(
*/
def reset(): Unit = synchronized {
addTime = 0L
- numExecutorsTarget = initialNumExecutors
+ numExecutorsTargetPerResourceProfileId.keys.foreach { rpId =>
+ // Note this means every profile will be allowed to have initial number
+ // we may want to make this configurable per Profile in the future
+ numExecutorsTargetPerResourceProfileId(rpId) = initialNumExecutors
+ }
executorMonitor.reset()
}
@@ -261,15 +269,18 @@ private[spark] class ExecutorAllocationManager(
* The maximum number of executors we would need under the current load to satisfy all running
* and pending tasks, rounded up.
*/
- private def maxNumExecutorsNeeded(): Int = {
- val numRunningOrPendingTasks = listener.totalPendingTasks + listener.totalRunningTasks
- math.ceil(numRunningOrPendingTasks * executorAllocationRatio /
- tasksPerExecutorForFullParallelism)
- .toInt
+ private def maxNumExecutorsNeededPerResourceProfile(rpId: Int): Int = {
+ val numRunningOrPendingTasks = listener.totalPendingTasksPerResourceProfile(rpId) +
+ listener.totalRunningTasksPerResourceProfile(rpId)
+ val tasksPerExecutor =
+ resourceProfileManager.resourceProfileFromId(rpId).maxTasksPerExecutor(conf)
+ logDebug(s"max needed executor rpId: $rpId numpending: $numRunningOrPendingTasks," +
+ s" tasksperexecutor: $tasksPerExecutor")
+ math.ceil(numRunningOrPendingTasks * executorAllocationRatio / tasksPerExecutor).toInt
}
- private def totalRunningTasks(): Int = synchronized {
- listener.totalRunningTasks
+ private def totalRunningTasksPerResourceProfile(id: Int): Int = synchronized {
+ listener.totalRunningTasksPerResourceProfile(id)
}
/**
@@ -307,113 +318,186 @@ private[spark] class ExecutorAllocationManager(
* @return the delta in the target number of executors.
*/
private def updateAndSyncNumExecutorsTarget(now: Long): Int = synchronized {
- val maxNeeded = maxNumExecutorsNeeded
if (initializing) {
// Do not change our target while we are still initializing,
// Otherwise the first job may have to ramp up unnecessarily
0
- } else if (maxNeeded < numExecutorsTarget) {
- // The target number exceeds the number we actually need, so stop adding new
- // executors and inform the cluster manager to cancel the extra pending requests
- val oldNumExecutorsTarget = numExecutorsTarget
- numExecutorsTarget = math.max(maxNeeded, minNumExecutors)
- numExecutorsToAdd = 1
-
- // If the new target has not changed, avoid sending a message to the cluster manager
- if (numExecutorsTarget < oldNumExecutorsTarget) {
- // We lower the target number of executors but don't actively kill any yet. Killing is
- // controlled separately by an idle timeout. It's still helpful to reduce the target number
- // in case an executor just happens to get lost (eg., bad hardware, or the cluster manager
- // preempts it) -- in that case, there is no point in trying to immediately get a new
- // executor, since we wouldn't even use it yet.
- client.requestTotalExecutors(numExecutorsTarget, localityAwareTasks, hostToLocalTaskCount)
- logDebug(s"Lowering target number of executors to $numExecutorsTarget (previously " +
- s"$oldNumExecutorsTarget) because not all requested executors are actually needed")
+ } else {
+ val updatesNeeded = new mutable.HashMap[Int, ExecutorAllocationManager.TargetNumUpdates]
+
+ // Update targets for all ResourceProfiles then do a single request to the cluster manager
+ numExecutorsTargetPerResourceProfileId.foreach { case (rProfId, targetExecs) =>
+ val maxNeeded = maxNumExecutorsNeededPerResourceProfile(rProfId)
+ if (maxNeeded < targetExecs) {
+ // The target number exceeds the number we actually need, so stop adding new
+ // executors and inform the cluster manager to cancel the extra pending requests
+
+ // We lower the target number of executors but don't actively kill any yet. Killing is
+ // controlled separately by an idle timeout. It's still helpful to reduce
+ // the target number in case an executor just happens to get lost (eg., bad hardware,
+ // or the cluster manager preempts it) -- in that case, there is no point in trying
+ // to immediately get a new executor, since we wouldn't even use it yet.
+ decrementExecutorsFromTarget(maxNeeded, rProfId, updatesNeeded)
+ } else if (addTime != NOT_SET && now >= addTime) {
+ addExecutorsToTarget(maxNeeded, rProfId, updatesNeeded)
+ }
+ }
+ doUpdateRequest(updatesNeeded.toMap, now)
+ }
+ }
+
+ private def addExecutorsToTarget(
+ maxNeeded: Int,
+ rpId: Int,
+ updatesNeeded: mutable.HashMap[Int, ExecutorAllocationManager.TargetNumUpdates]
+ ): Int = {
+ updateTargetExecs(addExecutors, maxNeeded, rpId, updatesNeeded)
+ }
+
+ private def decrementExecutorsFromTarget(
+ maxNeeded: Int,
+ rpId: Int,
+ updatesNeeded: mutable.HashMap[Int, ExecutorAllocationManager.TargetNumUpdates]
+ ): Int = {
+ updateTargetExecs(decrementExecutors, maxNeeded, rpId, updatesNeeded)
+ }
+
+ private def updateTargetExecs(
+ updateTargetFn: (Int, Int) => Int,
+ maxNeeded: Int,
+ rpId: Int,
+ updatesNeeded: mutable.HashMap[Int, ExecutorAllocationManager.TargetNumUpdates]
+ ): Int = {
+ val oldNumExecutorsTarget = numExecutorsTargetPerResourceProfileId(rpId)
+ // update the target number (add or remove)
+ val delta = updateTargetFn(maxNeeded, rpId)
+ if (delta != 0) {
+ updatesNeeded(rpId) = ExecutorAllocationManager.TargetNumUpdates(delta, oldNumExecutorsTarget)
+ }
+ delta
+ }
+
+ private def doUpdateRequest(
+ updates: Map[Int, ExecutorAllocationManager.TargetNumUpdates],
+ now: Long): Int = {
+ // Only call cluster manager if target has changed.
+ if (updates.size > 0) {
+ val requestAcknowledged = try {
+ logInfo("requesting updates: " + updates)
+ testing ||
+ client.requestTotalExecutors(numLocalityAwareTasksPerResourceProfileId.toMap,
+ rpIdToHostToLocalTaskCount, numExecutorsTargetPerResourceProfileId.toMap)
+ } catch {
+ case NonFatal(e) =>
+ // Use INFO level so the error it doesn't show up by default in shells.
+ // Errors here are more commonly caused by YARN AM restarts, which is a recoverable
+ // issue, and generate a lot of noisy output.
+ logInfo("Error reaching cluster manager.", e)
+ false
+ }
+ if (requestAcknowledged) {
+ // have to go through all resource profiles that changed
+ var totalDelta = 0
+ updates.foreach { case (rpId, targetNum) =>
+ val delta = targetNum.delta
+ totalDelta += delta
+ if (delta > 0) {
+ val executorsString = "executor" + { if (delta > 1) "s" else "" }
+ logInfo(s"Requesting $delta new $executorsString because tasks are backlogged " +
+ s"(new desired total will be ${numExecutorsTargetPerResourceProfileId(rpId)} " +
+ s"for resource profile id: ${rpId})")
+ numExecutorsToAddPerResourceProfileId(rpId) =
+ if (delta == numExecutorsToAddPerResourceProfileId(rpId)) {
+ numExecutorsToAddPerResourceProfileId(rpId) * 2
+ } else {
+ 1
+ }
+ logDebug(s"Starting timer to add more executors (to " +
+ s"expire in $sustainedSchedulerBacklogTimeoutS seconds)")
+ addTime = now + TimeUnit.SECONDS.toNanos(sustainedSchedulerBacklogTimeoutS)
+ } else {
+ logDebug(s"Lowering target number of executors to" +
+ s" ${numExecutorsTargetPerResourceProfileId(rpId)} (previously " +
+ s"$targetNum.oldNumExecutorsTarget for resource profile id: ${rpId}) " +
+ "because not all requested executors " +
+ "are actually needed")
+ }
+ }
+ totalDelta
+ } else {
+ // request was for all profiles so we have to go through all to reset to old num
+ updates.foreach { case (rpId, targetNum) =>
+ logWarning(
+ s"Unable to reach the cluster manager to request more executors!")
+ numExecutorsTargetPerResourceProfileId(rpId) = targetNum.oldNumExecutorsTarget
+ }
+ 0
}
- numExecutorsTarget - oldNumExecutorsTarget
- } else if (addTime != NOT_SET && now >= addTime) {
- val delta = addExecutors(maxNeeded)
- logDebug(s"Starting timer to add more executors (to " +
- s"expire in $sustainedSchedulerBacklogTimeoutS seconds)")
- addTime = now + TimeUnit.SECONDS.toNanos(sustainedSchedulerBacklogTimeoutS)
- delta
} else {
+ logDebug("No change in number of executors")
0
}
}
+ private def decrementExecutors(maxNeeded: Int, rpId: Int): Int = {
+ val oldNumExecutorsTarget = numExecutorsTargetPerResourceProfileId(rpId)
+ numExecutorsTargetPerResourceProfileId(rpId) = math.max(maxNeeded, minNumExecutors)
+ numExecutorsToAddPerResourceProfileId(rpId) = 1
+ numExecutorsTargetPerResourceProfileId(rpId) - oldNumExecutorsTarget
+ }
+
/**
- * Request a number of executors from the cluster manager.
+ * Update the target number of executors and figure out how many to add.
* If the cap on the number of executors is reached, give up and reset the
* number of executors to add next round instead of continuing to double it.
*
* @param maxNumExecutorsNeeded the maximum number of executors all currently running or pending
* tasks could fill
+ * @param rp the ResourceProfile of the executors
* @return the number of additional executors actually requested.
*/
- private def addExecutors(maxNumExecutorsNeeded: Int): Int = {
+ private def addExecutors(maxNumExecutorsNeeded: Int, rpId: Int): Int = {
+ val oldNumExecutorsTarget = numExecutorsTargetPerResourceProfileId(rpId)
// Do not request more executors if it would put our target over the upper bound
- if (numExecutorsTarget >= maxNumExecutors) {
+ // this is doing a max check per ResourceProfile
+ if (oldNumExecutorsTarget >= maxNumExecutors) {
logDebug(s"Not adding executors because our current target total " +
- s"is already $numExecutorsTarget (limit $maxNumExecutors)")
- numExecutorsToAdd = 1
+ s"is already ${oldNumExecutorsTarget} (limit $maxNumExecutors)")
+ numExecutorsToAddPerResourceProfileId(rpId) = 1
return 0
}
-
- val oldNumExecutorsTarget = numExecutorsTarget
// There's no point in wasting time ramping up to the number of executors we already have, so
// make sure our target is at least as much as our current allocation:
- numExecutorsTarget = math.max(numExecutorsTarget, executorMonitor.executorCount)
+ numExecutorsTargetPerResourceProfileId(rpId) =
+ math.max(numExecutorsTargetPerResourceProfileId(rpId),
+ executorMonitor.executorCountWithResourceProfile(rpId))
+
// Boost our target with the number to add for this round:
- numExecutorsTarget += numExecutorsToAdd
+ numExecutorsTargetPerResourceProfileId(rpId) +=
+ numExecutorsToAddPerResourceProfileId.getOrElseUpdate(rpId, 1)
+
// Ensure that our target doesn't exceed what we need at the present moment:
- numExecutorsTarget = math.min(numExecutorsTarget, maxNumExecutorsNeeded)
+ numExecutorsTargetPerResourceProfileId(rpId) =
+ math.min(numExecutorsTargetPerResourceProfileId(rpId), maxNumExecutorsNeeded)
+
// Ensure that our target fits within configured bounds:
- numExecutorsTarget = math.max(math.min(numExecutorsTarget, maxNumExecutors), minNumExecutors)
+ numExecutorsTargetPerResourceProfileId(rpId) = math.max(
+ math.min(numExecutorsTargetPerResourceProfileId(rpId), maxNumExecutors), minNumExecutors)
- val delta = numExecutorsTarget - oldNumExecutorsTarget
+ val delta = numExecutorsTargetPerResourceProfileId(rpId) - oldNumExecutorsTarget
+ // logWarning("add executors delta is: " + delta + " old is: " + oldNumExecutorsTarget)
// If our target has not changed, do not send a message
// to the cluster manager and reset our exponential growth
if (delta == 0) {
- // Check if there is any speculative jobs pending
- if (listener.pendingTasks == 0 && listener.pendingSpeculativeTasks > 0) {
- numExecutorsTarget =
- math.max(math.min(maxNumExecutorsNeeded + 1, maxNumExecutors), minNumExecutors)
- } else {
- numExecutorsToAdd = 1
- return 0
- }
+ numExecutorsToAddPerResourceProfileId(rpId) = 1
}
+ delta
+ }
- val addRequestAcknowledged = try {
- testing ||
- client.requestTotalExecutors(numExecutorsTarget, localityAwareTasks, hostToLocalTaskCount)
- } catch {
- case NonFatal(e) =>
- // Use INFO level so the error it doesn't show up by default in shells. Errors here are more
- // commonly caused by YARN AM restarts, which is a recoverable issue, and generate a lot of
- // noisy output.
- logInfo("Error reaching cluster manager.", e)
- false
- }
- if (addRequestAcknowledged) {
- val executorsString = "executor" + { if (delta > 1) "s" else "" }
- logInfo(s"Requesting $delta new $executorsString because tasks are backlogged" +
- s" (new desired total will be $numExecutorsTarget)")
- numExecutorsToAdd = if (delta == numExecutorsToAdd) {
- numExecutorsToAdd * 2
- } else {
- 1
- }
- delta
- } else {
- logWarning(
- s"Unable to reach the cluster manager to request $numExecutorsTarget total executors!")
- numExecutorsTarget = oldNumExecutorsTarget
- 0
- }
+ private def getResourceProfileIdOfExecutor(executorId: String): Int = {
+ executorMonitor.getResourceProfileId(executorId)
}
/**
@@ -424,19 +508,30 @@ private[spark] class ExecutorAllocationManager(
val executorIdsToBeRemoved = new ArrayBuffer[String]
logDebug(s"Request to remove executorIds: ${executors.mkString(", ")}")
- val numExistingExecutors = executorMonitor.executorCount - executorMonitor.pendingRemovalCount
+ val numExecutorsTotalPerRpId = mutable.Map[Int, Int]()
- var newExecutorTotal = numExistingExecutors
executors.foreach { executorIdToBeRemoved =>
- if (newExecutorTotal - 1 < minNumExecutors) {
- logDebug(s"Not removing idle executor $executorIdToBeRemoved because there are only " +
- s"$newExecutorTotal executor(s) left (minimum number of executor limit $minNumExecutors)")
- } else if (newExecutorTotal - 1 < numExecutorsTarget) {
- logDebug(s"Not removing idle executor $executorIdToBeRemoved because there are only " +
- s"$newExecutorTotal executor(s) left (number of executor target $numExecutorsTarget)")
+ val rpId = getResourceProfileIdOfExecutor(executorIdToBeRemoved)
+ if (rpId == UNKNOWN_RESOURCE_PROFILE_ID) {
+ logWarning(s"Not removing executor $executorIdsToBeRemoved because couldn't find " +
+ "ResourceProfile for it!")
} else {
- executorIdsToBeRemoved += executorIdToBeRemoved
- newExecutorTotal -= 1
+ // get the running total as we remove or initialize it to the count - pendingRemoval
+ val newExecutorTotal = numExecutorsTotalPerRpId.getOrElseUpdate(rpId,
+ (executorMonitor.executorCountWithResourceProfile(rpId) -
+ executorMonitor.pendingRemovalCountPerResourceProfileId(rpId)))
+ if (newExecutorTotal - 1 < minNumExecutors) {
+ logDebug(s"Not removing idle executor $executorIdToBeRemoved because there " +
+ s"are only $newExecutorTotal executor(s) left (minimum number of executor limit " +
+ s"$minNumExecutors)")
+ } else if (newExecutorTotal - 1 < numExecutorsTargetPerResourceProfileId(rpId)) {
+ logDebug(s"Not removing idle executor $executorIdToBeRemoved because there " +
+ s"are only $newExecutorTotal executor(s) left (number of executor " +
+ s"target ${numExecutorsTargetPerResourceProfileId(rpId)})")
+ } else {
+ executorIdsToBeRemoved += executorIdToBeRemoved
+ numExecutorsTotalPerRpId(rpId) -= 1
+ }
}
}
@@ -456,14 +551,13 @@ private[spark] class ExecutorAllocationManager(
// [SPARK-21834] killExecutors api reduces the target number of executors.
// So we need to update the target with desired value.
- client.requestTotalExecutors(numExecutorsTarget, localityAwareTasks, hostToLocalTaskCount)
+ client.requestTotalExecutors(numLocalityAwareTasksPerResourceProfileId.toMap,
+ rpIdToHostToLocalTaskCount, numExecutorsTargetPerResourceProfileId.toMap)
+
// reset the newExecutorTotal to the existing number of executors
- newExecutorTotal = numExistingExecutors
if (testing || executorsRemoved.nonEmpty) {
- newExecutorTotal -= executorsRemoved.size
executorMonitor.executorsKilled(executorsRemoved)
- logInfo(s"Executors ${executorsRemoved.mkString(",")} removed due to idle timeout." +
- s"(new desired total will be $newExecutorTotal)")
+ logInfo(s"Executors ${executorsRemoved.mkString(",")} removed due to idle timeout.")
executorsRemoved
} else {
logWarning(s"Unable to reach the cluster manager to kill executor/s " +
@@ -492,7 +586,7 @@ private[spark] class ExecutorAllocationManager(
private def onSchedulerQueueEmpty(): Unit = synchronized {
logDebug("Clearing timer to add executors because there are no more pending tasks")
addTime = NOT_SET
- numExecutorsToAdd = 1
+ numExecutorsToAddPerResourceProfileId.keys.foreach(numExecutorsToAddPerResourceProfileId(_) = 1)
}
private case class StageAttempt(stageId: Int, stageAttemptId: Int) {
@@ -518,12 +612,16 @@ private[spark] class ExecutorAllocationManager(
private val stageAttemptToSpeculativeTaskIndices =
new mutable.HashMap[StageAttempt, mutable.HashSet[Int]]
+ private val resourceProfileIdToStageAttempt =
+ new mutable.HashMap[Int, mutable.Set[StageAttempt]]
+
// stageAttempt to tuple (the number of task with locality preferences, a map where each pair
- // is a node and the number of tasks that would like to be scheduled on that node) map,
+ // is a node and the number of tasks that would like to be scheduled on that node, and
+ // the resource profile id) map,
// maintain the executor placement hints for each stageAttempt used by resource framework
// to better place the executors.
private val stageAttemptToExecutorPlacementHints =
- new mutable.HashMap[StageAttempt, (Int, Map[String, Int])]
+ new mutable.HashMap[StageAttempt, (Int, Map[String, Int], Int)]
override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = {
initializing = false
@@ -534,6 +632,13 @@ private[spark] class ExecutorAllocationManager(
allocationManager.synchronized {
stageAttemptToNumTasks(stageAttempt) = numTasks
allocationManager.onSchedulerBacklogged()
+ // need to keep stage task requirements to ask for the right containers
+ val profId = stageSubmitted.stageInfo.resourceProfileId
+ logDebug(s"Stage resource profile id is: $profId with numTasks: $numTasks")
+ resourceProfileIdToStageAttempt.getOrElseUpdate(
+ profId, new mutable.HashSet[StageAttempt]) += stageAttempt
+ numExecutorsToAddPerResourceProfileId.getOrElseUpdate(profId, 1)
+ numExecutorsTargetPerResourceProfileId.getOrElseUpdate(profId, initialNumExecutors)
// Compute the number of tasks requested by the stage on each host
var numTasksPending = 0
@@ -547,8 +652,10 @@ private[spark] class ExecutorAllocationManager(
}
}
}
+
stageAttemptToExecutorPlacementHints.put(stageAttempt,
- (numTasksPending, hostToLocalTaskCountPerStage.toMap))
+ (numTasksPending, hostToLocalTaskCountPerStage.toMap,
+ stageSubmitted.stageInfo.resourceProfileId))
// Update the executor placement hints
updateExecutorPlacementHints()
@@ -560,9 +667,10 @@ private[spark] class ExecutorAllocationManager(
val stageAttemptId = stageCompleted.stageInfo.attemptNumber()
val stageAttempt = StageAttempt(stageId, stageAttemptId)
allocationManager.synchronized {
- // do NOT remove stageAttempt from stageAttemptToNumRunningTasks,
+ // do NOT remove stageAttempt from stageAttemptToNumRunningTask
// because the attempt may still have running tasks,
// even after another attempt for the stage is submitted.
+ val numTasks = stageAttemptToNumTasks(stageAttempt)
stageAttemptToNumTasks -= stageAttempt
stageAttemptToNumSpeculativeTasks -= stageAttempt
stageAttemptToTaskIndices -= stageAttempt
@@ -596,7 +704,7 @@ private[spark] class ExecutorAllocationManager(
stageAttemptToTaskIndices.getOrElseUpdate(stageAttempt,
new mutable.HashSet[Int]) += taskIndex
}
- if (totalPendingTasks() == 0) {
+ if (!hasPendingTasks) {
allocationManager.onSchedulerQueueEmpty()
}
}
@@ -612,13 +720,27 @@ private[spark] class ExecutorAllocationManager(
stageAttemptToNumRunningTask(stageAttempt) -= 1
if (stageAttemptToNumRunningTask(stageAttempt) == 0) {
stageAttemptToNumRunningTask -= stageAttempt
+ if (!stageAttemptToNumTasks.contains(stageAttempt)) {
+ val rpForStage = resourceProfileIdToStageAttempt.filter { case (k, v) =>
+ v.contains(stageAttempt)
+ }.keys
+ if (rpForStage.size == 1) {
+ // be careful about the removal from here due to late tasks, make sure stage is
+ // really complete and no tasks left
+ resourceProfileIdToStageAttempt(rpForStage.head) -= stageAttempt
+ } else {
+ logWarning(s"Should have exactly one resource profile for stage $stageAttempt," +
+ s" but have $rpForStage")
+ }
+ }
+
}
}
// If the task failed, we expect it to be resubmitted later. To ensure we have
// enough resources to run the resubmitted task, we need to mark the scheduler
// as backlogged again if it's not already marked as such (SPARK-8366)
if (taskEnd.reason != Success) {
- if (totalPendingTasks() == 0) {
+ if (!hasPendingTasks) {
allocationManager.onSchedulerBacklogged()
}
if (taskEnd.taskInfo.speculative) {
@@ -648,20 +770,50 @@ private[spark] class ExecutorAllocationManager(
*
* Note: This is not thread-safe without the caller owning the `allocationManager` lock.
*/
- def pendingTasks(): Int = {
- stageAttemptToNumTasks.map { case (stageAttempt, numTasks) =>
- numTasks - stageAttemptToTaskIndices.get(stageAttempt).map(_.size).getOrElse(0)
+ def pendingTasksPerResourceProfile(rpId: Int): Int = {
+ val attempts = resourceProfileIdToStageAttempt.getOrElse(rpId, Set.empty).toSeq
+ attempts.map { attempt =>
+ val numTotalTasks = stageAttemptToNumTasks.getOrElse(attempt, 0)
+ val numRunning = stageAttemptToTaskIndices.get(attempt).map(_.size).getOrElse(0)
+ numTotalTasks - numRunning
+ }.sum
+ }
+
+ def hasPendingRegularTasks: Boolean = {
+ val attempts = resourceProfileIdToStageAttempt.values.flatten
+ val pending = attempts.map { attempt =>
+ val numTotalTasks = stageAttemptToNumTasks.getOrElse(attempt, 0)
+ val numRunning = stageAttemptToTaskIndices.get(attempt).map(_.size).getOrElse(0)
+ numTotalTasks - numRunning
}.sum
+ (pending > 0)
}
- def pendingSpeculativeTasks(): Int = {
- stageAttemptToNumSpeculativeTasks.map { case (stageAttempt, numTasks) =>
- numTasks - stageAttemptToSpeculativeTaskIndices.get(stageAttempt).map(_.size).getOrElse(0)
+ def pendingSpeculativeTasksPerResourceProfile(rp: Int): Int = {
+ val attempts = resourceProfileIdToStageAttempt.getOrElse(rp, Set.empty).toSeq
+ attempts.map { attempt =>
+ val numTotalTasks = stageAttemptToNumSpeculativeTasks.getOrElse(attempt, 0)
+ val numRunning = stageAttemptToSpeculativeTaskIndices.get(attempt).map(_.size).getOrElse(0)
+ numTotalTasks - numRunning
}.sum
}
- def totalPendingTasks(): Int = {
- pendingTasks + pendingSpeculativeTasks
+ def hasPendingSpeculativeTasks: Boolean = {
+ val attempts = resourceProfileIdToStageAttempt.values.flatten
+ val pending = attempts.map { attempt =>
+ val numTotalTasks = stageAttemptToNumSpeculativeTasks.getOrElse(attempt, 0)
+ val numRunning = stageAttemptToSpeculativeTaskIndices.get(attempt).map(_.size).getOrElse(0)
+ numTotalTasks - numRunning
+ }.sum
+ (pending > 0)
+ }
+
+ def hasPendingTasks(): Boolean = {
+ hasPendingSpeculativeTasks || hasPendingRegularTasks
+ }
+
+ def totalPendingTasksPerResourceProfile(rp: Int): Int = {
+ pendingTasksPerResourceProfile(rp) + pendingSpeculativeTasksPerResourceProfile(rp)
}
/**
@@ -672,6 +824,14 @@ private[spark] class ExecutorAllocationManager(
stageAttemptToNumRunningTask.values.sum
}
+ def totalRunningTasksPerResourceProfile(rp: Int): Int = {
+ val attempts = resourceProfileIdToStageAttempt.getOrElse(rp, Set.empty).toSeq
+ // attempts is a Set, change to Seq so we keep all values
+ attempts.map { attempt =>
+ stageAttemptToNumRunningTask.getOrElseUpdate(attempt, 0)
+ }.sum
+ }
+
/**
* Update the Executor placement hints (the number of tasks with locality preferences,
* a map where each pair is a node and the number of tasks that would like to be scheduled
@@ -681,18 +841,28 @@ private[spark] class ExecutorAllocationManager(
* granularity within stages.
*/
def updateExecutorPlacementHints(): Unit = {
- var localityAwareTasks = 0
- val localityToCount = new mutable.HashMap[String, Int]()
- stageAttemptToExecutorPlacementHints.values.foreach { case (numTasksPending, localities) =>
- localityAwareTasks += numTasksPending
- localities.foreach { case (hostname, count) =>
- val updatedCount = localityToCount.getOrElse(hostname, 0) + count
- localityToCount(hostname) = updatedCount
- }
+ var localityAwareTasksPerResourceProfileId = new mutable.HashMap[Int, Int]
+
+ // ResourceProfile id => map[host, count]
+ val rplocalityToCount = new mutable.HashMap[Int, mutable.HashMap[String, Int]]()
+
+ stageAttemptToExecutorPlacementHints.values.foreach {
+ case (numTasksPending, localities, rpId) =>
+ val rpNumPending =
+ localityAwareTasksPerResourceProfileId.getOrElse(rpId, 0)
+ localityAwareTasksPerResourceProfileId(rpId) = rpNumPending + numTasksPending
+ localities.foreach { case (hostname, count) =>
+ val rpBasedHostToCount =
+ rplocalityToCount.getOrElseUpdate(rpId, new mutable.HashMap[String, Int])
+ val newUpdated = rpBasedHostToCount.getOrElse(hostname, 0) + count
+ rpBasedHostToCount(hostname) = newUpdated
+ }
}
- allocationManager.localityAwareTasks = localityAwareTasks
- allocationManager.hostToLocalTaskCount = localityToCount.toMap
+ allocationManager.numLocalityAwareTasksPerResourceProfileId =
+ localityAwareTasksPerResourceProfileId
+ allocationManager.rpIdToHostToLocalTaskCount =
+ rplocalityToCount.map { case (k, v) => (k, v.toMap)}.toMap
}
}
@@ -713,14 +883,22 @@ private[spark] class ExecutorAllocationManager(
})
}
- registerGauge("numberExecutorsToAdd", numExecutorsToAdd, 0)
+ // the metrics are going to return the numbers for the default ResourceProfile
+ registerGauge("numberExecutorsToAdd",
+ numExecutorsToAddPerResourceProfileId(defaultProfile.id), 0)
registerGauge("numberExecutorsPendingToRemove", executorMonitor.pendingRemovalCount, 0)
registerGauge("numberAllExecutors", executorMonitor.executorCount, 0)
- registerGauge("numberTargetExecutors", numExecutorsTarget, 0)
- registerGauge("numberMaxNeededExecutors", maxNumExecutorsNeeded(), 0)
+ registerGauge("numberTargetExecutors",
+ numExecutorsTargetPerResourceProfileId(defaultProfile.id), 0)
+ registerGauge("numberMaxNeededExecutors",
+ maxNumExecutorsNeededPerResourceProfile(defaultProfile.id), 0)
}
}
private object ExecutorAllocationManager {
val NOT_SET = Long.MaxValue
+
+ // helper case class for requesting executors, here to be visible for testing
+ private[spark] case class TargetNumUpdates(delta: Int, oldNumExecutorsTarget: Int)
+
}
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 3262631616279..0f0ce84dba774 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -25,6 +25,7 @@ import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicReferenc
import scala.collection.JavaConverters._
import scala.collection.Map
+import scala.collection.immutable
import scala.collection.mutable.HashMap
import scala.language.implicitConversions
import scala.reflect.{classTag, ClassTag}
@@ -53,7 +54,7 @@ import org.apache.spark.io.CompressionCodec
import org.apache.spark.metrics.source.JVMCPUSource
import org.apache.spark.partial.{ApproximateEvaluator, PartialResult}
import org.apache.spark.rdd._
-import org.apache.spark.resource.{ResourceID, ResourceInformation}
+import org.apache.spark.resource._
import org.apache.spark.resource.ResourceUtils._
import org.apache.spark.rpc.RpcEndpointRef
import org.apache.spark.scheduler._
@@ -219,9 +220,10 @@ class SparkContext(config: SparkConf) extends Logging {
private var _shutdownHookRef: AnyRef = _
private var _statusStore: AppStatusStore = _
private var _heartbeater: Heartbeater = _
- private var _resources: scala.collection.immutable.Map[String, ResourceInformation] = _
+ private var _resources: immutable.Map[String, ResourceInformation] = _
private var _shuffleDriverComponents: ShuffleDriverComponents = _
private var _plugins: Option[PluginContainer] = None
+ private var _resourceProfileManager: ResourceProfileManager = _
/* ------------------------------------------------------------------------------------- *
| Accessors and public fields. These provide access to the internal state of the |
@@ -343,6 +345,8 @@ class SparkContext(config: SparkConf) extends Logging {
private[spark] def executorAllocationManager: Option[ExecutorAllocationManager] =
_executorAllocationManager
+ private[spark] def resourceProfileManager: ResourceProfileManager = _resourceProfileManager
+
private[spark] def cleaner: Option[ContextCleaner] = _cleaner
private[spark] var checkpointDir: Option[String] = None
@@ -451,6 +455,7 @@ class SparkContext(config: SparkConf) extends Logging {
}
_listenerBus = new LiveListenerBus(_conf)
+ _resourceProfileManager = new ResourceProfileManager(_conf, listenerBus)
// Initialize the app status store and listener before SparkEnv is created so that it gets
// all events.
@@ -611,7 +616,7 @@ class SparkContext(config: SparkConf) extends Logging {
case b: ExecutorAllocationClient =>
Some(new ExecutorAllocationManager(
schedulerBackend.asInstanceOf[ExecutorAllocationClient], listenerBus, _conf,
- cleaner = cleaner))
+ cleaner = cleaner, resourceProfileManager = resourceProfileManager))
case _ =>
None
}
@@ -1612,17 +1617,21 @@ class SparkContext(config: SparkConf) extends Logging {
}
/**
- * Get the max number of tasks that can be concurrent launched currently.
+ * Get the max number of tasks that can be concurrent launched based on the ResourceProfile
+ * being used.
* Note that please don't cache the value returned by this method, because the number can change
* due to add/remove executors.
*
+ * @param rp ResourceProfile which to use to calculate max concurrent tasks.
* @return The max number of tasks that can be concurrent launched currently.
*/
- private[spark] def maxNumConcurrentTasks(): Int = schedulerBackend.maxNumConcurrentTasks()
+ private[spark] def maxNumConcurrentTasks(rp: ResourceProfile): Int = {
+ schedulerBackend.maxNumConcurrentTasks(rp)
+ }
/**
* Update the cluster manager on our scheduling needs. Three bits of information are included
- * to help it make decisions.
+ * to help it make decisions. This applies to the default ResourceProfile.
* @param numExecutors The total number of executors we'd like to have. The cluster manager
* shouldn't kill any running executor to reach this number, but,
* if all existing executors were to die, this is the number of executors
@@ -1638,11 +1647,17 @@ class SparkContext(config: SparkConf) extends Logging {
def requestTotalExecutors(
numExecutors: Int,
localityAwareTasks: Int,
- hostToLocalTaskCount: scala.collection.immutable.Map[String, Int]
+ hostToLocalTaskCount: immutable.Map[String, Int]
): Boolean = {
schedulerBackend match {
case b: ExecutorAllocationClient =>
- b.requestTotalExecutors(numExecutors, localityAwareTasks, hostToLocalTaskCount)
+ // this is being applied to the default resource profile, would need to add api to support
+ // others
+ val defaultProfId = resourceProfileManager.defaultResourceProfile.id
+ b.requestTotalExecutors(
+ immutable.Map(localityAwareTasks -> defaultProfId),
+ immutable.Map(defaultProfId -> hostToLocalTaskCount),
+ immutable.Map(defaultProfId-> numExecutors))
case _ =>
logWarning("Requesting executors is not supported by current scheduler.")
false
@@ -2036,6 +2051,7 @@ class SparkContext(config: SparkConf) extends Logging {
// Clear this `InheritableThreadLocal`, or it will still be inherited in child threads even this
// `SparkContext` is stopped.
localProperties.remove()
+ ResourceProfile.clearDefaultProfile()
// Unset YARN mode system env variable, to allow switching between cluster types.
SparkContext.clearActiveContext()
logInfo("Successfully stopped SparkContext")
@@ -2771,109 +2787,19 @@ object SparkContext extends Logging {
// When running locally, don't try to re-execute tasks on failure.
val MAX_LOCAL_TASK_FAILURES = 1
- // Ensure that executor's resources satisfies one or more tasks requirement.
- def checkResourcesPerTask(clusterMode: Boolean, executorCores: Option[Int]): Unit = {
+ // Ensure that default executor's resources satisfies one or more tasks requirement.
+ // This function is for cluster managers that don't set the executor cores config, for
+ // others its checked in ResourceProfile.
+ def checkResourcesPerTask(executorCores: Int): Unit = {
val taskCores = sc.conf.get(CPUS_PER_TASK)
- val execCores = if (clusterMode) {
- executorCores.getOrElse(sc.conf.get(EXECUTOR_CORES))
- } else {
- executorCores.get
- }
- // some cluster managers don't set the EXECUTOR_CORES config by default (standalone
- // and mesos coarse grained), so we can't rely on that config for those.
- val shouldCheckExecCores = executorCores.isDefined || sc.conf.contains(EXECUTOR_CORES) ||
- (master.equalsIgnoreCase("yarn") || master.startsWith("k8s"))
-
- // Number of cores per executor must meet at least one task requirement.
- if (shouldCheckExecCores && execCores < taskCores) {
- throw new SparkException(s"The number of cores per executor (=$execCores) has to be >= " +
- s"the task config: ${CPUS_PER_TASK.key} = $taskCores when run on $master.")
- }
-
- // Calculate the max slots each executor can provide based on resources available on each
- // executor and resources required by each task.
- val taskResourceRequirements = parseResourceRequirements(sc.conf, SPARK_TASK_PREFIX)
- val executorResourcesAndAmounts = parseAllResourceRequests(sc.conf, SPARK_EXECUTOR_PREFIX)
- .map(request => (request.id.resourceName, request.amount)).toMap
-
- var (numSlots, limitingResourceName) = if (shouldCheckExecCores) {
- (execCores / taskCores, "CPU")
- } else {
- (-1, "")
- }
-
- taskResourceRequirements.foreach { taskReq =>
- // Make sure the executor resources were specified through config.
- val execAmount = executorResourcesAndAmounts.getOrElse(taskReq.resourceName,
- throw new SparkException("The executor resource config: " +
- ResourceID(SPARK_EXECUTOR_PREFIX, taskReq.resourceName).amountConf +
- " needs to be specified since a task requirement config: " +
- ResourceID(SPARK_TASK_PREFIX, taskReq.resourceName).amountConf +
- " was specified")
- )
- // Make sure the executor resources are large enough to launch at least one task.
- if (execAmount < taskReq.amount) {
- throw new SparkException("The executor resource config: " +
- ResourceID(SPARK_EXECUTOR_PREFIX, taskReq.resourceName).amountConf +
- s" = $execAmount has to be >= the requested amount in task resource config: " +
- ResourceID(SPARK_TASK_PREFIX, taskReq.resourceName).amountConf +
- s" = ${taskReq.amount}")
- }
- // Compare and update the max slots each executor can provide.
- // If the configured amount per task was < 1.0, a task is subdividing
- // executor resources. If the amount per task was > 1.0, the task wants
- // multiple executor resources.
- val resourceNumSlots = Math.floor(execAmount * taskReq.numParts / taskReq.amount).toInt
- if (resourceNumSlots < numSlots) {
- if (shouldCheckExecCores) {
- throw new IllegalArgumentException("The number of slots on an executor has to be " +
- "limited by the number of cores, otherwise you waste resources and " +
- "dynamic allocation doesn't work properly. Your configuration has " +
- s"core/task cpu slots = ${numSlots} and " +
- s"${taskReq.resourceName} = ${resourceNumSlots}. " +
- "Please adjust your configuration so that all resources require same number " +
- "of executor slots.")
- }
- numSlots = resourceNumSlots
- limitingResourceName = taskReq.resourceName
- }
- }
- if(!shouldCheckExecCores && Utils.isDynamicAllocationEnabled(sc.conf)) {
- // if we can't rely on the executor cores config throw a warning for user
- logWarning("Please ensure that the number of slots available on your " +
- "executors is limited by the number of cores to task cpus and not another " +
- "custom resource. If cores is not the limiting resource then dynamic " +
- "allocation will not work properly!")
- }
- // warn if we would waste any resources due to another resource limiting the number of
- // slots on an executor
- taskResourceRequirements.foreach { taskReq =>
- val execAmount = executorResourcesAndAmounts(taskReq.resourceName)
- if ((numSlots * taskReq.amount / taskReq.numParts) < execAmount) {
- val taskReqStr = if (taskReq.numParts > 1) {
- s"${taskReq.amount}/${taskReq.numParts}"
- } else {
- s"${taskReq.amount}"
- }
- val resourceNumSlots = Math.floor(execAmount * taskReq.numParts / taskReq.amount).toInt
- val message = s"The configuration of resource: ${taskReq.resourceName} " +
- s"(exec = ${execAmount}, task = ${taskReqStr}, " +
- s"runnable tasks = ${resourceNumSlots}) will " +
- s"result in wasted resources due to resource ${limitingResourceName} limiting the " +
- s"number of runnable tasks per executor to: ${numSlots}. Please adjust " +
- s"your configuration."
- if (Utils.isTesting) {
- throw new SparkException(message)
- } else {
- logWarning(message)
- }
- }
- }
+ validateTaskCpusLargeEnough(sc.conf, executorCores, taskCores)
+ val defaultProf = sc.resourceProfileManager.defaultResourceProfile
+ ResourceUtils.warnOnWastedResources(defaultProf, sc.conf, Some(executorCores))
}
master match {
case "local" =>
- checkResourcesPerTask(clusterMode = false, Some(1))
+ checkResourcesPerTask(1)
val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true)
val backend = new LocalSchedulerBackend(sc.getConf, scheduler, 1)
scheduler.initialize(backend)
@@ -2886,7 +2812,7 @@ object SparkContext extends Logging {
if (threadCount <= 0) {
throw new SparkException(s"Asked to run locally with $threadCount threads")
}
- checkResourcesPerTask(clusterMode = false, Some(threadCount))
+ checkResourcesPerTask(threadCount)
val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true)
val backend = new LocalSchedulerBackend(sc.getConf, scheduler, threadCount)
scheduler.initialize(backend)
@@ -2897,14 +2823,13 @@ object SparkContext extends Logging {
// local[*, M] means the number of cores on the computer with M failures
// local[N, M] means exactly N threads with M failures
val threadCount = if (threads == "*") localCpuCount else threads.toInt
- checkResourcesPerTask(clusterMode = false, Some(threadCount))
+ checkResourcesPerTask(threadCount)
val scheduler = new TaskSchedulerImpl(sc, maxFailures.toInt, isLocal = true)
val backend = new LocalSchedulerBackend(sc.getConf, scheduler, threadCount)
scheduler.initialize(backend)
(backend, scheduler)
case SPARK_REGEX(sparkUrl) =>
- checkResourcesPerTask(clusterMode = true, None)
val scheduler = new TaskSchedulerImpl(sc)
val masterUrls = sparkUrl.split(",").map("spark://" + _)
val backend = new StandaloneSchedulerBackend(scheduler, sc, masterUrls)
@@ -2912,7 +2837,7 @@ object SparkContext extends Logging {
(backend, scheduler)
case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) =>
- checkResourcesPerTask(clusterMode = true, Some(coresPerSlave.toInt))
+ checkResourcesPerTask(coresPerSlave.toInt)
// Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang.
val memoryPerSlaveInt = memoryPerSlave.toInt
if (sc.executorMemory > memoryPerSlaveInt) {
@@ -2941,7 +2866,6 @@ object SparkContext extends Logging {
(backend, scheduler)
case masterUrl =>
- checkResourcesPerTask(clusterMode = true, None)
val cm = getClusterManager(masterUrl) match {
case Some(clusterMgr) => clusterMgr
case None => throw new SparkException("Could not parse Master URL: '" + master + "'")
diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala
index fd41facf95c76..b116690c62d31 100644
--- a/core/src/main/scala/org/apache/spark/TaskContext.scala
+++ b/core/src/main/scala/org/apache/spark/TaskContext.scala
@@ -193,6 +193,12 @@ abstract class TaskContext extends Serializable {
@Evolving
def resourcesJMap(): java.util.Map[String, ResourceInformation]
+ /**
+ * ResourceProfile Id used by the task.
+ */
+ @Evolving
+ def resourceProfileId(): Int
+
@DeveloperApi
def taskMetrics(): TaskMetrics
diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala
index 08a58a029528b..01ff58d89dbac 100644
--- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala
+++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala
@@ -29,6 +29,7 @@ import org.apache.spark.memory.TaskMemoryManager
import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.metrics.source.Source
import org.apache.spark.resource.ResourceInformation
+import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID
import org.apache.spark.shuffle.FetchFailedException
import org.apache.spark.util._
@@ -54,7 +55,8 @@ private[spark] class TaskContextImpl(
@transient private val metricsSystem: MetricsSystem,
// The default value is only used in tests.
override val taskMetrics: TaskMetrics = TaskMetrics.empty,
- override val resources: Map[String, ResourceInformation] = Map.empty)
+ override val resources: Map[String, ResourceInformation] = Map.empty,
+ override val resourceProfileId: Int = DEFAULT_RESOURCE_PROFILE_ID)
extends TaskContext
with Logging {
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
index 6f0182255e5fe..36307e9fc325d 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
@@ -27,6 +27,7 @@ import org.apache.spark._
import org.apache.spark.api.java.JavaSparkContext.fakeClassTag
import org.apache.spark.api.java.function.{Function => JFunction}
import org.apache.spark.rdd.RDD
+import org.apache.spark.resource.ResourceProfile
import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.Utils
@@ -49,6 +50,20 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T])
*/
def persist(newLevel: StorageLevel): JavaRDD[T] = wrapRDD(rdd.persist(newLevel))
+ /**
+ * Specify a ResourceProfile to use when calculating this RDD. This is only supported on
+ * certain cluster managers and currently requires dynamic allocation to be enabled.
+ * It will result in new executors with the resources specified being acquired to
+ * calculate the RDD.
+ */
+ def withResources(prof: ResourceProfile): JavaRDD[T] = wrapRDD(rdd.withResources(prof))
+
+ /**
+ * Get the ResourceProfile specified with this RDD or None if it wasn't specified.
+ * @return the user specified ResourceProfile or null if none was specified
+ */
+ def getResourceProfile(): ResourceProfile = rdd.getResourceProfile()
+
/**
* Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.
* This method blocks until all blocks are deleted.
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala
index 658e0d593a167..e2b985d5e2b72 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala
@@ -30,6 +30,7 @@ import org.apache.spark._
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config.{BUFFER_SIZE, EXECUTOR_CORES}
import org.apache.spark.internal.config.Python._
+import org.apache.spark.resource.ResourceProfile.{getPysparkMemoryFromInternalConfs, DEFAULT_RESOURCE_PROFILE_ID}
import org.apache.spark.security.SocketAuthHelper
import org.apache.spark.util._
@@ -80,9 +81,6 @@ private[spark] abstract class BasePythonRunner[IN, OUT](
private val conf = SparkEnv.get.conf
protected val bufferSize: Int = conf.get(BUFFER_SIZE)
private val reuseWorker = conf.get(PYTHON_WORKER_REUSE)
- // each python worker gets an equal part of the allocation. the worker pool will grow to the
- // number of concurrent tasks, which is determined by the number of cores in this executor.
- private val memoryMb = conf.get(PYSPARK_EXECUTOR_MEMORY).map(_ / conf.get(EXECUTOR_CORES))
// All the Python functions should have the same exec, version and envvars.
protected val envVars: java.util.Map[String, String] = funcs.head.funcs.head.envVars
@@ -101,6 +99,12 @@ private[spark] abstract class BasePythonRunner[IN, OUT](
// Authentication helper used when serving method calls via socket from Python side.
private lazy val authHelper = new SocketAuthHelper(conf)
+ // each python worker gets an equal part of the allocation. the worker pool will grow to the
+ // number of concurrent tasks, which is determined by the number of cores in this executor.
+ private def getWorkerMemoryMb(mem: Option[Long]): Option[Long] = {
+ mem.map(_ / conf.get(EXECUTOR_CORES))
+ }
+
def compute(
inputIterator: Iterator[IN],
partitionIndex: Int,
@@ -115,12 +119,27 @@ private[spark] abstract class BasePythonRunner[IN, OUT](
// see https://github.com/numpy/numpy/issues/10455
conf.getOption("spark.executor.cores").foreach(envVars.put("OMP_NUM_THREADS", _))
}
+ logInfo("omp num threads is: " + envVars.get("OMP_NUM_THREADS"))
envVars.put("SPARK_LOCAL_DIRS", localdir) // it's also used in monitor thread
if (reuseWorker) {
envVars.put("SPARK_REUSE_WORKER", "1")
}
- if (memoryMb.isDefined) {
- envVars.put("PYSPARK_EXECUTOR_MEMORY_MB", memoryMb.get.toString)
+ // Check to see if the pyspark memory conf set for the resource profile id being used.
+ // Not all cluster managers are supported so fall back to the application level config
+ // when its the default profile id.
+ val rpId = context.resourceProfileId()
+ val memoryMb = if (rpId == DEFAULT_RESOURCE_PROFILE_ID) {
+ logInfo("using default profile so default executor memory")
+ conf.get(PYSPARK_EXECUTOR_MEMORY)
+ } else {
+ val mem = getPysparkMemoryFromInternalConfs(conf, rpId)
+ logInfo(s"using prorfile $rpId memory $mem")
+ mem
+
+ }
+ val workerMemoryMb = getWorkerMemoryMb(memoryMb)
+ if (workerMemoryMb.isDefined) {
+ envVars.put("PYSPARK_EXECUTOR_MEMORY_MB", workerMemoryMb.get.toString)
}
envVars.put("SPARK_BUFFER_SIZE", bufferSize.toString)
val worker: Socket = env.createPythonWorker(pythonExec, envVars.asScala.toMap)
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
index df236ba8926c1..09665fff877d7 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
@@ -115,6 +115,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
*/
private def createThroughDaemon(): Socket = {
+ logInfo("createing through daemon")
def createSocket(): Socket = {
val socket = new Socket(daemonHost, daemonPort)
val pid = new DataInputStream(socket.getInputStream).readInt()
@@ -150,6 +151,8 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
*/
private def createSimpleWorker(): Socket = {
var serverSocket: ServerSocket = null
+ logInfo("createing through simple worker")
+
try {
serverSocket = new ServerSocket(0, 1, InetAddress.getByAddress(Array(127, 0, 0, 1)))
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
index 54c50006c33b7..1576c7c4defd2 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
@@ -45,6 +45,7 @@ import org.apache.spark.internal.config.History._
import org.apache.spark.internal.config.Status._
import org.apache.spark.internal.config.Tests.IS_TESTING
import org.apache.spark.internal.config.UI._
+import org.apache.spark.resource.ResourceProfileManager
import org.apache.spark.scheduler._
import org.apache.spark.scheduler.ReplayListenerBus._
import org.apache.spark.status._
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala
index 741050027fc6b..7973652b3e254 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala
@@ -72,7 +72,8 @@ private[spark] class HistoryAppStatusStore(
source.totalGCTime, source.totalInputBytes, source.totalShuffleRead,
source.totalShuffleWrite, source.isBlacklisted, source.maxMemory, source.addTime,
source.removeTime, source.removeReason, newExecutorLogs, source.memoryMetrics,
- source.blacklistedInStages, source.peakMemoryMetrics, source.attributes, source.resources)
+ source.blacklistedInStages, source.peakMemoryMetrics, source.attributes, source.resources,
+ source.resourceProfileId)
}
}
diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
index f56e7c6d78c48..887179329e6ca 100644
--- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -70,11 +70,12 @@ private[spark] class CoarseGrainedExecutorBackend(
private[this] val ser: SerializerInstance = env.closureSerializer.newInstance()
/**
- * Map each taskId to the information about the resource allocated to it, Please refer to
- * [[ResourceInformation]] for specifics.
+ * Map each taskId to a tuple of cpus and the information about the resource allocated to it.
+ * Please refer to [[ResourceInformation]] for specifics.
* Exposed for testing only.
*/
- private[executor] val taskResources = new mutable.HashMap[Long, Map[String, ResourceInformation]]
+ private[executor] val taskResources =
+ new mutable.HashMap[Long, (Int, Map[String, ResourceInformation])]
override def onStart(): Unit = {
logInfo("Connecting to driver: " + driverUrl)
@@ -132,7 +133,7 @@ private[spark] class CoarseGrainedExecutorBackend(
} else {
val taskDesc = TaskDescription.decode(data.value)
logInfo("Got assigned task " + taskDesc.taskId)
- taskResources(taskDesc.taskId) = taskDesc.resources
+ taskResources(taskDesc.taskId) = (taskDesc.cpus, taskDesc.resources)
executor.launchTask(this, taskDesc)
}
@@ -179,8 +180,9 @@ private[spark] class CoarseGrainedExecutorBackend(
}
override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer): Unit = {
- val resources = taskResources.getOrElse(taskId, Map.empty[String, ResourceInformation])
- val msg = StatusUpdate(executorId, taskId, state, data, resources)
+ val (taskCpus, resources) =
+ taskResources.getOrElse(taskId, (1, Map.empty[String, ResourceInformation]))
+ val msg = StatusUpdate(executorId, taskId, state, data, taskCpus, resources)
if (TaskState.isFinished(state)) {
taskResources.remove(taskId)
}
@@ -278,8 +280,13 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
}
}
+
val cfg = driver.askSync[SparkAppConfig](RetrieveSparkAppConfig(arguments.resourceProfileId))
- val props = cfg.sparkProperties ++ Seq[(String, String)](("spark.app.id", arguments.appId))
+ // we have to add the pyspark memory conf into SparkConfs so the PythonRunner can
+ // pick it up properly
+ val pysparkMemoryConf = cfg.resourceProfile.getInternalPysparkMemoryConfs
+ val props = cfg.sparkProperties ++
+ Seq[(String, String)](("spark.app.id", arguments.appId)) ++ pysparkMemoryConf
fetcher.shutdown()
// Create SparkEnv using properties we fetched from the driver.
diff --git a/core/src/main/scala/org/apache/spark/internal/config/Tests.scala b/core/src/main/scala/org/apache/spark/internal/config/Tests.scala
index 21660ab3a9512..e99845764ae54 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/Tests.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/Tests.scala
@@ -53,4 +53,17 @@ private[spark] object Tests {
val TEST_N_CORES_EXECUTOR = ConfigBuilder("spark.testing.nCoresPerExecutor")
.intConf
.createWithDefault(2)
+
+ val RESOURCES_WARNING_TESTING =
+ ConfigBuilder("spark.resources.warnings.testing").booleanConf.createWithDefault(false)
+
+ val RESOURCE_PROFILE_MANAGER_TESTING =
+ ConfigBuilder("spark.testing.resourceProfileManager")
+ .booleanConf
+ .createWithDefault(false)
+
+ val TASKSET_MANAGER_SPECULATION_TESTING =
+ ConfigBuilder("spark.testing.taskSetManagerSpeculation")
+ .booleanConf
+ .createWithDefault(false)
}
diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala
index 110198815c255..4e093f76eff6c 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/package.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala
@@ -251,6 +251,8 @@ package object config {
ConfigBuilder("spark.executor.userClassPathFirst").booleanConf.createWithDefault(false)
private[spark] val EXECUTOR_CORES = ConfigBuilder(SparkLauncher.EXECUTOR_CORES)
+ .doc("The number of cores to use on each executor. In standalone and Mesos " +
+ "coarse-grained modes this config may not be correct if not explicitly set.")
.intConf
.createWithDefault(1)
@@ -1492,6 +1494,11 @@ package object config {
.stringConf
.createWithDefault(SchedulingMode.FIFO.toString)
+ private[spark] val RESOURCE_PROFILE_MERGE_CONFLICTS =
+ ConfigBuilder("spark.scheduler.resourceProfile.mergeConflicts")
+ .booleanConf
+ .createWithDefault(false)
+
private[spark] val SCHEDULER_REVIVE_INTERVAL =
ConfigBuilder("spark.scheduler.revive.interval")
.timeConf(TimeUnit.MILLISECONDS)
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index 5788b70e75a7a..37e7180fed954 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -42,6 +42,7 @@ import org.apache.spark.partial.BoundedDouble
import org.apache.spark.partial.CountEvaluator
import org.apache.spark.partial.GroupedCountEvaluator
import org.apache.spark.partial.PartialResult
+import org.apache.spark.resource.ResourceProfile
import org.apache.spark.storage.{RDDBlockId, StorageLevel}
import org.apache.spark.util.{BoundedPriorityQueue, Utils}
import org.apache.spark.util.collection.{ExternalAppendOnlyMap, OpenHashMap,
@@ -1714,12 +1715,37 @@ abstract class RDD[T: ClassTag](
@Since("2.4.0")
def barrier(): RDDBarrier[T] = withScope(new RDDBarrier[T](this))
+ /**
+ * Specify a ResourceProfile to use when calculating this RDD. This is only supported on
+ * certain cluster managers and currently requires dynamic allocation to be enabled.
+ * It will result in new executors with the resources specified being acquired to
+ * calculate the RDD.
+ */
+ @Experimental
+ @Since("3.0.0")
+ def withResources(rp: ResourceProfile): this.type = {
+ resourceProfile = Some(rp)
+ sc.resourceProfileManager.addResourceProfile(resourceProfile.get)
+ this
+ }
+
+ /**
+ * Get the ResourceProfile specified with this RDD or None if it wasn't specified.
+ * @return the user specified ResourceProfile or null (for Java compiatibility) if
+ * none was specified
+ */
+ @Experimental
+ @Since("3.0.0")
+ def getResourceProfile(): ResourceProfile = resourceProfile.getOrElse(null)
+
// =======================================================================
// Other internal methods and fields
// =======================================================================
private var storageLevel: StorageLevel = StorageLevel.NONE
+ private var resourceProfile: Option[ResourceProfile] = None
+
/** User code that created this RDD (e.g. `textFile`, `parallelize`). */
@transient private[spark] val creationSite = sc.getCallSite()
@@ -1838,7 +1864,9 @@ abstract class RDD[T: ClassTag](
info.numCachedPartitions, bytesToString(info.memSize),
bytesToString(info.externalBlockStoreSize), bytesToString(info.diskSize)))
- s"$rdd [$persistence]" +: storageInfo
+ val resourceProfileInfo = rdd.resourceProfile.map(x => x.toString()).getOrElse("")
+
+ s"$rdd [$persistence][$resourceProfileInfo][isBarrier = $isBarrier()]" +: storageInfo
}
// Apply a different rule to the last child
diff --git a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequest.scala b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequest.scala
index 9a920914ed674..94aa40a0c6723 100644
--- a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequest.scala
+++ b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequest.scala
@@ -46,11 +46,8 @@ package org.apache.spark.resource
* allocated. The script runs on Executors startup to discover the addresses
* of the resources available.
* @param vendor Optional vendor, required for some cluster managers
- *
- * This api is currently private until the rest of the pieces are in place and then it
- * will become public.
*/
-private[spark] class ExecutorResourceRequest(
+class ExecutorResourceRequest(
val resourceName: String,
val amount: Long,
val discoveryScript: String = "",
diff --git a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala
index d345674d6635c..04fe39fc340eb 100644
--- a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala
+++ b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala
@@ -17,10 +17,12 @@
package org.apache.spark.resource
+import java.util.{Map => JMap}
import java.util.concurrent.ConcurrentHashMap
import scala.collection.JavaConverters._
+import org.apache.spark.internal.Logging
import org.apache.spark.network.util.JavaUtils
import org.apache.spark.resource.ResourceProfile._
@@ -28,16 +30,15 @@ import org.apache.spark.resource.ResourceProfile._
* A set of Executor resource requests. This is used in conjunction with the ResourceProfile to
* programmatically specify the resources needed for an RDD that will be applied at the
* stage level.
- *
- * This api is currently private until the rest of the pieces are in place and then it
- * will become public.
*/
-private[spark] class ExecutorResourceRequests() extends Serializable {
+class ExecutorResourceRequests() extends Serializable with Logging {
private val _executorResources = new ConcurrentHashMap[String, ExecutorResourceRequest]()
def requests: Map[String, ExecutorResourceRequest] = _executorResources.asScala.toMap
+ def requestsJMap: JMap[String, ExecutorResourceRequest] = _executorResources.asScala.asJava
+
/**
* Specify heap memory. The value specified will be converted to MiB.
*
@@ -46,11 +47,22 @@ private[spark] class ExecutorResourceRequests() extends Serializable {
*/
def memory(amount: String): this.type = {
val amountMiB = JavaUtils.byteStringAsMb(amount)
+ if (amountMiB <= 0) {
+ throw new IllegalArgumentException("Memory size must be > 0")
+ }
val req = new ExecutorResourceRequest(MEMORY, amountMiB)
_executorResources.put(MEMORY, req)
this
}
+ /**
+ * Removes any heap memory requests.
+ */
+ def removeMemory(): this.type = {
+ _executorResources.remove(MEMORY)
+ this
+ }
+
/**
* Specify overhead memory. The value specified will be converted to MiB.
*
@@ -59,11 +71,22 @@ private[spark] class ExecutorResourceRequests() extends Serializable {
*/
def memoryOverhead(amount: String): this.type = {
val amountMiB = JavaUtils.byteStringAsMb(amount)
+ if (amountMiB <= 0) {
+ throw new IllegalArgumentException("Overhead memory size must be > 0")
+ }
val req = new ExecutorResourceRequest(OVERHEAD_MEM, amountMiB)
_executorResources.put(OVERHEAD_MEM, req)
this
}
+ /**
+ * Removes any overhead memory requests.
+ */
+ def removeMemoryOverhead(): this.type = {
+ _executorResources.remove(OVERHEAD_MEM)
+ this
+ }
+
/**
* Specify pyspark memory. The value specified will be converted to MiB.
*
@@ -72,22 +95,44 @@ private[spark] class ExecutorResourceRequests() extends Serializable {
*/
def pysparkMemory(amount: String): this.type = {
val amountMiB = JavaUtils.byteStringAsMb(amount)
+ if (amountMiB <= 0) {
+ throw new IllegalArgumentException("Pyspark memory size must be > 0")
+ }
val req = new ExecutorResourceRequest(PYSPARK_MEM, amountMiB)
_executorResources.put(PYSPARK_MEM, req)
this
}
+ /**
+ * Removes any pyspark memory requests.
+ */
+ def removePysparkMemory(): this.type = {
+ _executorResources.remove(PYSPARK_MEM)
+ this
+ }
+
/**
* Specify number of cores per Executor.
*
* @param amount Number of cores to allocate per Executor.
*/
def cores(amount: Int): this.type = {
+ if (amount <= 0) {
+ throw new IllegalArgumentException("Cores amount must be > 0")
+ }
val req = new ExecutorResourceRequest(CORES, amount)
_executorResources.put(CORES, req)
this
}
+ /**
+ * Removes any executor core requests.
+ */
+ def removeCores(): this.type = {
+ _executorResources.remove(CORES)
+ this
+ }
+
/**
* Amount of a particular custom resource(GPU, FPGA, etc) to use. The resource names supported
* correspond to the regular Spark configs with the prefix removed. For instance, resources
@@ -110,11 +155,26 @@ private[spark] class ExecutorResourceRequests() extends Serializable {
vendor: String = ""): this.type = {
// a bit weird but for Java api use empty string as meaning None because empty
// string is otherwise invalid for those paramters anyway
+ if (amount <= 0) {
+ throw new IllegalArgumentException(s"$resourceName amount must be > 0")
+ }
val req = new ExecutorResourceRequest(resourceName, amount, discoveryScript, vendor)
_executorResources.put(resourceName, req)
this
}
+ /**
+ * Removes the specific custom resource requests.
+ *
+ * @param resourceName name of the resource to remove
+ */
+ def removeResource(resourceName: String): this.type = {
+ if (resourceName != null) {
+ _executorResources.remove(resourceName)
+ }
+ this
+ }
+
override def toString: String = {
s"Executor resource requests: ${_executorResources}"
}
diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala
index eb713a27be603..30c4f8759f20a 100644
--- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala
+++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala
@@ -22,8 +22,9 @@ import java.util.concurrent.atomic.AtomicInteger
import javax.annotation.concurrent.GuardedBy
import scala.collection.JavaConverters._
+import scala.collection.mutable
-import org.apache.spark.SparkConf
+import org.apache.spark.{SparkConf, SparkException}
import org.apache.spark.annotation.Evolving
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config._
@@ -42,6 +43,13 @@ class ResourceProfile(
// _id is only a var for testing purposes
private var _id = ResourceProfile.getNextProfileId
+ // this is used for any resources that use fractional amounts
+ private var _executorResourceNumParts: Option[Map[String, Int]] = None
+ private var _limitingResource: Option[String] = None
+ private var _maxTasksPerExecutor: Option[Int] = None
+ private var _coresLimitKnown: Boolean = false
+ private var _internalPysparkMemoryConf: Seq[(String, String)] =
+ ResourceProfile.createPysparkMemoryInternalConfs(this)
def id: Int = _id
@@ -57,6 +65,10 @@ class ResourceProfile(
executorResources.asJava
}
+ private[spark] def getInternalPysparkMemoryConfs: Seq[(String, String)] = {
+ _internalPysparkMemoryConf
+ }
+
// Note that some cluster managers don't set the executor cores explicitly so
// be sure to check the Option as required
private[spark] def getExecutorCores: Option[Int] = {
@@ -67,6 +79,112 @@ class ResourceProfile(
taskResources.get(ResourceProfile.CPUS).map(_.amount.toInt)
}
+ private[spark] def getNumSlotsPerAddress(resource: String, sparkConf: SparkConf): Int = {
+ _executorResourceNumParts.getOrElse {
+ calculateTasksAndLimitingResource(sparkConf)
+ }
+ _executorResourceNumParts.get.getOrElse(resource,
+ throw new SparkException(s"Resource $resource doesn't exist in profile id: $id"))
+ }
+
+ // maximum tasks you could put on an executor with this profile based on the limiting resource
+ // If the executor cores config is not present this value is based on the other resources
+ // available or 1 if no other resources. You need to check the isCoresLimitKnown to
+ // calculate proper value.
+ private[spark] def maxTasksPerExecutor(sparkConf: SparkConf): Int = {
+ _maxTasksPerExecutor.getOrElse {
+ calculateTasksAndLimitingResource(sparkConf)
+ _maxTasksPerExecutor.get
+ }
+ }
+
+ // Returns whether the executor cores was available to use to calculate the max tasks
+ // per executor and limiting resource.
+ private[spark] def isCoresLimitKnown: Boolean = _coresLimitKnown
+
+ // If the executor cores config is not present this value is based on the other resources
+ // available or 1 if no other resources. You need to check the isCoresLimitKnown to
+ // calculate proper value.
+ private[spark] def limitingResource(sparkConf: SparkConf): String = {
+ _limitingResource.getOrElse {
+ calculateTasksAndLimitingResource(sparkConf)
+ _limitingResource.get
+ }
+ }
+
+ /**
+ * Utility function to calculate the number of tasks you can run on a single Executor based
+ * on the task and executor resource requests in the ResourceProfile. This will be based
+ * off the resource that is most restrictive. For instance, if the executor
+ * request is for 4 cpus and 2 gpus and your task request is for 1 cpu and 1 gpu each, the
+ * limiting resource is gpu, and this function will return 2.
+ */
+ private def calculateTasksAndLimitingResource(sparkConf: SparkConf): Unit = synchronized {
+ val coresPerExecutor = getExecutorCores.getOrElse(sparkConf.get(EXECUTOR_CORES))
+ val master = sparkConf.getOption("spark.master")
+ // executor cores config is not set for some masters by default and the default value
+ // only applies to yarn/k8s
+ val shouldCheckExecCores = sparkConf.contains(EXECUTOR_CORES) ||
+ (master.isDefined && (master.get.equalsIgnoreCase("yarn") || master.get.startsWith("k8s")))
+ val cpusPerTask = taskResources.get(ResourceProfile.CPUS)
+ .map(_.amount).getOrElse(sparkConf.get(CPUS_PER_TASK).toDouble).toInt
+ if (shouldCheckExecCores) {
+ _coresLimitKnown = true
+ ResourceUtils.validateTaskCpusLargeEnough(sparkConf, coresPerExecutor, cpusPerTask)
+ }
+ val tasksBasedOnCores = coresPerExecutor / cpusPerTask
+ val numPartsMap = new mutable.HashMap[String, Int]
+ numPartsMap(ResourceProfile.CORES) = 1
+ // Note that if the cores per executor aren't set properly
+ // this calculation could be off, we default it to just be 1 in order to allow checking
+ // of the rest of the custom resources. We set the limit based on the other resources available.
+ var (taskLimit, limitingResource) = if (shouldCheckExecCores) {
+ (tasksBasedOnCores, ResourceProfile.CPUS)
+ } else {
+ (-1, "")
+ }
+ val taskResourcesToCheck = new mutable.HashMap[String, TaskResourceRequest]
+ taskResourcesToCheck ++= ResourceProfile.getCustomTaskResources(this)
+ val execResourceToCheck = ResourceProfile.getCustomExecutorResources(this)
+ execResourceToCheck.foreach { case (rName, execReq) =>
+ val taskReq = taskResources.get(rName).map(_.amount).getOrElse(0.0)
+ numPartsMap(rName) = 1
+ if (taskReq > 0.0) {
+ if (taskReq > execReq.amount) {
+ throw new SparkException(s"The executor resource: $rName, amount: ${execReq.amount} " +
+ s"needs to be >= the task resource request amount of $taskReq")
+ }
+ val (numPerTask, parts) = ResourceUtils.calculateAmountAndPartsForFraction(taskReq)
+ numPartsMap(rName) = parts
+ val numTasks = ((execReq.amount * parts) / numPerTask).toInt
+ if (taskLimit == -1 || numTasks < taskLimit) {
+ limitingResource = rName
+ taskLimit = numTasks
+ }
+ taskResourcesToCheck -= rName
+ } else {
+ logWarning(s"The executor resource config for resource: $rName was specified but " +
+ "no corresponding task resource request was specified.")
+ }
+ }
+ if (taskResourcesToCheck.nonEmpty) {
+ throw new SparkException("No executor resource configs were not specified for the " +
+ s"following task configs: ${taskResourcesToCheck.keys.mkString(",")}")
+ }
+ logInfo(s"Limiting resource is $limitingResource at $taskLimit tasks per executor")
+ _executorResourceNumParts = Some(numPartsMap.toMap)
+ _maxTasksPerExecutor = if (taskLimit == -1) Some(1) else Some(taskLimit)
+ _limitingResource = Some(limitingResource)
+ if (shouldCheckExecCores) {
+ ResourceUtils.warnOnWastedResources(this, sparkConf)
+ }
+ }
+
+ // to be used only by history server for reconstruction from events
+ private[spark] def setResourceProfileId(id: Int): Unit = {
+ _id = id
+ }
+
// testing only
private[spark] def setToDefaultProfile(): Unit = {
_id = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID
@@ -123,7 +241,7 @@ object ResourceProfile extends Logging {
val taskResources = getDefaultTaskResources(conf)
val executorResources = getDefaultExecutorResources(conf)
val defProf = new ResourceProfile(executorResources, taskResources)
- defProf.setToDefaultProfile
+ defProf.setToDefaultProfile()
defaultProfile = Some(defProf)
logInfo("Default ResourceProfile created, executor resources: " +
s"${defProf.executorResources}, task resources: " +
@@ -157,13 +275,12 @@ object ResourceProfile extends Logging {
// for testing only
private[spark] def reInitDefaultProfile(conf: SparkConf): Unit = {
- clearDefaultProfile
+ clearDefaultProfile()
// force recreate it after clearing
getOrCreateDefaultProfile(conf)
}
- // for testing only
- private[spark] def clearDefaultProfile: Unit = {
+ private[spark] def clearDefaultProfile(): Unit = {
DEFAULT_PROFILE_LOCK.synchronized {
defaultProfile = None
}
@@ -178,4 +295,47 @@ object ResourceProfile extends Logging {
rp: ResourceProfile): Map[String, ExecutorResourceRequest] = {
rp.executorResources.filterKeys(k => !ResourceProfile.allSupportedExecutorResources.contains(k))
}
+
+ private[spark] val SPARK_RP_EXEC_PREFIX = "spark.resourceProfile.executor"
+
+ private[spark] def resourceProfileIntConfPrefix(rpId: Int): String = {
+ s"$SPARK_RP_EXEC_PREFIX.$rpId."
+ }
+
+ // Helper class for constructing the resource profile internal configs. The configs look like:
+ // spark.resourceProfile.executor.[rpId].[resourceName].amount
+ private[spark] case class ResourceProfileInternalConf(prefix: String, resourceName: String) {
+ def resourceNameConf: String = s"$prefix$resourceName"
+ def resourceNameAndAmount: String = s"$resourceName.${ResourceUtils.AMOUNT}"
+ def amountConf: String = s"$prefix$resourceNameAndAmount"
+ }
+
+ /**
+ * Create the ResourceProfile internal pyspark memory conf that are used by the executors.
+ * It pulls any pyspark.memory config from the profile returns a Seq of key and value
+ * where the keys get formatted as:
+ *
+ * spark.resourceProfile.executor.[rpId].[resourceName].[amount, vendor, discoveryScript]
+ */
+ private[spark] def createPysparkMemoryInternalConfs(
+ rp: ResourceProfile
+ ): Seq[(String, String)] = {
+ rp.executorResources.get(ResourceProfile.PYSPARK_MEM).map { pysparkMem =>
+ val prefix = resourceProfileIntConfPrefix(rp.id)
+ val pysparkMemIntConf = ResourceProfileInternalConf(prefix, ResourceProfile.PYSPARK_MEM)
+ Seq((pysparkMemIntConf.amountConf, pysparkMem.amount.toString))
+ }.getOrElse(Seq.empty)
+ }
+
+ /**
+ * Get the pyspark memory from internal resource confs
+ * The config looks like: spark.resourceProfile.executor.[rpId].pyspark.memory.amount
+ */
+ private[spark] def getPysparkMemoryFromInternalConfs(
+ sparkConf: SparkConf,
+ rpId: Int): Option[Long] = {
+ val rName = ResourceProfile.PYSPARK_MEM
+ val intConf = ResourceProfileInternalConf(resourceProfileIntConfPrefix(rpId), rName)
+ sparkConf.getOption(intConf.amountConf).map(_.toLong)
+ }
}
diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala
new file mode 100644
index 0000000000000..a5332b919f2f1
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.resource
+
+import java.util.concurrent.ConcurrentHashMap
+
+import org.apache.spark.{SparkConf, SparkException}
+import org.apache.spark.annotation.Evolving
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
+import org.apache.spark.internal.config.Tests._
+import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerResourceProfileAdded}
+import org.apache.spark.util.Utils.isTesting
+
+/**
+ * Manager of resource profiles. The manager allows one place to keep the actual ResourceProfiles
+ * and everywhere else we can use the ResourceProfile Id to save on space.
+ * Note we never remove a resource profile at this point. Its expected this number if small
+ * so this shouldn't be much overhead.
+ */
+@Evolving
+private[spark] class ResourceProfileManager(sparkConf: SparkConf,
+ listenerBus: LiveListenerBus) extends Logging {
+ private val resourceProfileIdToResourceProfile =
+ new ConcurrentHashMap[Int, ResourceProfile]()
+
+ private val master = sparkConf.getOption("spark.master")
+
+ private val defaultProfile = ResourceProfile.getOrCreateDefaultProfile(sparkConf)
+ addResourceProfile(defaultProfile)
+
+ def defaultResourceProfile: ResourceProfile = defaultProfile
+
+ private val taskCpusDefaultProfile = defaultProfile.getTaskCpus.get
+ private val dynamicEnabled = sparkConf.get(DYN_ALLOCATION_ENABLED)
+ private val isNotYarn = master.isDefined && !master.get.equals("yarn")
+
+ // If we use anything except the default profile, its only supported on YARN right now.
+ // Throw an exception if not supported.
+ private[spark] def isSupported(rp: ResourceProfile): Boolean = {
+ // if the master isn't defined we go ahead and allow it for testing purposes
+ val shouldError = !isTesting || sparkConf.get(RESOURCE_PROFILE_MANAGER_TESTING)
+ val isNotDefaultProfile = rp.id != ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID
+ val notYarnAndNotDefaultProfile = isNotDefaultProfile && isNotYarn
+ val YarnNotDynAllocAndNotDefaultProfile = isNotDefaultProfile && !isNotYarn && !dynamicEnabled
+ if (shouldError && (notYarnAndNotDefaultProfile || YarnNotDynAllocAndNotDefaultProfile)) {
+ throw new SparkException("ResourceProfiles are only supported on YARN with dynamic " +
+ "allocation enabled.")
+ }
+ true
+ }
+
+ def addResourceProfile(rp: ResourceProfile): Unit = {
+ isSupported(rp)
+ // force the computation of maxTasks and limitingResource now so we don't have cost later
+ rp.limitingResource(sparkConf)
+ logInfo(s"Adding ResourceProfile id: ${rp.id}")
+ val wasPresent = resourceProfileIdToResourceProfile.putIfAbsent(rp.id, rp)
+ if (wasPresent == null) {
+ listenerBus.post(SparkListenerResourceProfileAdded(rp))
+ }
+ }
+
+ /*
+ * Gets the ResourceProfile associated with the id, if a profile doesn't exist
+ * it returns the default ResourceProfile created from the application level configs.
+ */
+ def resourceProfileFromId(rpId: Int): ResourceProfile = {
+ val rp = resourceProfileIdToResourceProfile.get(rpId)
+ if (rp == null) {
+ throw new SparkException(s"ResourceProfileId $rpId not found!")
+ }
+ rp
+ }
+
+ def taskCpusForProfileId(rpId: Int): Int = {
+ resourceProfileFromId(rpId).getTaskCpus.getOrElse(taskCpusDefaultProfile)
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala
index 190b0cdc8887a..cfa1c4d09738e 100644
--- a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala
+++ b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala
@@ -27,7 +27,8 @@ import org.json4s.jackson.JsonMethods._
import org.apache.spark.{SparkConf, SparkException}
import org.apache.spark.internal.Logging
-import org.apache.spark.internal.config.SPARK_TASK_PREFIX
+import org.apache.spark.internal.config.{CPUS_PER_TASK, EXECUTOR_CORES, SPARK_TASK_PREFIX}
+import org.apache.spark.internal.config.Tests.{RESOURCES_WARNING_TESTING, TASKSET_MANAGER_SPECULATION_TESTING}
import org.apache.spark.util.Utils.executeAndGetOutput
/**
@@ -344,6 +345,89 @@ private[spark] object ResourceUtils extends Logging {
discoverResource(resourceName, script)
}
+ def validateTaskCpusLargeEnough(sparkConf: SparkConf, execCores: Int, taskCpus: Int): Boolean = {
+ // Number of cores per executor must meet at least one task requirement.
+ if (!sparkConf.get(TASKSET_MANAGER_SPECULATION_TESTING)) {
+ if (execCores < taskCpus) {
+ throw new SparkException(s"The number of cores per executor (=$execCores) has to be >= " +
+ s"the number of cpus per task = $taskCpus.")
+ }
+ }
+ true
+ }
+
+ def warnOnWastedResources(
+ rp: ResourceProfile,
+ sparkConf: SparkConf,
+ execCores: Option[Int] = None): Unit = {
+ // There have been checks on the ResourceProfile to make sure the executor resources were
+ // specified and are large enough if any task resources were specified.
+ // Now just do some sanity test and log warnings when it looks like the user will
+ // waste some resources.
+ val coresKnown = rp.isCoresLimitKnown
+ var limitingResource = rp.limitingResource(sparkConf)
+ var maxTaskPerExec = rp.maxTasksPerExecutor(sparkConf)
+ val taskCpus = rp.getTaskCpus.getOrElse(sparkConf.get(CPUS_PER_TASK))
+ val cores = if (execCores.isDefined) {
+ execCores.get
+ } else if (coresKnown) {
+ rp.getExecutorCores.getOrElse(sparkConf.get(EXECUTOR_CORES))
+ } else {
+ return
+ }
+ // when executor cores config isn't set, we can't calculate the real limiting resource
+ // and number of tasks per executor ahead of time, so calculate it now.
+ if (!coresKnown) {
+ val numTasksPerExecCores = cores / taskCpus
+ val numTasksPerExecCustomResource = rp.maxTasksPerExecutor(sparkConf)
+ if (limitingResource.isEmpty ||
+ (limitingResource.nonEmpty && numTasksPerExecCores < numTasksPerExecCustomResource)) {
+ limitingResource = ResourceProfile.CPUS
+ maxTaskPerExec = numTasksPerExecCores
+ }
+ }
+ val taskReq = ResourceProfile.getCustomTaskResources(rp)
+ val execReq = ResourceProfile.getCustomExecutorResources(rp)
+
+ if (limitingResource.nonEmpty && !limitingResource.equals(ResourceProfile.CPUS)) {
+ if ((taskCpus * maxTaskPerExec) < cores) {
+ val resourceNumSlots = Math.floor(cores/taskCpus).toInt
+ val message = s"The configuration of cores (exec = ${cores} " +
+ s"task = ${taskCpus}, runnable tasks = ${resourceNumSlots}) will " +
+ s"result in wasted resources due to resource ${limitingResource} limiting the " +
+ s"number of runnable tasks per executor to: ${maxTaskPerExec}. Please adjust " +
+ s"your configuration."
+ if (sparkConf.get(RESOURCES_WARNING_TESTING)) {
+ throw new SparkException(message)
+ } else {
+ logWarning(message)
+ }
+ }
+ }
+
+ taskReq.foreach { case (rName, treq) =>
+ val execAmount = execReq(rName).amount
+ val numParts = rp.getNumSlotsPerAddress(rName, sparkConf)
+ // handle fractional
+ val taskAmount = if (numParts > 1) 1 else treq.amount
+ if (maxTaskPerExec < (execAmount * numParts / taskAmount)) {
+ val taskReqStr = s"${taskAmount}/${numParts}"
+ val resourceNumSlots = Math.floor(execAmount * numParts / taskAmount).toInt
+ val message = s"The configuration of resource: ${treq.resourceName} " +
+ s"(exec = ${execAmount}, task = ${taskReqStr}, " +
+ s"runnable tasks = ${resourceNumSlots}) will " +
+ s"result in wasted resources due to resource ${limitingResource} limiting the " +
+ s"number of runnable tasks per executor to: ${maxTaskPerExec}. Please adjust " +
+ s"your configuration."
+ if (sparkConf.get(RESOURCES_WARNING_TESTING)) {
+ throw new SparkException(message)
+ } else {
+ logWarning(message)
+ }
+ }
+ }
+ }
+
// known types of resources
final val GPU: String = "gpu"
final val FPGA: String = "fpga"
diff --git a/core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala
index bffb0a2f523b1..45538b5adeeb1 100644
--- a/core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala
+++ b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequest.scala
@@ -23,11 +23,8 @@ package org.apache.spark.resource
* stage level.
*
* Use TaskResourceRequests class as a convenience API.
- *
- * This api is currently private until the rest of the pieces are in place and then it
- * will become public.
*/
-private[spark] class TaskResourceRequest(val resourceName: String, val amount: Double)
+class TaskResourceRequest(val resourceName: String, val amount: Double)
extends Serializable {
assert(amount <= 0.5 || amount % 1 == 0,
diff --git a/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala
index 9624b51dd158e..31e0b31672224 100644
--- a/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala
+++ b/core/src/main/scala/org/apache/spark/resource/TaskResourceRequests.scala
@@ -17,6 +17,7 @@
package org.apache.spark.resource
+import java.util.{Map => JMap}
import java.util.concurrent.ConcurrentHashMap
import scala.collection.JavaConverters._
@@ -27,22 +28,24 @@ import org.apache.spark.resource.ResourceProfile._
* A set of task resource requests. This is used in conjuntion with the ResourceProfile to
* programmatically specify the resources needed for an RDD that will be applied at the
* stage level.
- *
- * This api is currently private until the rest of the pieces are in place and then it
- * will become public.
*/
-private[spark] class TaskResourceRequests() extends Serializable {
+class TaskResourceRequests() extends Serializable {
private val _taskResources = new ConcurrentHashMap[String, TaskResourceRequest]()
def requests: Map[String, TaskResourceRequest] = _taskResources.asScala.toMap
+ def requestsJMap: JMap[String, TaskResourceRequest] = _taskResources.asScala.asJava
+
/**
* Specify number of cpus per Task.
*
* @param amount Number of cpus to allocate per Task.
*/
def cpus(amount: Int): this.type = {
+ if (amount <= 0) {
+ throw new IllegalArgumentException(s"Cpus amount must be > 0")
+ }
val treq = new TaskResourceRequest(CPUS, amount)
_taskResources.put(CPUS, treq)
this
@@ -58,6 +61,9 @@ private[spark] class TaskResourceRequests() extends Serializable {
* ie amount equals 0.5 translates into 2 tasks per resource address.
*/
def resource(resourceName: String, amount: Double): this.type = {
+ if (amount <= 0) {
+ throw new IllegalArgumentException(s"$resourceName amount must be > 0")
+ }
val treq = new TaskResourceRequest(resourceName, amount)
_taskResources.put(resourceName, treq)
this
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index 7bf363dd71c1b..f0692427c8ecb 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -38,6 +38,7 @@ import org.apache.spark.internal.config.Tests.TEST_NO_STAGE_RETRY
import org.apache.spark.network.util.JavaUtils
import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult}
import org.apache.spark.rdd.{DeterministicLevel, RDD, RDDCheckpointData}
+import org.apache.spark.resource.ResourceProfile
import org.apache.spark.rpc.RpcTimeout
import org.apache.spark.storage._
import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat
@@ -185,6 +186,8 @@ private[spark] class DAGScheduler(
/** If enabled, FetchFailed will not cause stage retry, in order to surface the problem. */
private val disallowStageRetryForTest = sc.getConf.get(TEST_NO_STAGE_RETRY)
+ private val shouldMergeResourceProfiles = sc.getConf.get(config.RESOURCE_PROFILE_MERGE_CONFLICTS)
+
/**
* Whether to unregister all the outputs on the host in condition that we receive a FetchFailure,
* this is set default to false, which means, we only unregister the outputs related to the exact
@@ -384,14 +387,23 @@ private[spark] class DAGScheduler(
def createShuffleMapStage[K, V, C](
shuffleDep: ShuffleDependency[K, V, C], jobId: Int): ShuffleMapStage = {
val rdd = shuffleDep.rdd
+
+ // something we may consider later is do we want to track the merged profile
+ // so we don't recreate it if the user runs the same operation multiple times
+ val resourceProfile = mergeResourceProfilesForStage(rdd)
+ // this ResourceProfile could be different if it was merged so we have to add it to
+ // our ResourceProfileManager
+ sc.resourceProfileManager.addResourceProfile(resourceProfile)
checkBarrierStageWithDynamicAllocation(rdd)
- checkBarrierStageWithNumSlots(rdd)
+ checkBarrierStageWithNumSlots(rdd, resourceProfile)
checkBarrierStageWithRDDChainPattern(rdd, rdd.getNumPartitions)
+
val numTasks = rdd.partitions.length
val parents = getOrCreateParentStages(rdd, jobId)
val id = nextStageId.getAndIncrement()
val stage = new ShuffleMapStage(
- id, rdd, numTasks, parents, jobId, rdd.creationSite, shuffleDep, mapOutputTracker)
+ id, rdd, numTasks, parents, jobId, rdd.creationSite, shuffleDep, mapOutputTracker,
+ resourceProfile.id)
stageIdToStage(id) = stage
shuffleIdToMapStage(shuffleDep.shuffleId) = stage
@@ -431,14 +443,78 @@ private[spark] class DAGScheduler(
* the check fails consecutively beyond a configured number for a job, then fail current job
* submission.
*/
- private def checkBarrierStageWithNumSlots(rdd: RDD[_]): Unit = {
+ private def checkBarrierStageWithNumSlots(rdd: RDD[_], rp: ResourceProfile): Unit = {
val numPartitions = rdd.getNumPartitions
- val maxNumConcurrentTasks = sc.maxNumConcurrentTasks
+ val maxNumConcurrentTasks = sc.maxNumConcurrentTasks(rp)
if (rdd.isBarrier() && numPartitions > maxNumConcurrentTasks) {
throw new BarrierJobSlotsNumberCheckFailed(numPartitions, maxNumConcurrentTasks)
}
}
+ // This is a basic merge resource profiles function that takes the max
+ // value of the profiles. We may want to make this more complex in the future as
+ // you may want to sum some resources (like memory).
+ private[scheduler] def mergeResourceProfiles(
+ r1: ResourceProfile,
+ r2: ResourceProfile): ResourceProfile = {
+ val mergedExecKeys = r1.executorResources.keySet ++ r2.executorResources.keySet
+ val mergedExecReq = mergedExecKeys.map { rName =>
+ // For now we just choose the max resource value - eventually we may want to
+ // expand to do sum or something user specified
+ val r2ri = r2.executorResources.get(rName)
+ val r1ri = r1.executorResources.get(rName)
+ if (r2ri.isDefined && r1ri.isDefined) {
+ if (r2ri.get.amount > r1ri.get.amount) (rName, r2ri.get) else (rName, r1ri.get)
+ } else if (r2ri.isDefined) {
+ (rName, r2ri.get)
+ } else {
+ (rName, r1ri.get)
+ }
+ }.toMap
+ val mergedTaskKeys = r1.taskResources.keySet ++ r2.taskResources.keySet
+ val mergedTaskReq = mergedTaskKeys.map { rName =>
+ // For now we just choose the max resource value - eventually we may want to
+ // expand to do sum or something user specified
+ val r2ri = r2.taskResources.get(rName)
+ val r1ri = r1.taskResources.get(rName)
+ if (r2ri.isDefined && r1ri.isDefined) {
+ if (r2ri.get.amount > r1ri.get.amount) (rName, r2ri.get) else (rName, r1ri.get)
+ } else if (r2ri.isDefined) {
+ (rName, r2ri.get)
+ } else {
+ (rName, r1ri.get)
+ }
+ }.toMap
+ new ResourceProfile(mergedExecReq, mergedTaskReq)
+ }
+
+ private[scheduler] def mergeResourceProfilesForStage(rdd: RDD[_]): ResourceProfile = {
+ val stageResourceProfiles = getResourceProfilesForRDDsInStage(rdd)
+ logDebug("rdd profiles: " + stageResourceProfiles)
+ val resourceProfile = if (stageResourceProfiles.size > 1) {
+ if (shouldMergeResourceProfiles) {
+ // need to resolve conflicts if multiple
+ var mergedProfile: ResourceProfile = stageResourceProfiles.head
+ for (profile <- stageResourceProfiles.drop(1)) {
+ mergedProfile = mergeResourceProfiles(mergedProfile, profile)
+ }
+ mergedProfile
+ } else {
+ throw new IllegalArgumentException("Multiple ResourceProfile's specified in the RDDs for " +
+ "this stage, either resolve the conflict ResourceProfile's yourself or enable " +
+ "spark.scheduler.resourceProfile.mergeConflicts and understand how Spark handles " +
+ "the conflicts.")
+ }
+ } else {
+ if (stageResourceProfiles.size == 1) {
+ stageResourceProfiles.head
+ } else {
+ sc.resourceProfileManager.defaultResourceProfile
+ }
+ }
+ resourceProfile
+ }
+
/**
* Create a ResultStage associated with the provided jobId.
*/
@@ -448,12 +524,22 @@ private[spark] class DAGScheduler(
partitions: Array[Int],
jobId: Int,
callSite: CallSite): ResultStage = {
+ // something we may consider later is do we want to track the merged profile
+ // so we don't recreate it if the user runs the same operation multiple times
+ val resourceProfile = mergeResourceProfilesForStage(rdd)
+ // this ResourceProfile could be different if it was merged so we have to add it to
+ // our ResourceProfileManager.
+ sc.resourceProfileManager.addResourceProfile(resourceProfile)
+
checkBarrierStageWithDynamicAllocation(rdd)
- checkBarrierStageWithNumSlots(rdd)
+ checkBarrierStageWithNumSlots(rdd, resourceProfile)
checkBarrierStageWithRDDChainPattern(rdd, partitions.toSet.size)
+
val parents = getOrCreateParentStages(rdd, jobId)
val id = nextStageId.getAndIncrement()
- val stage = new ResultStage(id, rdd, func, partitions, parents, jobId, callSite)
+ logInfo("creating result stage with profile: " + resourceProfile)
+ val stage = new ResultStage(id, rdd, func, partitions, parents, jobId,
+ callSite, resourceProfile.id)
stageIdToStage(id) = stage
updateJobIdStageIdMaps(jobId, stage)
stage
@@ -526,6 +612,28 @@ private[spark] class DAGScheduler(
parents
}
+ private[scheduler] def getResourceProfilesForRDDsInStage(
+ rdd: RDD[_]): HashSet[ResourceProfile] = {
+ val resourceProfiles = new HashSet[ResourceProfile]
+ val visited = new HashSet[RDD[_]]
+ val waitingForVisit = new ListBuffer[RDD[_]]
+ waitingForVisit += rdd
+ while (waitingForVisit.nonEmpty) {
+ val toVisit = waitingForVisit.remove(0)
+ if (!visited(toVisit)) {
+ visited += toVisit
+ Option(toVisit.getResourceProfile).foreach(resourceProfiles += _)
+ toVisit.dependencies.foreach {
+ case _: ShuffleDependency[_, _, _] =>
+ // Not within the same stage with current rdd, do nothing.
+ case dependency =>
+ waitingForVisit.prepend(dependency.rdd)
+ }
+ }
+ }
+ resourceProfiles
+ }
+
/**
* Traverses the given RDD and its ancestors within the same stage and checks whether all of the
* RDDs satisfy a given predicate.
@@ -1225,7 +1333,8 @@ private[spark] class DAGScheduler(
stage.pendingPartitions += id
new ShuffleMapTask(stage.id, stage.latestInfo.attemptNumber,
taskBinary, part, locs, properties, serializedTaskMetrics, Option(jobId),
- Option(sc.applicationId), sc.applicationAttemptId, stage.rdd.isBarrier())
+ Option(sc.applicationId), sc.applicationAttemptId, stage.rdd.isBarrier(),
+ stage.resourceProfileId)
}
case stage: ResultStage =>
@@ -1236,7 +1345,7 @@ private[spark] class DAGScheduler(
new ResultTask(stage.id, stage.latestInfo.attemptNumber,
taskBinary, part, locs, id, properties, serializedTaskMetrics,
Option(jobId), Option(sc.applicationId), sc.applicationAttemptId,
- stage.rdd.isBarrier())
+ stage.rdd.isBarrier(), stage.resourceProfileId)
}
}
} catch {
@@ -1250,7 +1359,8 @@ private[spark] class DAGScheduler(
logInfo(s"Submitting ${tasks.size} missing tasks from $stage (${stage.rdd}) (first 15 " +
s"tasks are for partitions ${tasks.take(15).map(_.partitionId)})")
taskScheduler.submitTasks(new TaskSet(
- tasks.toArray, stage.id, stage.latestInfo.attemptNumber, jobId, properties))
+ tasks.toArray, stage.id, stage.latestInfo.attemptNumber, jobId, properties,
+ stage.resourceProfileId))
} else {
// Because we posted SparkListenerStageSubmitted earlier, we should mark
// the stage as completed here in case there are no tasks to run
diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
index 8c23388b37a3d..a52f1afec984d 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
@@ -235,6 +235,10 @@ private[spark] class EventLoggingListener(
}
}
+ override def onResourceProfileAdded(event: SparkListenerResourceProfileAdded): Unit = {
+ logEvent(event, flushLogger = true)
+ }
+
override def onOtherEvent(event: SparkListenerEvent): Unit = {
if (event.logEvent) {
logEvent(event, flushLogger = true)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala
index d1687830ff7bf..7fdc3186e86bd 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala
@@ -34,8 +34,9 @@ private[spark] class ResultStage(
val partitions: Array[Int],
parents: List[Stage],
firstJobId: Int,
- callSite: CallSite)
- extends Stage(id, rdd, partitions.length, parents, firstJobId, callSite) {
+ callSite: CallSite,
+ resourceProfileId: Int)
+ extends Stage(id, rdd, partitions.length, parents, firstJobId, callSite, resourceProfileId) {
/**
* The active job for this result stage. Will be empty if the job has already finished
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
index 857c89d7a98f5..49d77f0fd9a10 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
@@ -25,6 +25,7 @@ import java.util.Properties
import org.apache.spark._
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.RDD
+import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID
/**
* A task that sends back the output to the driver application.
@@ -63,9 +64,10 @@ private[spark] class ResultTask[T, U](
jobId: Option[Int] = None,
appId: Option[String] = None,
appAttemptId: Option[String] = None,
- isBarrier: Boolean = false)
+ isBarrier: Boolean = false,
+ resourceProfileId: Int = DEFAULT_RESOURCE_PROFILE_ID)
extends Task[U](stageId, stageAttemptId, partition.index, localProperties, serializedTaskMetrics,
- jobId, appId, appAttemptId, isBarrier)
+ jobId, appId, appAttemptId, isBarrier, resourceProfileId)
with Serializable {
@transient private[this] val preferredLocs: Seq[TaskLocation] = {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala
index 9159d2a0158d5..c3093219f40c5 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala
@@ -17,6 +17,8 @@
package org.apache.spark.scheduler
+import org.apache.spark.resource.ResourceProfile
+
/**
* A backend interface for scheduling systems that allows plugging in different ones under
* TaskSchedulerImpl. We assume a Mesos-like model where the application gets resource offers as
@@ -77,12 +79,14 @@ private[spark] trait SchedulerBackend {
def getDriverAttributes: Option[Map[String, String]] = None
/**
- * Get the max number of tasks that can be concurrent launched currently.
+ * Get the max number of tasks that can be concurrent launched based on the ResourceProfile
+ * being used.
* Note that please don't cache the value returned by this method, because the number can change
* due to add/remove executors.
*
+ * @param rp ResourceProfile which to use to calculate max concurrent tasks.
* @return The max number of tasks that can be concurrent launched currently.
*/
- def maxNumConcurrentTasks(): Int
+ def maxNumConcurrentTasks(rp: ResourceProfile): Int
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala
index 1b44d0aee3195..be1984de9837f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala
@@ -42,8 +42,9 @@ private[spark] class ShuffleMapStage(
firstJobId: Int,
callSite: CallSite,
val shuffleDep: ShuffleDependency[_, _, _],
- mapOutputTrackerMaster: MapOutputTrackerMaster)
- extends Stage(id, rdd, numTasks, parents, firstJobId, callSite) {
+ mapOutputTrackerMaster: MapOutputTrackerMaster,
+ resourceProfileId: Int)
+ extends Stage(id, rdd, numTasks, parents, firstJobId, callSite, resourceProfileId) {
private[this] var _mapStageJobs: List[ActiveJob] = Nil
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
index 4c0c30a3caf67..2e97d2d0cdfb2 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
@@ -25,6 +25,7 @@ import org.apache.spark._
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.internal.{config, Logging}
import org.apache.spark.rdd.RDD
+import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID
/**
* A ShuffleMapTask divides the elements of an RDD into multiple buckets (based on a partitioner
@@ -60,9 +61,10 @@ private[spark] class ShuffleMapTask(
jobId: Option[Int] = None,
appId: Option[String] = None,
appAttemptId: Option[String] = None,
- isBarrier: Boolean = false)
+ isBarrier: Boolean = false,
+ resourceProfileId: Int = DEFAULT_RESOURCE_PROFILE_ID)
extends Task[MapStatus](stageId, stageAttemptId, partition.index, localProperties,
- serializedTaskMetrics, jobId, appId, appAttemptId, isBarrier)
+ serializedTaskMetrics, jobId, appId, appAttemptId, isBarrier, resourceProfileId)
with Logging {
/** A constructor used only in test suites. This does not require passing in an RDD. */
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
index c150b0341500c..850c1754dffb7 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
@@ -27,6 +27,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo
import org.apache.spark.TaskEndReason
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics}
+import org.apache.spark.resource.ResourceProfile
import org.apache.spark.scheduler.cluster.ExecutorInfo
import org.apache.spark.storage.{BlockManagerId, BlockUpdatedInfo}
@@ -207,6 +208,10 @@ case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent
@DeveloperApi
case class SparkListenerLogStart(sparkVersion: String) extends SparkListenerEvent
+@DeveloperApi
+case class SparkListenerResourceProfileAdded(resourceProfile: ResourceProfile)
+ extends SparkListenerEvent
+
/**
* Interface for listening to events from the Spark scheduler. Most applications should probably
* extend SparkListener or SparkFirehoseListener directly, rather than implementing this class.
@@ -348,8 +353,12 @@ private[spark] trait SparkListenerInterface {
* Called when other events like SQL-specific events are posted.
*/
def onOtherEvent(event: SparkListenerEvent): Unit
-}
+ /**
+ * Called when a Resource Profile is added to the manager.
+ */
+ def onResourceProfileAdded(event: SparkListenerResourceProfileAdded): Unit
+}
/**
* :: DeveloperApi ::
@@ -421,4 +430,6 @@ abstract class SparkListener extends SparkListenerInterface {
speculativeTask: SparkListenerSpeculativeTaskSubmitted): Unit = { }
override def onOtherEvent(event: SparkListenerEvent): Unit = { }
+
+ override def onResourceProfileAdded(event: SparkListenerResourceProfileAdded): Unit = { }
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
index 8f6b7ad309602..3d316c948db7e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
@@ -79,6 +79,8 @@ private[spark] trait SparkListenerBus
listener.onBlockUpdated(blockUpdated)
case speculativeTaskSubmitted: SparkListenerSpeculativeTaskSubmitted =>
listener.onSpeculativeTaskSubmitted(speculativeTaskSubmitted)
+ case resourceProfileAdded: SparkListenerResourceProfileAdded =>
+ listener.onResourceProfileAdded(resourceProfileAdded)
case _ => listener.onOtherEvent(event)
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
index a9f72eae71368..ca4528dbe6c48 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
@@ -52,6 +52,7 @@ import org.apache.spark.util.CallSite
* @param firstJobId ID of the first job this stage was part of, for FIFO scheduling.
* @param callSite Location in the user program associated with this stage: either where the target
* RDD was created, for a shuffle map stage, or where the action for a result stage was called.
+ * @param Int The ResourceProfileId to use with this Stage
*/
private[scheduler] abstract class Stage(
val id: Int,
@@ -59,7 +60,8 @@ private[scheduler] abstract class Stage(
val numTasks: Int,
val parents: List[Stage],
val firstJobId: Int,
- val callSite: CallSite)
+ val callSite: CallSite,
+ val resourceProfileId: Int)
extends Logging {
val numPartitions = rdd.partitions.length
@@ -79,7 +81,8 @@ private[scheduler] abstract class Stage(
* StageInfo to tell SparkListeners when a job starts (which happens before any stage attempts
* have been created).
*/
- private var _latestInfo: StageInfo = StageInfo.fromStage(this, nextAttemptId)
+ private var _latestInfo: StageInfo = StageInfo.fromStage(this, nextAttemptId,
+ resourceProfileId = resourceProfileId)
/**
* Set of stage attempt IDs that have failed. We keep track of these failures in order to avoid
@@ -100,7 +103,8 @@ private[scheduler] abstract class Stage(
val metrics = new TaskMetrics
metrics.register(rdd.sparkContext)
_latestInfo = StageInfo.fromStage(
- this, nextAttemptId, Some(numPartitionsToCompute), metrics, taskLocalityPreferences)
+ this, nextAttemptId, Some(numPartitionsToCompute), metrics, taskLocalityPreferences,
+ resourceProfileId = resourceProfileId)
nextAttemptId += 1
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
index fdc50328b43d8..556478d83cf39 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
@@ -38,7 +38,8 @@ class StageInfo(
val details: String,
val taskMetrics: TaskMetrics = null,
private[spark] val taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty,
- private[spark] val shuffleDepId: Option[Int] = None) {
+ private[spark] val shuffleDepId: Option[Int] = None,
+ val resourceProfileId: Int) {
/** 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. */
@@ -87,7 +88,8 @@ private[spark] object StageInfo {
attemptId: Int,
numTasks: Option[Int] = None,
taskMetrics: TaskMetrics = null,
- taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty
+ taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty,
+ resourceProfileId: Int
): StageInfo = {
val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd)
val rddInfos = Seq(RDDInfo.fromRdd(stage.rdd)) ++ ancestorRddInfos
@@ -105,6 +107,7 @@ private[spark] object StageInfo {
stage.details,
taskMetrics,
taskLocalityPreferences,
- shuffleDepId)
+ shuffleDepId,
+ resourceProfileId)
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
index ebc1c05435fee..aaa1e77e816e2 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
@@ -27,6 +27,7 @@ import org.apache.spark.memory.{MemoryMode, TaskMemoryManager}
import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.rdd.InputFileBlockHolder
import org.apache.spark.resource.ResourceInformation
+import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID
import org.apache.spark.util._
/**
@@ -65,7 +66,8 @@ private[spark] abstract class Task[T](
val jobId: Option[Int] = None,
val appId: Option[String] = None,
val appAttemptId: Option[String] = None,
- val isBarrier: Boolean = false) extends Serializable {
+ val isBarrier: Boolean = false,
+ val resourceProfileId: Int = DEFAULT_RESOURCE_PROFILE_ID) extends Serializable {
@transient lazy val metrics: TaskMetrics =
SparkEnv.get.closureSerializer.newInstance().deserialize(ByteBuffer.wrap(serializedTaskMetrics))
@@ -96,7 +98,8 @@ private[spark] abstract class Task[T](
localProperties,
metricsSystem,
metrics,
- resources)
+ resources,
+ resourceProfileId)
context = if (isBarrier) {
new BarrierTaskContext(taskContext)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala
index 247cfe721b553..22edc6ab8c949 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala
@@ -53,6 +53,7 @@ private[spark] class TaskDescription(
val name: String,
val index: Int, // Index within this task's TaskSet
val partitionId: Int,
+ val cpus: Int,
val addedFiles: Map[String, Long],
val addedJars: Map[String, Long],
val properties: Properties,
@@ -92,6 +93,7 @@ private[spark] object TaskDescription {
dataOut.writeUTF(taskDescription.name)
dataOut.writeInt(taskDescription.index)
dataOut.writeInt(taskDescription.partitionId)
+ dataOut.writeInt(taskDescription.cpus)
// Write files.
serializeStringLongMap(taskDescription.addedFiles, dataOut)
@@ -160,6 +162,7 @@ private[spark] object TaskDescription {
val name = dataIn.readUTF()
val index = dataIn.readInt()
val partitionId = dataIn.readInt()
+ val cpus = dataIn.readInt()
// Read files.
val taskFiles = deserializeStringLongMap(dataIn)
@@ -184,7 +187,7 @@ private[spark] object TaskDescription {
// Create a sub-buffer for the serialized task into its own buffer (to be deserialized later).
val serializedTask = byteBuffer.slice()
- new TaskDescription(taskId, attemptNumber, executorId, name, index, partitionId, taskFiles,
- taskJars, properties, resources, serializedTask)
+ new TaskDescription(taskId, attemptNumber, executorId, name, index, partitionId, cpus,
+ taskFiles, taskJars, properties, resources, serializedTask)
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index f25a36c7af22a..ae683ac58898e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -31,7 +31,7 @@ import org.apache.spark.TaskState.TaskState
import org.apache.spark.executor.ExecutorMetrics
import org.apache.spark.internal.{config, Logging}
import org.apache.spark.internal.config._
-import org.apache.spark.resource.ResourceUtils
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile}
import org.apache.spark.rpc.RpcEndpoint
import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
import org.apache.spark.scheduler.TaskLocality.TaskLocality
@@ -93,9 +93,6 @@ private[spark] class TaskSchedulerImpl(
// CPUs to request per task
val CPUS_PER_TASK = conf.get(config.CPUS_PER_TASK)
- // Resources to request per task
- val resourcesReqsPerTask = ResourceUtils.parseResourceRequirements(sc.conf, SPARK_TASK_PREFIX)
-
// TaskSetManagers are not thread safe, so any access to one should be synchronized
// on this class. Protected by `this`
private val taskSetsByStageIdAndAttempt = new HashMap[Int, HashMap[Int, TaskSetManager]]
@@ -209,7 +206,8 @@ private[spark] class TaskSchedulerImpl(
override def submitTasks(taskSet: TaskSet): Unit = {
val tasks = taskSet.tasks
- logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks")
+ logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks "
+ + "resource profile " + taskSet.resourceProfileId)
this.synchronized {
val manager = createTaskSetManager(taskSet, maxTaskFailures)
val stage = taskSet.stageId
@@ -340,20 +338,28 @@ private[spark] class TaskSchedulerImpl(
for (i <- 0 until shuffledOffers.size) {
val execId = shuffledOffers(i).executorId
val host = shuffledOffers(i).host
- if (availableCpus(i) >= CPUS_PER_TASK &&
- resourcesMeetTaskRequirements(availableResources(i))) {
+ // the specific resource assignments that would be given to a task
+ val taskResAssignments = HashMap[String, ResourceInformation]()
+ val taskSetRpID = taskSet.taskSet.resourceProfileId
+ // make the resource profile id a hard requirement for now - ie only put tasksets
+ // on executors where resource profile exactly matches.
+ if (taskSetRpID == shuffledOffers(i).resourceProfileId &&
+ resourcesMeetTaskRequirements(taskSet, availableCpus(i), availableResources(i),
+ taskResAssignments)) {
try {
- for (task <- taskSet.resourceOffer(execId, host, maxLocality, availableResources(i))) {
+ val taskCpus = sc.resourceProfileManager.taskCpusForProfileId(taskSetRpID)
+ for (task <- taskSet.resourceOffer(execId, host, maxLocality, taskCpus,
+ taskResAssignments.toMap)) {
tasks(i) += task
val tid = task.taskId
taskIdToTaskSetManager.put(tid, taskSet)
taskIdToExecutorId(tid) = execId
executorIdToRunningTaskIds(execId).add(tid)
- availableCpus(i) -= CPUS_PER_TASK
+ availableCpus(i) -= task.cpus
assert(availableCpus(i) >= 0)
task.resources.foreach { case (rName, rInfo) =>
// Remove the first n elements from availableResources addresses, these removed
- // addresses are the same as that we allocated in taskSet.resourceOffer() since it's
+ // addresses are the same as that we allocated in taskResourceAssignments since it's
// synchronized. We don't remove the exact addresses allocated because the current
// approach produces the identical result with less time complexity.
availableResources(i).getOrElse(rName,
@@ -381,10 +387,87 @@ private[spark] class TaskSchedulerImpl(
/**
* Check whether the resources from the WorkerOffer are enough to run at least one task.
+ * The only task resources we look at from global confs are CPUs if not specified in the
+ * ResourceProfile, otherwise global task resources are ignored if a ResourceProfile is
+ * specified.
*/
- private def resourcesMeetTaskRequirements(resources: Map[String, Buffer[String]]): Boolean = {
- val resourcesFree = resources.map(r => r._1 -> r._2.length)
- ResourceUtils.resourcesMeetRequirements(resourcesFree, resourcesReqsPerTask)
+ private def resourcesMeetTaskRequirements(
+ taskSet: TaskSetManager,
+ availCpus: Int,
+ availWorkerResources: Map[String, Buffer[String]],
+ taskResourceAssignments: HashMap[String, ResourceInformation]): Boolean = {
+ val rpId = taskSet.taskSet.resourceProfileId
+ val taskCpus = sc.resourceProfileManager.taskCpusForProfileId(rpId)
+ // check is ResourceProfile has cpus first since that is common case
+ if (availCpus < taskCpus) return false
+
+ val taskSetProf = sc.resourceProfileManager.resourceProfileFromId(rpId)
+ // remove task cpus since we checked already
+ val tsResources = taskSetProf.taskResources.filterKeys(!_.equals(ResourceProfile.CPUS))
+ if (tsResources.isEmpty) return true
+ val localTaskReqAssign = HashMap[String, ResourceInformation]()
+ // we go through all resources here so that we can make sure they match and also get what the
+ // assignments are for the next task
+ for ((rName, taskReqs) <- tsResources) {
+ availWorkerResources.get(rName) match {
+ case Some(workerRes) =>
+ val taskReqAmount = taskReqs.amount
+ val workerAvail = availWorkerResources.get(rName).map(_.size).getOrElse(0)
+ if (workerAvail >= taskReqAmount) {
+ localTaskReqAssign.put(rName, new ResourceInformation(rName,
+ workerRes.take(taskReqAmount.toInt).toArray))
+ } else {
+ return false
+ }
+ case None => return false
+ }
+ }
+ taskResourceAssignments ++= localTaskReqAssign
+ true
+ }
+
+ // Use the resource that the resourceProfile has as the limiting resource to calculate the
+ // total number of slots available based on the current offers.
+ private def calculateAvailableSlots(
+ resourceProfileIds: Array[Int],
+ availableCpus: Array[Int],
+ availableResources: Array[Map[String, Buffer[String]]],
+ rpId: Int): Int = {
+ val resourceProfile = sc.resourceProfileManager.resourceProfileFromId(rpId)
+ val offersForResourceProfile = resourceProfileIds.zipWithIndex.filter { case (id, _) =>
+ (id == resourceProfile.id)
+ }
+ val coresKnown = resourceProfile.isCoresLimitKnown
+ var limitingResource = resourceProfile.limitingResource(sc.getConf)
+ val taskCpus = sc.resourceProfileManager.taskCpusForProfileId(rpId)
+
+ val availSlots = offersForResourceProfile.map { case (o, index) =>
+ val numTasksPerExecCores = availableCpus(index) / taskCpus
+ // when executor cores config isn't set, we can't calculate the real limiting resource
+ // and number of tasks per executor ahead of time, so calculate it now.
+ if (!coresKnown) {
+ val numTasksPerExecCustomResource = resourceProfile.maxTasksPerExecutor(sc.getConf)
+ if (limitingResource.isEmpty ||
+ (limitingResource.nonEmpty && numTasksPerExecCores < numTasksPerExecCustomResource)) {
+ limitingResource = ResourceProfile.CPUS
+ }
+ }
+
+ if (limitingResource == ResourceProfile.CPUS) {
+ numTasksPerExecCores
+ } else {
+ val taskLimit = resourceProfile.taskResources.get(limitingResource).map(_.amount).getOrElse(
+ throw new SparkException("limitingResource returns from ResourceProfile" +
+ s" $resourceProfile doesn't actually contain that task resource!")
+ )
+ // available addresses already takes into account if there are fractional
+ // task resource requests
+ val availableAddrs = availableResources(index).get(limitingResource)
+ .map(_.size).getOrElse(0)
+ (availableAddrs / taskLimit).toInt
+ }
+ }.sum
+ availSlots
}
/**
@@ -427,9 +510,12 @@ private[spark] class TaskSchedulerImpl(
val shuffledOffers = shuffleOffers(filteredOffers)
// Build a list of tasks to assign to each worker.
+ // Note the size estimate here might be off with different ResourceProfiles but should be
+ // close estimate
val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores / CPUS_PER_TASK))
val availableResources = shuffledOffers.map(_.resources).toArray
val availableCpus = shuffledOffers.map(o => o.cores).toArray
+ val resourceProfileIds = shuffledOffers.map(o => o.resourceProfileId).toArray
val sortedTaskSets = rootPool.getSortedTaskSetQueue
for (taskSet <- sortedTaskSets) {
logDebug("parentName: %s, name: %s, runningTasks: %s".format(
@@ -439,19 +525,27 @@ private[spark] class TaskSchedulerImpl(
}
}
- // Take each TaskSet in our scheduling order, and then offer it each node in increasing order
+ // Take each TaskSet in our scheduling order, and then offer it to each node in increasing order
// of locality levels so that it gets a chance to launch local tasks on all of them.
// NOTE: the preferredLocality order: PROCESS_LOCAL, NODE_LOCAL, NO_PREF, RACK_LOCAL, ANY
for (taskSet <- sortedTaskSets) {
- val availableSlots = availableCpus.map(c => c / CPUS_PER_TASK).sum
+ // we only need to calculate available slots if using barrier scheduling, otherwise the
+ // value is -1
+ val numBarrierSlotsAvailable = if (taskSet.isBarrier) {
+ val slots = calculateAvailableSlots(resourceProfileIds, availableCpus, availableResources,
+ taskSet.taskSet.resourceProfileId)
+ slots
+ } else {
+ -1
+ }
// Skip the barrier taskSet if the available slots are less than the number of pending tasks.
- if (taskSet.isBarrier && availableSlots < taskSet.numTasks) {
- // Skip the launch process.
- // TODO SPARK-24819 If the job requires more slots than available (both busy and free
- // slots), fail the job on submit.
- logInfo(s"Skip current round of resource offers for barrier stage ${taskSet.stageId} " +
- s"because the barrier taskSet requires ${taskSet.numTasks} slots, while the total " +
- s"number of available slots is $availableSlots.")
+ if (taskSet.isBarrier && numBarrierSlotsAvailable < taskSet.numTasks) {
+ // Skip the launch process.
+ // TODO SPARK-24819 If the job requires more slots than available (both busy and free
+ // slots), fail the job on submit.
+ logInfo(s"Skip current round of resource offers for barrier stage ${taskSet.stageId} " +
+ s"because the barrier taskSet requires ${taskSet.numTasks} slots, while the total " +
+ s"number of available slots is $numBarrierSlotsAvailable.")
} else {
var launchedAnyTask = false
// Record all the executor IDs assigned barrier tasks on.
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala
index 517c8991aed78..7a8ed16f6eb29 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala
@@ -28,7 +28,8 @@ private[spark] class TaskSet(
val stageId: Int,
val stageAttemptId: Int,
val priority: Int,
- val properties: Properties) {
+ val properties: Properties,
+ val resourceProfileId: Int) {
val id: String = stageId + "." + stageAttemptId
override def toString: String = "TaskSet " + id
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index 3b620ec69a9ab..01edf5291ee96 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -83,17 +83,21 @@ private[spark] class TaskSetManager(
val minFinishedForSpeculation = math.max((speculationQuantile * numTasks).floor.toInt, 1)
// User provided threshold for speculation regardless of whether the quantile has been reached
val speculationTaskDurationThresOpt = conf.get(SPECULATION_TASK_DURATION_THRESHOLD)
- // SPARK-29976: Only when the total number of tasks in the stage is less than or equal to the
- // number of slots on a single executor, would the task manager speculative run the tasks if
- // their duration is longer than the given threshold. In this way, we wouldn't speculate too
// aggressively but still handle basic cases.
// SPARK-30417: #cores per executor might not be set in spark conf for standalone mode, then
// the value of the conf would 1 by default. However, the executor would use all the cores on
// the worker. Therefore, CPUS_PER_TASK is okay to be greater than 1 without setting #cores.
- // To handle this case, we assume the minimum number of slots is 1.
// TODO: use the actual number of slots for standalone mode.
- val speculationTasksLessEqToSlots =
- numTasks <= Math.max(conf.get(EXECUTOR_CORES) / sched.CPUS_PER_TASK, 1)
+ val speculationTasksLessEqToSlots = {
+ val rpId = taskSet.resourceProfileId
+ val resourceProfile = sched.sc.resourceProfileManager.resourceProfileFromId(rpId)
+ val slots = if (!resourceProfile.isCoresLimitKnown) {
+ 1
+ } else {
+ resourceProfile.maxTasksPerExecutor(conf)
+ }
+ numTasks <= slots
+ }
// For each task, tracks whether a copy of the task has succeeded. A task will also be
// marked as "succeeded" if it failed with a fetch failure, in which case it should not
@@ -392,7 +396,8 @@ private[spark] class TaskSetManager(
execId: String,
host: String,
maxLocality: TaskLocality.TaskLocality,
- availableResources: Map[String, Seq[String]] = Map.empty)
+ taskCpus: Int,
+ taskResourceAssignments: Map[String, ResourceInformation] = Map.empty)
: Option[TaskDescription] =
{
val offerBlacklisted = taskSetBlacklistHelperOpt.exists { blacklist =>
@@ -455,18 +460,8 @@ private[spark] class TaskSetManager(
// val timeTaken = clock.getTime() - startTime
val taskName = s"task ${info.id} in stage ${taskSet.id}"
logInfo(s"Starting $taskName (TID $taskId, $host, executor ${info.executorId}, " +
- s"partition ${task.partitionId}, $taskLocality, ${serializedTask.limit()} bytes)")
-
- val extraResources = sched.resourcesReqsPerTask.map { taskReq =>
- val rName = taskReq.resourceName
- val count = taskReq.amount
- val rAddresses = availableResources.getOrElse(rName, Seq.empty)
- assert(rAddresses.size >= count, s"Required $count $rName addresses, but only " +
- s"${rAddresses.size} available.")
- // We'll drop the allocated addresses later inside TaskSchedulerImpl.
- val allocatedAddresses = rAddresses.take(count)
- (rName, new ResourceInformation(rName, allocatedAddresses.toArray))
- }.toMap
+ s"partition ${task.partitionId}, $taskLocality, ${serializedTask.limit()} bytes) " +
+ s"cpus: $taskCpus, taskResourceAssignments ${taskResourceAssignments}")
sched.dagScheduler.taskStarted(task, info)
new TaskDescription(
@@ -476,10 +471,11 @@ private[spark] class TaskSetManager(
taskName,
index,
task.partitionId,
+ taskCpus,
addedFiles,
addedJars,
task.localProperties,
- extraResources,
+ taskResourceAssignments,
serializedTask)
}
} else {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala
index 522dbfa9457b9..92a12f13576c1 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala
@@ -19,6 +19,8 @@ package org.apache.spark.scheduler
import scala.collection.mutable.Buffer
+import org.apache.spark.resource.ResourceProfile
+
/**
* Represents free resources available on an executor.
*/
@@ -30,4 +32,5 @@ case class WorkerOffer(
// `address` is an optional hostPort string, it provide more useful information than `host`
// when multiple executors are launched on the same host.
address: Option[String] = None,
- resources: Map[String, Buffer[String]] = Map.empty)
+ resources: Map[String, Buffer[String]] = Map.empty,
+ resourceProfileId: Int = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
index 283390814a6c0..f954942c2fbbf 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
@@ -71,14 +71,15 @@ private[spark] object CoarseGrainedClusterMessages {
taskId: Long,
state: TaskState,
data: SerializableBuffer,
+ taskCpus: Int,
resources: Map[String, ResourceInformation] = Map.empty)
extends CoarseGrainedClusterMessage
object StatusUpdate {
/** Alternate factory method that takes a ByteBuffer directly for the data field */
- def apply(executorId: String, taskId: Long, state: TaskState, data: ByteBuffer,
+ def apply(executorId: String, taskId: Long, state: TaskState, data: ByteBuffer, taskCpus: Int,
resources: Map[String, ResourceInformation]): StatusUpdate = {
- StatusUpdate(executorId, taskId, state, new SerializableBuffer(data), resources)
+ StatusUpdate(executorId, taskId, state, new SerializableBuffer(data), taskCpus, resources)
}
}
@@ -115,10 +116,10 @@ private[spark] object CoarseGrainedClusterMessages {
// Request executors by specifying the new total number of executors desired
// This includes executors already pending or running
case class RequestExecutors(
- requestedTotal: Int,
- localityAwareTasks: Int,
- hostToLocalTaskCount: Map[String, Int],
- nodeBlacklist: Set[String])
+ numLocalityAwareTasksPerResourceProfileId: Map[Int, Int],
+ hostToLocalTaskCount: Map[Int, Map[String, Int]],
+ nodeBlacklist: Set[String],
+ resourceProfileToTotalExecs: Map[ResourceProfile, Int])
extends CoarseGrainedClusterMessage
// Check if an executor was force-killed but for a reason unrelated to the running tasks.
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index 55f4005ef1b45..bce6becc6bd98 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -69,13 +69,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
conf.get(SCHEDULER_MAX_REGISTERED_RESOURCE_WAITING_TIME))
private val createTimeNs = System.nanoTime()
- private val taskResourceNumParts: Map[String, Int] =
- if (scheduler.resourcesReqsPerTask != null) {
- scheduler.resourcesReqsPerTask.map(req => req.resourceName -> req.numParts).toMap
- } else {
- Map.empty
- }
-
// Accessing `executorDataMap` in the inherited methods from ThreadSafeRpcEndpoint doesn't need
// any protection. But accessing `executorDataMap` out of the inherited methods must be
// protected by `CoarseGrainedSchedulerBackend.this`. Besides, `executorDataMap` should only
@@ -83,13 +76,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
// `CoarseGrainedSchedulerBackend.this`.
private val executorDataMap = new HashMap[String, ExecutorData]
- // Number of executors requested by the cluster manager, [[ExecutorAllocationManager]]
+ // Number of executors for each ResourceProfile requested by the cluster
+ // manager, [[ExecutorAllocationManager]]
@GuardedBy("CoarseGrainedSchedulerBackend.this")
- private var requestedTotalExecutors = 0
-
- // Number of executors requested from the cluster manager that have not registered yet
- @GuardedBy("CoarseGrainedSchedulerBackend.this")
- private var numPendingExecutors = 0
+ private var requestedTotalExecutorsPerResourceProfile = new HashMap[ResourceProfile, Int]
private val listenerBus = scheduler.sc.listenerBus
@@ -102,13 +92,14 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
// Executors that have been lost, but for which we don't yet know the real exit reason.
private val executorsPendingLossReason = new HashSet[String]
+ // A map of ResourceProfile id to map of hostname with its possible task number running on it
// A map to store hostname with its possible task number running on it
@GuardedBy("CoarseGrainedSchedulerBackend.this")
- protected var hostToLocalTaskCount: Map[String, Int] = Map.empty
+ protected var rpHostToLocalTaskCount: Map[Int, Map[String, Int]] = Map.empty
- // The number of pending tasks which is locality required
+ // The number of pending tasks per ResourceProfile id which is locality required
@GuardedBy("CoarseGrainedSchedulerBackend.this")
- protected var localityAwareTasks = 0
+ protected var numLocalityAwareTasksPerResourceProfileId = Map.empty[Int, Int]
// The num of current max ExecutorId used to re-register appMaster
@volatile protected var currentExecutorIdCounter = 0
@@ -147,12 +138,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
}
override def receive: PartialFunction[Any, Unit] = {
- case StatusUpdate(executorId, taskId, state, data, resources) =>
+ case StatusUpdate(executorId, taskId, state, data, taskCpus, resources) =>
scheduler.statusUpdate(taskId, state, data.value)
if (TaskState.isFinished(state)) {
executorDataMap.get(executorId) match {
case Some(executorInfo) =>
- executorInfo.freeCores += scheduler.CPUS_PER_TASK
+ executorInfo.freeCores += taskCpus
resources.foreach { case (k, v) =>
executorInfo.resourcesInfo.get(k).foreach { r =>
r.release(v.addresses)
@@ -223,16 +214,18 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
} else {
context.senderAddress
}
- logInfo(s"Registered executor $executorRef ($executorAddress) with ID $executorId")
+ logInfo(s"Registered executor $executorRef ($executorAddress) with ID $executorId, " +
+ s" ResourceProfileId $resourceProfileId")
addressToExecutorId(executorAddress) = executorId
totalCoreCount.addAndGet(cores)
totalRegisteredExecutors.addAndGet(1)
- val resourcesInfo = resources.map{ case (k, v) =>
- (v.name,
- new ExecutorResourceInfo(v.name, v.addresses,
- // tell the executor it can schedule resources up to numParts times,
- // as configured by the user, or set to 1 as that is the default (1 task/resource)
- taskResourceNumParts.getOrElse(v.name, 1)))
+ val resourcesInfo = resources.map { case (rName, info) =>
+ // tell the executor it can schedule resources up to numParts times,
+ // as configured by the user, or set to 1 as that is the default (1 task/resource)
+ val numParts = scheduler.sc.resourceProfileManager
+ .resourceProfileFromId(resourceProfileId).getNumSlotsPerAddress(rName, conf)
+ (info.name,
+ new ExecutorResourceInfo(info.name, info.addresses, numParts))
}
val data = new ExecutorData(executorRef, executorAddress, hostname,
0, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes,
@@ -244,10 +237,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
if (currentExecutorIdCounter < executorId.toInt) {
currentExecutorIdCounter = executorId.toInt
}
- if (numPendingExecutors > 0) {
- numPendingExecutors -= 1
- logDebug(s"Decremented number of pending executors ($numPendingExecutors left)")
- }
}
// Note: some tests expect the reply to come after we put the executor in the map
context.reply(true)
@@ -271,10 +260,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
context.reply(true)
case RetrieveSparkAppConfig(resourceProfileId) =>
- // note this will be updated in later prs to get the ResourceProfile from a
- // ResourceProfileManager that matches the resource profile id
- // for now just use default profile
- val rp = ResourceProfile.getOrCreateDefaultProfile(conf)
+ val rp = scheduler.sc.resourceProfileManager.resourceProfileFromId(resourceProfileId)
val reply = SparkAppConfig(
sparkProperties,
SparkEnv.get.securityManager.getIOEncryptionKey(),
@@ -295,7 +281,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
Some(executorData.executorAddress.hostPort),
executorData.resourcesInfo.map { case (rName, rInfo) =>
(rName, rInfo.availableAddrs.toBuffer)
- })
+ }, executorData.resourceProfileId)
}.toIndexedSeq
scheduler.resourceOffers(workOffers)
}
@@ -324,7 +310,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
Some(executorData.executorAddress.hostPort),
executorData.resourcesInfo.map { case (rName, rInfo) =>
(rName, rInfo.availableAddrs.toBuffer)
- }))
+ }, executorData.resourceProfileId))
scheduler.resourceOffers(workOffers)
} else {
Seq.empty
@@ -356,7 +342,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
val executorData = executorDataMap(task.executorId)
// Do resources allocation here. The allocated resources will get released after the task
// finishes.
- executorData.freeCores -= scheduler.CPUS_PER_TASK
+ executorData.freeCores -= task.cpus
task.resources.foreach { case (rName, rInfo) =>
assert(executorData.resourcesInfo.contains(rName))
executorData.resourcesInfo(rName).acquire(rInfo.addresses)
@@ -494,8 +480,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
* */
protected[scheduler] def reset(): Unit = {
val executors: Set[String] = synchronized {
- requestedTotalExecutors = 0
- numPendingExecutors = 0
+ requestedTotalExecutorsPerResourceProfile.clear()
executorDataMap.keys.toSet
}
@@ -553,6 +538,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
*/
private def numExistingExecutors: Int = synchronized { executorDataMap.size }
+ private def numExistingExecutorsForRpId(id: Int): Int = synchronized {
+ executorDataMap.filter { case (exec, info) => info.resourceProfileId == id }.size
+ }
+
override def getExecutorIds(): Seq[String] = synchronized {
executorDataMap.keySet.toSeq
}
@@ -563,9 +552,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
!executorsPendingLossReason.contains(id)
}
- override def maxNumConcurrentTasks(): Int = synchronized {
- executorDataMap.values.map { executor =>
- executor.totalCores / scheduler.CPUS_PER_TASK
+ override def maxNumConcurrentTasks(rp: ResourceProfile): Int = synchronized {
+ val cpusPerTask = rp.getTaskCpus.getOrElse(scheduler.CPUS_PER_TASK)
+ val executorsWithResourceProfile = executorDataMap.filter { case (e, data) =>
+ data.resourceProfileId == rp.id
+ }
+ executorsWithResourceProfile.values.map { executor =>
+ executor.totalCores / cpusPerTask
}.sum
}
@@ -582,7 +575,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
}
/**
- * Request an additional number of executors from the cluster manager.
+ * Request an additional number of executors from the cluster manager. This is
+ * requesting against the default ResourceProfile, we will need an API change to
+ * allow against other profiles.
* @return whether the request is acknowledged.
*/
final override def requestExecutors(numAdditionalExecutors: Int): Boolean = {
@@ -594,21 +589,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
logInfo(s"Requesting $numAdditionalExecutors additional executor(s) from the cluster manager")
val response = synchronized {
- requestedTotalExecutors += numAdditionalExecutors
- numPendingExecutors += numAdditionalExecutors
- logDebug(s"Number of pending executors is now $numPendingExecutors")
- if (requestedTotalExecutors !=
- (numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size)) {
- logDebug(
- s"""requestExecutors($numAdditionalExecutors): Executor request doesn't match:
- |requestedTotalExecutors = $requestedTotalExecutors
- |numExistingExecutors = $numExistingExecutors
- |numPendingExecutors = $numPendingExecutors
- |executorsPendingToRemove = ${executorsPendingToRemove.size}""".stripMargin)
- }
-
+ val defaultProf = scheduler.sc.resourceProfileManager.defaultResourceProfile
+ val numExisting = requestedTotalExecutorsPerResourceProfile.getOrElse(defaultProf, 0)
+ requestedTotalExecutorsPerResourceProfile(defaultProf) = numExisting + numAdditionalExecutors
// Account for executors pending to be added or removed
- doRequestTotalExecutors(requestedTotalExecutors)
+ doRequestTotalExecutors(requestedTotalExecutorsPerResourceProfile.toMap)
}
defaultAskTimeout.awaitResult(response)
@@ -617,39 +602,43 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
/**
* Update the cluster manager on our scheduling needs. Three bits of information are included
* to help it make decisions.
- * @param numExecutors The total number of executors we'd like to have. The cluster manager
- * shouldn't kill any running executor to reach this number, but,
- * if all existing executors were to die, this is the number of executors
- * we'd want to be allocated.
- * @param localityAwareTasks The number of tasks in all active stages that have a locality
- * preferences. This includes running, pending, and completed tasks.
+ *
+ * @param numLocalityAwareTasksPerResourceProfileId The number of tasks in all active stages that
+ * have a locality preferences per
+ * ResourceProfile. This includes running,
+ * pending, and completed tasks.
* @param hostToLocalTaskCount A map of hosts to the number of tasks from all active stages
* that would like to like to run on that host.
* This includes running, pending, and completed tasks.
+ * @param resourceProfileToNumExecutors The total number of executors we'd like to have per
+ * ResourceProfile. The cluster manager shouldn't kill any
+ * running executor to reach this number, but, if all
+ * existing executors were to die, this is the number
+ * of executors we'd want to be allocated.
+ *
* @return whether the request is acknowledged by the cluster manager.
*/
final override def requestTotalExecutors(
- numExecutors: Int,
- localityAwareTasks: Int,
- hostToLocalTaskCount: Map[String, Int]
+ numLocalityAwareTasksPerResourceProfileId: Map[Int, Int],
+ hostToLocalTaskCount: Map[Int, Map[String, Int]],
+ resourceProfileIdToNumExecutors: Map[Int, Int]
): Boolean = {
- if (numExecutors < 0) {
+ val totalExecs = resourceProfileIdToNumExecutors.values.sum
+ if (totalExecs < 0) {
throw new IllegalArgumentException(
"Attempted to request a negative number of executor(s) " +
- s"$numExecutors from the cluster manager. Please specify a positive number!")
+ s"$totalExecs from the cluster manager. Please specify a positive number!")
+ }
+ val resourceProfileToNumExecutors = resourceProfileIdToNumExecutors.map { case (rpid, num) =>
+ (scheduler.sc.resourceProfileManager.resourceProfileFromId(rpid), num)
}
-
val response = synchronized {
- this.requestedTotalExecutors = numExecutors
- this.localityAwareTasks = localityAwareTasks
- this.hostToLocalTaskCount = hostToLocalTaskCount
-
- numPendingExecutors =
- math.max(numExecutors - numExistingExecutors + executorsPendingToRemove.size, 0)
-
- doRequestTotalExecutors(numExecutors)
+ this.requestedTotalExecutorsPerResourceProfile =
+ new HashMap[ResourceProfile, Int] ++= resourceProfileToNumExecutors
+ this.numLocalityAwareTasksPerResourceProfileId = numLocalityAwareTasksPerResourceProfileId
+ this.rpHostToLocalTaskCount = hostToLocalTaskCount
+ doRequestTotalExecutors(requestedTotalExecutorsPerResourceProfile.toMap)
}
-
defaultAskTimeout.awaitResult(response)
}
@@ -665,7 +654,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
*
* @return a future whose evaluation indicates whether the request is acknowledged.
*/
- protected def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] =
+ protected def doRequestTotalExecutors(
+ resourceProfileToTotalExecs: Map[ResourceProfile, Int]): Future[Boolean] =
Future.successful(false)
/**
@@ -706,20 +696,20 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
// take into account executors that are pending to be added or removed.
val adjustTotalExecutors =
if (adjustTargetNumExecutors) {
- requestedTotalExecutors = math.max(requestedTotalExecutors - executorsToKill.size, 0)
- if (requestedTotalExecutors !=
- (numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size)) {
- logDebug(
- s"""killExecutors($executorIds, $adjustTargetNumExecutors, $countFailures, $force):
- |Executor counts do not match:
- |requestedTotalExecutors = $requestedTotalExecutors
- |numExistingExecutors = $numExistingExecutors
- |numPendingExecutors = $numPendingExecutors
- |executorsPendingToRemove = ${executorsPendingToRemove.size}""".stripMargin)
+ executorsToKill.foreach { exec =>
+ val rpId = executorDataMap(exec).resourceProfileId
+ val rp = scheduler.sc.resourceProfileManager.resourceProfileFromId(rpId)
+ if (requestedTotalExecutorsPerResourceProfile.isEmpty) {
+ // Assume that we are killing an executor that was started by default and
+ // not through the request api
+ requestedTotalExecutorsPerResourceProfile(rp) = 0
+ } else {
+ val requestedTotalForRp = requestedTotalExecutorsPerResourceProfile(rp)
+ requestedTotalExecutorsPerResourceProfile(rp) = math.max(requestedTotalForRp - 1, 0)
+ }
}
- doRequestTotalExecutors(requestedTotalExecutors)
+ doRequestTotalExecutors(requestedTotalExecutorsPerResourceProfile.toMap)
} else {
- numPendingExecutors += executorsToKill.size
Future.successful(true)
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
index a9b607d8cc38c..6cd97ff03eb5f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
@@ -28,7 +28,7 @@ import org.apache.spark.deploy.client.{StandaloneAppClient, StandaloneAppClientL
import org.apache.spark.internal.{config, Logging}
import org.apache.spark.internal.config.Tests.IS_TESTING
import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle}
-import org.apache.spark.resource.ResourceUtils
+import org.apache.spark.resource.{ResourceProfile, ResourceUtils}
import org.apache.spark.rpc.RpcEndpointAddress
import org.apache.spark.scheduler._
import org.apache.spark.util.Utils
@@ -194,9 +194,14 @@ private[spark] class StandaloneSchedulerBackend(
*
* @return whether the request is acknowledged.
*/
- protected override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = {
+ protected override def doRequestTotalExecutors(
+ resourceProfileToTotalExecs: Map[ResourceProfile, Int]): Future[Boolean] = {
+ // resources profiles not supported
Option(client) match {
- case Some(c) => c.requestTotalExecutors(requestedTotal)
+ case Some(c) =>
+ val defaultProf = sc.resourceProfileManager.defaultResourceProfile
+ val numExecs = resourceProfileToTotalExecs.getOrElse(defaultProf, 0)
+ c.requestTotalExecutors(numExecs)
case None =>
logWarning("Attempted to request executors before driver fully initialized.")
Future.successful(false)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala
index d2c0dc88d987e..55200f64d16ea 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala
@@ -26,6 +26,7 @@ import org.apache.spark.TaskState.TaskState
import org.apache.spark.executor.{Executor, ExecutorBackend}
import org.apache.spark.internal.{config, Logging}
import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle}
+import org.apache.spark.resource.{ResourceProfile, ResourceProfileBuilder}
import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint}
import org.apache.spark.scheduler._
import org.apache.spark.scheduler.cluster.ExecutorInfo
@@ -159,7 +160,13 @@ private[spark] class LocalSchedulerBackend(
override def applicationId(): String = appId
- override def maxNumConcurrentTasks(): Int = totalCores / scheduler.CPUS_PER_TASK
+ // Doesn't support different ResourceProfiles yet
+ // so we expect all executors to be of same ResourceProfile
+ override def maxNumConcurrentTasks(rp: ResourceProfile): Int = {
+ val cpusPerTask = rp.taskResources.get(ResourceProfile.CPUS)
+ .map(_.amount.toInt).getOrElse(scheduler.CPUS_PER_TASK)
+ totalCores / cpusPerTask
+ }
private def stop(finalState: SparkAppHandle.State): Unit = {
localEndpoint.ask(StopExecutor)
diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
index a5850fc2ac4b9..d0c20b53a8367 100644
--- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
@@ -28,6 +28,7 @@ import org.apache.spark.executor.TaskMetrics
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config.CPUS_PER_TASK
import org.apache.spark.internal.config.Status._
+import org.apache.spark.resource.ResourceProfile.CPUS
import org.apache.spark.scheduler._
import org.apache.spark.status.api.v1
import org.apache.spark.storage._
@@ -51,7 +52,7 @@ private[spark] class AppStatusListener(
private var sparkVersion = SPARK_VERSION
private var appInfo: v1.ApplicationInfo = null
private var appSummary = new AppSummary(0, 0)
- private var coresPerTask: Int = 1
+ private var defaultCoresPerTask: Int = 1
// How often to update live entities. -1 means "never update" when replaying applications,
// meaning only the last write will happen. For live applications, this avoids a few
@@ -76,6 +77,7 @@ private[spark] class AppStatusListener(
private val liveTasks = new HashMap[Long, LiveTask]()
private val liveRDDs = new HashMap[Int, LiveRDD]()
private val pools = new HashMap[String, SchedulerPool]()
+ private val liveResourceProfiles = new HashMap[Int, LiveResourceProfile]()
private val SQL_EXECUTION_ID_KEY = "spark.sql.execution.id"
// Keep the active executor count as a separate variable to avoid having to do synchronization
@@ -145,6 +147,23 @@ private[spark] class AppStatusListener(
}
}
+ override def onResourceProfileAdded(event: SparkListenerResourceProfileAdded): Unit = {
+ val liveRP = new LiveResourceProfile(event.resourceProfile.id)
+ liveResourceProfiles(event.resourceProfile.id) = liveRP
+ liveRP.taskResources = event.resourceProfile.taskResources
+ liveRP.executorResources = event.resourceProfile.executorResources
+ val maxTasks = event.resourceProfile.maxTasksPerExecutor(conf)
+ liveRP.maxTasksPerExecutor = if (event.resourceProfile.isCoresLimitKnown) {
+ Some(maxTasks)
+ } else {
+ None
+ }
+ val rpInfo = new v1.ResourceProfileInfo(liveRP.resourceProfileId,
+ liveRP.executorResources, liveRP.taskResources)
+ logWarning("Resource Profile added id " + liveRP.resourceProfileId)
+ kvstore.write(new ResourceProfileWrapper(rpInfo))
+ }
+
override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Unit = {
val details = event.environmentDetails
@@ -161,8 +180,8 @@ private[spark] class AppStatusListener(
details.getOrElse("System Properties", Nil),
details.getOrElse("Classpath Entries", Nil))
- coresPerTask = envInfo.sparkProperties.toMap.get(CPUS_PER_TASK.key).map(_.toInt)
- .getOrElse(coresPerTask)
+ defaultCoresPerTask = envInfo.sparkProperties.toMap.get(CPUS_PER_TASK.key).map(_.toInt)
+ .getOrElse(defaultCoresPerTask)
kvstore.write(new ApplicationEnvironmentInfoWrapper(envInfo))
}
@@ -197,10 +216,16 @@ private[spark] class AppStatusListener(
exec.host = event.executorInfo.executorHost
exec.isActive = true
exec.totalCores = event.executorInfo.totalCores
- exec.maxTasks = event.executorInfo.totalCores / coresPerTask
+ val rpId = event.executorInfo.resourceProfileId
+ val liveRP = liveResourceProfiles.get(rpId)
+ val cpusPerTask = liveRP.map(_.taskResources.get(CPUS)
+ .map(_.amount.toInt).getOrElse(defaultCoresPerTask)).getOrElse(defaultCoresPerTask)
+ val maxTasksPerExec = liveRP.flatMap(_.maxTasksPerExecutor)
+ exec.maxTasks = maxTasksPerExec.getOrElse(event.executorInfo.totalCores / cpusPerTask)
exec.executorLogs = event.executorInfo.logUrlMap
exec.resources = event.executorInfo.resourcesInfo
exec.attributes = event.executorInfo.attributes
+ exec.resourceProfileId = rpId
liveUpdate(exec, System.nanoTime())
}
diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
index 6b89812cc2bf0..d18681f0326eb 100644
--- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
@@ -23,6 +23,7 @@ import scala.collection.JavaConverters._
import scala.collection.mutable.HashMap
import org.apache.spark.{JobExecutionStatus, SparkConf}
+import org.apache.spark.resource.ResourceProfileManager
import org.apache.spark.status.api.v1
import org.apache.spark.ui.scope._
import org.apache.spark.util.Utils
@@ -44,6 +45,13 @@ private[spark] class AppStatusStore(
store.read(klass, klass.getName()).info
}
+ def resourceProfileInfo(): Seq[v1.ResourceProfileInfo] = {
+ val klass = classOf[ResourceProfileWrapper]
+ // store.read(klass, 0).rpInfo
+ val it = store.view(classOf[ResourceProfileWrapper]).asScala.map(_.rpInfo)
+ it.toSeq
+ }
+
def jobsList(statuses: JList[JobExecutionStatus]): Seq[v1.JobData] = {
val it = store.view(classOf[JobDataWrapper]).reverse().asScala.map(_.info)
if (statuses != null && !statuses.isEmpty()) {
@@ -479,7 +487,8 @@ private[spark] class AppStatusStore(
accumulatorUpdates = stage.accumulatorUpdates,
tasks = Some(tasks),
executorSummary = Some(executorSummary(stage.stageId, stage.attemptId)),
- killedTasksSummary = stage.killedTasksSummary)
+ killedTasksSummary = stage.killedTasksSummary,
+ resourceProfileId = stage.resourceProfileId)
}
def rdd(rddId: Int): v1.RDDStorageInfo = {
diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
index e3046dce34e67..5db9602f300ab 100644
--- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
+++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
@@ -28,7 +28,7 @@ import com.google.common.collect.Interners
import org.apache.spark.JobExecutionStatus
import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics}
-import org.apache.spark.resource.ResourceInformation
+import org.apache.spark.resource.{ExecutorResourceRequest, ResourceInformation, ResourceProfile, TaskResourceRequest}
import org.apache.spark.scheduler.{AccumulableInfo, StageInfo, TaskInfo}
import org.apache.spark.status.api.v1
import org.apache.spark.storage.{RDDInfo, StorageLevel}
@@ -245,6 +245,22 @@ private class LiveTask(
}
+private class LiveResourceProfile(val resourceProfileId: Int) extends LiveEntity {
+
+ var executorResources = Map[String, ExecutorResourceRequest]()
+ var taskResources = Map[String, TaskResourceRequest]()
+ var maxTasksPerExecutor: Option[Int] = None
+
+ def toApi(): v1.ResourceProfileInfo = {
+ new v1.ResourceProfileInfo(resourceProfileId, executorResources, taskResources)
+ }
+
+ override protected def doUpdate(): Any = {
+ new ResourceProfileWrapper(toApi())
+ }
+}
+
+
private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveEntity {
var hostPort: String = null
@@ -285,6 +301,8 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE
var usedOnHeap = 0L
var usedOffHeap = 0L
+ var resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID
+
def hasMemoryInfo: Boolean = totalOnHeap >= 0L
// peak values for executor level metrics
@@ -327,7 +345,8 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE
blacklistedInStages,
Some(peakExecutorMetrics).filter(_.isSet),
attributes,
- resources)
+ resources,
+ resourceProfileId)
new ExecutorSummaryWrapper(info)
}
}
@@ -465,7 +484,8 @@ private class LiveStage extends LiveEntity {
accumulatorUpdates = newAccumulatorInfos(info.accumulables.values),
tasks = None,
executorSummary = None,
- killedTasksSummary = killedSummary)
+ killedTasksSummary = killedSummary,
+ resourceProfileId = info.resourceProfileId)
}
override protected def doUpdate(): Any = {
@@ -721,6 +741,7 @@ private[spark] object LiveEntityHelpers {
shuffleWriteTime,
shuffleRecordsWritten))
}
+
// scalastyle:on argcount
def createMetrics(default: Long): v1.TaskMetrics = {
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala
index cf5c759bebdbb..7d365f126a4f7 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala
@@ -99,14 +99,17 @@ private[v1] class AbstractApplicationResource extends BaseAppResource {
@GET
@Path("environment")
- def environmentInfo(): ApplicationEnvironmentInfo = withUI { ui =>
+ def environmentInfo(): ApplicationEnvironmentWithResourceProfileInfo = withUI { ui =>
val envInfo = ui.store.environmentInfo()
- new v1.ApplicationEnvironmentInfo(
+ // TODO - should we make separate endpoint for ResourceProfiles?
+ val resourceProfileInfo = ui.store.resourceProfileInfo()
+ new v1.ApplicationEnvironmentWithResourceProfileInfo(
envInfo.runtime,
Utils.redact(ui.conf, envInfo.sparkProperties),
Utils.redact(ui.conf, envInfo.hadoopProperties),
Utils.redact(ui.conf, envInfo.systemProperties),
- envInfo.classpathEntries)
+ envInfo.classpathEntries,
+ resourceProfileInfo)
}
@GET
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
index 5ec9b36393764..ef061222a42ef 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
@@ -30,7 +30,7 @@ import com.fasterxml.jackson.databind.annotation.{JsonDeserialize, JsonSerialize
import org.apache.spark.JobExecutionStatus
import org.apache.spark.executor.ExecutorMetrics
import org.apache.spark.metrics.ExecutorMetricType
-import org.apache.spark.resource.ResourceInformation
+import org.apache.spark.resource.{ExecutorResourceRequest, ResourceInformation, TaskResourceRequest}
case class ApplicationInfo private[spark](
id: String,
@@ -62,6 +62,11 @@ case class ApplicationAttemptInfo private[spark](
}
+class ResourceProfileInfo private[spark](
+ val id: Int,
+ val executorResources: Map[String, ExecutorResourceRequest],
+ val taskResources: Map[String, TaskResourceRequest])
+
class ExecutorStageSummary private[spark](
val taskTime : Long,
val failedTasks : Int,
@@ -109,7 +114,8 @@ class ExecutorSummary private[spark](
@JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer])
val peakMemoryMetrics: Option[ExecutorMetrics],
val attributes: Map[String, String],
- val resources: Map[String, ResourceInformation])
+ val resources: Map[String, ResourceInformation],
+ val resourceProfileId: Int)
class MemoryMetrics private[spark](
val usedOnHeapStorageMemory: Long,
@@ -252,7 +258,8 @@ class StageData private[spark](
val accumulatorUpdates: Seq[AccumulableInfo],
val tasks: Option[Map[Long, TaskData]],
val executorSummary: Option[Map[String, ExecutorStageSummary]],
- val killedTasksSummary: Map[String, Int])
+ val killedTasksSummary: Map[String, Int],
+ val resourceProfileId: Int)
class TaskData private[spark](
val taskId: Long,
@@ -372,6 +379,14 @@ class ApplicationEnvironmentInfo private[spark] (
val systemProperties: Seq[(String, String)],
val classpathEntries: Seq[(String, String)])
+class ApplicationEnvironmentWithResourceProfileInfo private[spark] (
+ val runtime: RuntimeInfo,
+ val sparkProperties: Seq[(String, String)],
+ val hadoopProperties: Seq[(String, String)],
+ val systemProperties: Seq[(String, String)],
+ val classpathEntries: Seq[(String, String)],
+ val resourceProfiles: Seq[ResourceProfileInfo])
+
class RuntimeInfo private[spark](
val javaVersion: String,
val javaHome: String,
diff --git a/core/src/main/scala/org/apache/spark/status/storeTypes.scala b/core/src/main/scala/org/apache/spark/status/storeTypes.scala
index f0a94d84d8a04..6b0179e5cc47c 100644
--- a/core/src/main/scala/org/apache/spark/status/storeTypes.scala
+++ b/core/src/main/scala/org/apache/spark/status/storeTypes.scala
@@ -374,6 +374,13 @@ private[spark] class RDDStorageInfoWrapper(val info: RDDStorageInfo) {
}
+private[spark] class ResourceProfileWrapper(val rpInfo: ResourceProfileInfo) {
+
+ @JsonIgnore @KVIndex
+ def id: Int = rpInfo.id
+
+}
+
private[spark] class ExecutorStageSummaryWrapper(
val stageId: Int,
val stageAttemptId: Int,
diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala
index c6eb461ad601c..2b6141e3f3b3f 100644
--- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala
@@ -19,9 +19,11 @@ package org.apache.spark.ui.env
import javax.servlet.http.HttpServletRequest
+import scala.collection.mutable
import scala.xml.Node
import org.apache.spark.SparkConf
+import org.apache.spark.resource.{ExecutorResourceRequest, TaskResourceRequest}
import org.apache.spark.status.AppStatusStore
import org.apache.spark.ui._
import org.apache.spark.util.Utils
@@ -38,6 +40,34 @@ private[ui] class EnvironmentPage(
"Java Home" -> appEnv.runtime.javaHome,
"Scala Version" -> appEnv.runtime.scalaVersion)
+ def constructExecutorRequestString(ereqs: Map[String, ExecutorResourceRequest]): String = {
+ ereqs.map {
+ case (_, ereq) =>
+ val execStr = new mutable.StringBuilder()
+ execStr ++= s"\t${ereq.resourceName}: [amount: ${ereq.amount}"
+ if (ereq.discoveryScript.nonEmpty) execStr ++= s", discovery: ${ereq.discoveryScript}"
+ if (ereq.vendor.nonEmpty) execStr ++= s", vendor: ${ereq.vendor}"
+ execStr ++= "]"
+ execStr.toString()
+ }.mkString("\n")
+ }
+
+ def constructTaskRequestString(treqs: Map[String, TaskResourceRequest]): String = {
+ treqs.map {
+ case (_, ereq) => s"\t${ereq.resourceName}: [amount: ${ereq.amount}]"
+ }.mkString("\n")
+ }
+
+ val resourceProfileInfo = store.resourceProfileInfo().map { rinfo =>
+ val einfo = constructExecutorRequestString(rinfo.executorResources)
+ val tinfo = constructTaskRequestString(rinfo.taskResources)
+ val res = s"Executor Reqs:\n$einfo\nTask Reqs:\n$tinfo"
+ (rinfo.id.toString, res)
+ }.toMap
+
+ val resourceProfileInformationTable = UIUtils.listingTable(
+ resourceProfileHeader, jvmRowDataPre, resourceProfileInfo.toSeq.sorted, fixedWidth = true,
+ headerClasses = headerClassesNoSortValues)
val runtimeInformationTable = UIUtils.listingTable(
propertyHeader, jvmRow, jvmInformation.toSeq.sorted, fixedWidth = true,
headerClasses = headerClasses)
@@ -55,6 +85,17 @@ private[ui] class EnvironmentPage(
headerClasses = headerClasses)
val content =
+
+