diff --git a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java index 731f6fc767dfd..579e7ff320f5c 100644 --- a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java +++ b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java @@ -162,6 +162,11 @@ public void onSpeculativeTaskSubmitted(SparkListenerSpeculativeTaskSubmitted spe onEvent(speculativeTask); } + @Override + public void onResourceProfileAdded(SparkListenerResourceProfileAdded event) { + onEvent(event); + } + @Override public void onOtherEvent(SparkListenerEvent event) { onEvent(event); diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html index e91449013e371..c7a6369b35224 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html @@ -90,6 +90,7 @@

Executors

Disk Used Cores Resources + Resource Profile ID Active Tasks Failed Tasks Complete Tasks diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index 090bc72dc9274..60f8fc8a73481 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -119,7 +119,7 @@ function totalDurationColor(totalGCTime, totalDuration) { } var sumOptionalColumns = [3, 4]; -var execOptionalColumns = [5, 6, 9]; +var execOptionalColumns = [5, 6, 9, 10]; var execDataTable; var sumDataTable; @@ -415,6 +415,7 @@ $(document).ready(function () { {data: 'diskUsed', render: formatBytes}, {data: 'totalCores'}, {name: 'resourcesCol', data: 'resources', render: formatResourceCells, orderable: false}, + {name: 'resourceProfileIdCol', data: 'resourceProfileId'}, { data: 'activeTasks', "fnCreatedCell": function (nTd, sData, oData, iRow, iCol) { @@ -461,7 +462,8 @@ $(document).ready(function () { "columnDefs": [ {"visible": false, "targets": 5}, {"visible": false, "targets": 6}, - {"visible": false, "targets": 9} + {"visible": false, "targets": 9}, + {"visible": false, "targets": 10} ], "deferRender": true }; @@ -570,6 +572,7 @@ $(document).ready(function () { "
On Heap Memory
" + "
Off Heap Memory
" + "
Resources
" + + "
Resource Prodile Id
" + ""); 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 = + +

+ + Resource Profiles +

+
+
+ {resourceProfileInformationTable} +
@@ -115,10 +156,14 @@ private[ui] class EnvironmentPage( UIUtils.headerSparkPage(request, "Environment", content, parent) } + private def resourceProfileHeader = Seq("Resource Profile ID", "Resource Profile Contents") private def propertyHeader = Seq("Name", "Value") private def classPathHeader = Seq("Resource", "Source") private def headerClasses = Seq("sorttable_alpha", "sorttable_alpha") + private def headerClassesNoSortValues = Seq("sorttable_alpha", "sorttable_nosort") + private def jvmRowDataPre(kv: (String, String)) = + {kv._1}
{kv._2}
private def jvmRow(kv: (String, String)) = {kv._1}{kv._2} private def propertyRow(kv: (String, String)) = {kv._1}{kv._2} private def classPathRow(data: (String, String)) = {data._1}{data._2} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 12f1aa25e8d2a..0b58c84721f4c 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -26,6 +26,7 @@ import scala.xml.{Node, NodeSeq, Unparsed, Utility} import org.apache.commons.text.StringEscapeUtils import org.apache.spark.JobExecutionStatus +import org.apache.spark.resource.ResourceProfile import org.apache.spark.status.AppStatusStore import org.apache.spark.status.api.v1 import org.apache.spark.ui._ @@ -252,7 +253,8 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP accumulatorUpdates = Nil, tasks = None, executorSummary = None, - killedTasksSummary = Map()) + killedTasksSummary = Map(), + ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 4dc5349cb8ad3..4eba89909f26b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -142,6 +142,10 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We val summary =
    +
  • + Resource Profile ID: + {stageData.resourceProfileId} +
  • Total Time Across All Tasks: {UIUtils.formatDuration(stageData.executorRunTime)} diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 4d89c4f079f29..6e5f90478f0ca 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -21,6 +21,7 @@ import java.util.{Properties, UUID} import scala.collection.JavaConverters._ import scala.collection.Map +import scala.util.control.NonFatal import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule @@ -33,7 +34,7 @@ import org.apache.spark._ import org.apache.spark.executor._ import org.apache.spark.metrics.ExecutorMetricType import org.apache.spark.rdd.RDDOperationScope -import org.apache.spark.resource.ResourceInformation +import org.apache.spark.resource.{ExecutorResourceRequest, ResourceInformation, ResourceProfile, TaskResourceRequest} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage._ @@ -104,6 +105,8 @@ private[spark] object JsonProtocol { stageExecutorMetricsToJson(stageExecutorMetrics) case blockUpdate: SparkListenerBlockUpdated => blockUpdateToJson(blockUpdate) + case resourceProfileAdded: SparkListenerResourceProfileAdded => + resourceProfileAddedToJson(resourceProfileAdded) case _ => parse(mapper.writeValueAsString(event)) } } @@ -223,6 +226,15 @@ private[spark] object JsonProtocol { ("Timestamp" -> applicationEnd.time) } + def resourceProfileAddedToJson(profileAdded: SparkListenerResourceProfileAdded): JValue = { + ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.resourceProfileAdded) ~ + ("Resource Profile Id" -> profileAdded.resourceProfile.id) ~ + ("Executor Resource Requests" -> + executorResourceRequestMapToJson(profileAdded.resourceProfile.executorResources)) ~ + ("Task Resource Requests" -> + taskResourceRequestMapToJson(profileAdded.resourceProfile.taskResources)) + } + def executorAddedToJson(executorAdded: SparkListenerExecutorAdded): JValue = { ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.executorAdded) ~ ("Timestamp" -> executorAdded.time) ~ @@ -296,7 +308,8 @@ private[spark] object JsonProtocol { ("Submission Time" -> submissionTime) ~ ("Completion Time" -> completionTime) ~ ("Failure Reason" -> failureReason) ~ - ("Accumulables" -> accumulablesToJson(stageInfo.accumulables.values)) + ("Accumulables" -> accumulablesToJson(stageInfo.accumulables.values)) ~ + ("Resource Profile Id" -> stageInfo.resourceProfileId) } def taskInfoToJson(taskInfo: TaskInfo): JValue = { @@ -499,7 +512,8 @@ private[spark] object JsonProtocol { ("Total Cores" -> executorInfo.totalCores) ~ ("Log Urls" -> mapToJson(executorInfo.logUrlMap)) ~ ("Attributes" -> mapToJson(executorInfo.attributes)) ~ - ("Resources" -> resourcesMapToJson(executorInfo.resourcesInfo)) + ("Resources" -> resourcesMapToJson(executorInfo.resourcesInfo)) ~ + ("Resource Profile Id" -> executorInfo.resourceProfileId) } def resourcesMapToJson(m: Map[String, ResourceInformation]): JValue = { @@ -509,6 +523,36 @@ private[spark] object JsonProtocol { JObject(jsonFields.toList) } + def executorResourceRequestToJson(execReq: ExecutorResourceRequest): JValue = { + ("Resource Name" -> execReq.resourceName) ~ + ("Amount" -> execReq.amount) ~ + ("Discovery Script" -> execReq.discoveryScript) ~ + ("Vendor" -> execReq.vendor) + } + + def executorResourceRequestMapToJson(m: Map[String, ExecutorResourceRequest]): JValue = { + val jsonFields = m.map { + case (k, execReq) => + JField(k, executorResourceRequestToJson(execReq)) + } + + JObject(jsonFields.toList) + } + + def taskResourceRequestToJson(taskReq: TaskResourceRequest): JValue = { + ("Resource Name" -> taskReq.resourceName) ~ + ("Amount" -> taskReq.amount) + } + + def taskResourceRequestMapToJson(m: Map[String, TaskResourceRequest]): JValue = { + val jsonFields = m.map { + case (k, taskReq) => + JField(k, taskResourceRequestToJson(taskReq)) + } + + JObject(jsonFields.toList) + } + def blockUpdatedInfoToJson(blockUpdatedInfo: BlockUpdatedInfo): JValue = { ("Block Manager ID" -> blockManagerIdToJson(blockUpdatedInfo.blockManagerId)) ~ ("Block ID" -> blockUpdatedInfo.blockId.toString) ~ @@ -576,6 +620,7 @@ private[spark] object JsonProtocol { val metricsUpdate = Utils.getFormattedClassName(SparkListenerExecutorMetricsUpdate) val stageExecutorMetrics = Utils.getFormattedClassName(SparkListenerStageExecutorMetrics) val blockUpdate = Utils.getFormattedClassName(SparkListenerBlockUpdated) + val resourceProfileAdded = Utils.getFormattedClassName(SparkListenerResourceProfileAdded) } def sparkEventFromJson(json: JValue): SparkListenerEvent = { @@ -601,6 +646,7 @@ private[spark] object JsonProtocol { case `metricsUpdate` => executorMetricsUpdateFromJson(json) case `stageExecutorMetrics` => stageExecutorMetricsFromJson(json) case `blockUpdate` => blockUpdateFromJson(json) + case `resourceProfileAdded` => resourceProfileAddedFromJson(json) case other => mapper.readValue(compact(render(json)), Utils.classForName(other)) .asInstanceOf[SparkListenerEvent] } @@ -662,7 +708,8 @@ private[spark] object JsonProtocol { val stageInfos = jsonOption(json \ "Stage Infos") .map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse { stageIds.map { id => - new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown") + new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) } } SparkListenerJobStart(jobId, submissionTime, stageInfos, properties) @@ -676,6 +723,45 @@ private[spark] object JsonProtocol { SparkListenerJobEnd(jobId, completionTime, jobResult) } + def resourceProfileAddedFromJson(json: JValue): SparkListenerResourceProfileAdded = { + val profId = (json \ "Resource Profile Id").extract[Int] + val executorReqs = executorResourceRequestMapFromJson(json \ "Executor Resource Requests") + val taskReqs = taskResourceRequestMapFromJson(json \ "Task Resource Requests") + val rp = new ResourceProfile(executorReqs.toMap, taskReqs.toMap) + rp.setResourceProfileId(profId) + SparkListenerResourceProfileAdded(rp) + } + + def executorResourceRequestFromJson(json: JValue): ExecutorResourceRequest = { + val rName = (json \ "Resource Name").extract[String] + val amount = (json \ "Amount").extract[Int] + val discoveryScript = (json \ "Discovery Script").extract[String] + val vendor = (json \ "Vendor").extract[String] + new ExecutorResourceRequest(rName, amount, discoveryScript, vendor) + } + + def taskResourceRequestFromJson(json: JValue): TaskResourceRequest = { + val rName = (json \ "Resource Name").extract[String] + val amount = (json \ "Amount").extract[Int] + new TaskResourceRequest(rName, amount) + } + + def taskResourceRequestMapFromJson(json: JValue): Map[String, TaskResourceRequest] = { + val jsonFields = json.asInstanceOf[JObject].obj + jsonFields.map { case JField(k, v) => + val req = taskResourceRequestFromJson(v) + (k, req) + }.toMap + } + + def executorResourceRequestMapFromJson(json: JValue): Map[String, ExecutorResourceRequest] = { + val jsonFields = json.asInstanceOf[JObject].obj + jsonFields.map { case JField(k, v) => + val req = executorResourceRequestFromJson(v) + (k, req) + }.toMap + } + def environmentUpdateFromJson(json: JValue): SparkListenerEnvironmentUpdate = { // For compatible with previous event logs val hadoopProperties = jsonOption(json \ "Hadoop Properties").map(mapFromJson(_).toSeq) @@ -801,9 +887,10 @@ private[spark] object JsonProtocol { case None => Seq.empty[AccumulableInfo] } } - - val stageInfo = new StageInfo( - stageId, attemptId, stageName, numTasks, rddInfos, parentIds, details) + val rpId = jsonOption(json \ "Resource Profile Id").map(_.extract[Int]) + val stageProf = rpId.getOrElse(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stageInfo = new StageInfo(stageId, attemptId, stageName, numTasks, rddInfos, + parentIds, details, resourceProfileId = stageProf) stageInfo.submissionTime = submissionTime stageInfo.completionTime = completionTime stageInfo.failureReason = failureReason @@ -1106,7 +1193,11 @@ private[spark] object JsonProtocol { case Some(resources) => resourcesMapFromJson(resources).toMap case None => Map.empty[String, ResourceInformation] } - new ExecutorInfo(executorHost, totalCores, logUrls, attributes, resources) + val resourceProfileId = jsonOption(json \ "Resource Profile Id") match { + case Some(id) => id.extract[Int] + case None => 0 + } + new ExecutorInfo(executorHost, totalCores, logUrls, attributes, resources, resourceProfileId) } def blockUpdatedInfoFromJson(json: JValue): BlockUpdatedInfo = { diff --git a/core/src/test/java/org/apache/spark/resource/JavaResourceProfileSuite.java b/core/src/test/java/org/apache/spark/resource/JavaResourceProfileSuite.java index bb413c00fb972..5e566bf29dca5 100644 --- a/core/src/test/java/org/apache/spark/resource/JavaResourceProfileSuite.java +++ b/core/src/test/java/org/apache/spark/resource/JavaResourceProfileSuite.java @@ -25,8 +25,8 @@ // Test the ResourceProfile and Request api's from Java public class JavaResourceProfileSuite { - String GpuResource = "resource.gpu"; - String FPGAResource = "resource.fpga"; + String GpuResource = "gpu"; + String FPGAResource = "fpga"; @Test public void testResourceProfileAccessFromJava() throws Exception { diff --git a/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json b/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json index a64617256d63a..0b617a7d0aced 100644 --- a/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json @@ -282,5 +282,6 @@ [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-jobclient-2.2.0.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sketch_2.11-2.1.0-SNAPSHOT.jar", "System Classpath" ], [ "/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-model-1.2.15.jar", "System Classpath" ] - ] + ], + "resourceProfiles" : [ ] } diff --git a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json index 6e6d28b6a57ec..d2b3d1b069204 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_1578436911597_0052", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-01-11T17:44:22.851GMT", + "endTime" : "2020-01-11T17:46:42.615GMT", + "lastUpdated" : "", + "duration" : 139764, + "sparkUser" : "tgraves", + "completed" : true, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "endTimeEpoch" : 1578764802615, + "startTimeEpoch" : 1578764662851, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1555004656427_0144", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json b/core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json index b18b19f7eeffb..0d197eab0e25d 100644 --- a/core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json @@ -717,5 +717,6 @@ "isBlacklistedForStage" : false } }, - "killedTasksSummary" : { } + "killedTasksSummary" : { }, + "resourceProfileId" : 0 } diff --git a/core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json b/core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json index 8d11081247913..24d73faa45021 100644 --- a/core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json @@ -876,5 +876,6 @@ "isBlacklistedForStage" : true } }, - "killedTasksSummary" : { } + "killedTasksSummary" : { }, + "resourceProfileId" : 0 } diff --git a/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json index a47cd26ed102b..a452488294547 100644 --- a/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json @@ -41,7 +41,8 @@ "schedulingPool" : "default", "rddIds" : [ 6, 5 ], "accumulatorUpdates" : [ ], - "killedTasksSummary" : { } + "killedTasksSummary" : { }, + "resourceProfileId" : 0 }, { "status" : "COMPLETE", "stageId" : 1, @@ -85,7 +86,8 @@ "schedulingPool" : "default", "rddIds" : [ 1, 0 ], "accumulatorUpdates" : [ ], - "killedTasksSummary" : { } + "killedTasksSummary" : { }, + "resourceProfileId" : 0 }, { "status" : "COMPLETE", "stageId" : 0, @@ -129,5 +131,6 @@ "schedulingPool" : "default", "rddIds" : [ 0 ], "accumulatorUpdates" : [ ], - "killedTasksSummary" : { } + "killedTasksSummary" : { }, + "resourceProfileId" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json index 6e6d28b6a57ec..d2b3d1b069204 100644 --- a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_1578436911597_0052", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-01-11T17:44:22.851GMT", + "endTime" : "2020-01-11T17:46:42.615GMT", + "lastUpdated" : "", + "duration" : 139764, + "sparkUser" : "tgraves", + "completed" : true, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "endTimeEpoch" : 1578764802615, + "startTimeEpoch" : 1578764662851, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1555004656427_0144", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json index eadf27164c814..67425676a62d6 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json @@ -22,5 +22,6 @@ "executorLogs" : { }, "blacklistedInStages" : [ ], "attributes" : { }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json index d322485baa8de..d052a27385f66 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json @@ -50,7 +50,8 @@ "MajorGCTime" : 144 }, "attributes" : { }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 }, { "id" : "3", "hostPort" : "test-3.vpc.company.com:37641", @@ -116,7 +117,8 @@ "NM_HOST" : "test-3.vpc.company.com", "CONTAINER_ID" : "container_1553914137147_0018_01_000004" }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 }, { "id" : "2", "hostPort" : "test-4.vpc.company.com:33179", @@ -182,7 +184,8 @@ "NM_HOST" : "test-4.vpc.company.com", "CONTAINER_ID" : "container_1553914137147_0018_01_000003" }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 }, { "id" : "1", "hostPort" : "test-2.vpc.company.com:43764", @@ -248,5 +251,6 @@ "NM_HOST" : "test-2.vpc.company.com", "CONTAINER_ID" : "container_1553914137147_0018_01_000002" }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json index 7c3f77d8c10cf..91574ca8266b2 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json @@ -28,7 +28,8 @@ }, "blacklistedInStages" : [ ], "attributes" : { }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 }, { "id" : "3", "hostPort" : "172.22.0.167:51485", @@ -62,7 +63,8 @@ }, "blacklistedInStages" : [ ], "attributes" : { }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 } ,{ "id" : "2", "hostPort" : "172.22.0.167:51487", @@ -96,7 +98,8 @@ }, "blacklistedInStages" : [ ], "attributes" : { }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 }, { "id" : "1", "hostPort" : "172.22.0.167:51490", @@ -130,7 +133,8 @@ }, "blacklistedInStages" : [ ], "attributes" : { }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 }, { "id" : "0", "hostPort" : "172.22.0.167:51491", @@ -164,5 +168,6 @@ }, "blacklistedInStages" : [ ], "attributes" : { }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json index 0986e85f16b3e..f14b9a5085a42 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json @@ -28,7 +28,8 @@ }, "blacklistedInStages" : [ ], "attributes" : { }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 }, { "id" : "3", "hostPort" : "172.22.0.167:51485", @@ -62,7 +63,8 @@ }, "blacklistedInStages" : [ ], "attributes" : { }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 }, { "id" : "2", "hostPort" : "172.22.0.167:51487", @@ -96,7 +98,8 @@ }, "blacklistedInStages" : [ ], "attributes" : { }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 }, { "id" : "1", "hostPort" : "172.22.0.167:51490", @@ -130,7 +133,8 @@ }, "blacklistedInStages" : [ ], "attributes" : { }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 }, { "id" : "0", "hostPort" : "172.22.0.167:51491", @@ -164,5 +168,6 @@ }, "blacklistedInStages" : [ ], "attributes" : { }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json index 26d665151a52d..3645387317ca1 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json @@ -22,7 +22,8 @@ "executorLogs" : { }, "blacklistedInStages" : [ ], "attributes" : { }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 }, { "id" : "3", "hostPort" : "172.22.0.111:64543", @@ -50,7 +51,8 @@ }, "blacklistedInStages" : [ ], "attributes" : { }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 }, { "id" : "2", "hostPort" : "172.22.0.111:64539", @@ -78,7 +80,8 @@ }, "blacklistedInStages" : [ ], "attributes" : { }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 }, { "id" : "1", "hostPort" : "172.22.0.111:64541", @@ -106,7 +109,8 @@ }, "blacklistedInStages" : [ ], "attributes" : { }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 }, { "id" : "0", "hostPort" : "172.22.0.111:64540", @@ -134,5 +138,6 @@ }, "blacklistedInStages" : [ ], "attributes" : { }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_resource_information_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_resource_information_expectation.json index e69ab3b49d455..7f990f4e6196a 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_resource_information_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_resource_information_expectation.json @@ -28,7 +28,8 @@ }, "blacklistedInStages" : [ ], "attributes" : { }, - "resources" : { } + "resources" : { }, + "resourceProfileId" : 0 }, { "id" : "2", "hostPort" : "tomg-test:46005", @@ -77,7 +78,8 @@ "name" : "gpu", "addresses" : [ "0", "1", "2", "3", "4", "5", "6", "7", "8", "9", "10", "11", "12" ] } - } + }, + "resourceProfileId" : 0 }, { "id" : "1", "hostPort" : "tomg-test:44873", @@ -126,5 +128,6 @@ "name" : "gpu", "addresses" : [ "0", "1", "2", "3", "4", "5", "6", "7", "8", "9", "10", "11", "12" ] } - } + }, + "resourceProfileId" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json index da26271e66bc4..c38741646c64b 100644 --- a/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json @@ -42,5 +42,6 @@ "schedulingPool" : "default", "rddIds" : [ 3, 2 ], "accumulatorUpdates" : [ ], - "killedTasksSummary" : { } + "killedTasksSummary" : { }, + "resourceProfileId" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json index 3102909f81116..82489e94a84c8 100644 --- a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_1578436911597_0052", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-01-11T17:44:22.851GMT", + "endTime" : "2020-01-11T17:46:42.615GMT", + "lastUpdated" : "", + "duration" : 139764, + "sparkUser" : "tgraves", + "completed" : true, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "endTimeEpoch" : 1578764802615, + "startTimeEpoch" : 1578764662851, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1555004656427_0144", "name" : "Spark shell", "attempts" : [ { @@ -28,19 +43,4 @@ "endTimeEpoch" : 1554756046454, "lastUpdatedEpoch" : 0 } ] -}, { - "id" : "application_1516285256255_0012", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2018-01-18T18:30:35.119GMT", - "endTime" : "2018-01-18T18:38:27.938GMT", - "lastUpdated" : "", - "duration" : 472819, - "sparkUser" : "attilapiros", - "completed" : true, - "appSparkVersion" : "2.3.0-SNAPSHOT", - "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1516300235119, - "endTimeEpoch" : 1516300707938 - } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json index 794f1514a6708..ac2bb0e29b2fb 100644 --- a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json @@ -1,5 +1,19 @@ -[ - { +[ { + "id" : "application_1578436911597_0052", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-01-11T17:44:22.851GMT", + "endTime" : "2020-01-11T17:46:42.615GMT", + "lastUpdated" : "", + "duration" : 139764, + "sparkUser" : "tgraves", + "completed" : true, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "endTimeEpoch" : 1578764802615, + "startTimeEpoch" : 1578764662851, + "lastUpdatedEpoch" : 0 + } ] +}, { "id": "application_1555004656427_0144", "name": "Spark shell", "attempts": [ diff --git a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json index adcdccef48450..156167606ff20 100644 --- a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json @@ -1,4 +1,19 @@ [ { + "id" : "application_1578436911597_0052", + "name" : "Spark shell", + "attempts" : [ { + "startTime" : "2020-01-11T17:44:22.851GMT", + "endTime" : "2020-01-11T17:46:42.615GMT", + "lastUpdated" : "", + "duration" : 139764, + "sparkUser" : "tgraves", + "completed" : true, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "endTimeEpoch" : 1578764802615, + "startTimeEpoch" : 1578764662851, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1555004656427_0144", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/multiple_resource_profiles_expectation.json b/core/src/test/resources/HistoryServerExpectations/multiple_resource_profiles_expectation.json new file mode 100644 index 0000000000000..5c1e4cc2337be --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/multiple_resource_profiles_expectation.json @@ -0,0 +1,112 @@ +{ + "runtime" : { + "javaVersion" : "1.8.0_232 (Private Build)", + "javaHome" : "/usr/lib/jvm/java-8-openjdk-amd64/jre", + "scalaVersion" : "version 2.12.10" + }, + "sparkProperties" : [ ], + "hadoopProperties" : [ ], + "systemProperties" : [ ], + "classpathEntries" : [ ], + "resourceProfiles" : [ { + "id" : 0, + "executorResources" : { + "cores" : { + "resourceName" : "cores", + "amount" : 1, + "discoveryScript" : "", + "vendor" : "" + }, + "memory" : { + "resourceName" : "memory", + "amount" : 1024, + "discoveryScript" : "", + "vendor" : "" + }, + "gpu" : { + "resourceName" : "gpu", + "amount" : 1, + "discoveryScript" : "/home/tgraves/getGpus", + "vendor" : "" + } + }, + "taskResources" : { + "cpus" : { + "resourceName" : "cpus", + "amount" : 1.0 + }, + "gpu" : { + "resourceName" : "gpu", + "amount" : 1.0 + } + } + }, { + "id" : 1, + "executorResources" : { + "cores" : { + "resourceName" : "cores", + "amount" : 4, + "discoveryScript" : "", + "vendor" : "" + }, + "gpu" : { + "resourceName" : "gpu", + "amount" : 1, + "discoveryScript" : "./getGpus", + "vendor" : "" + } + }, + "taskResources" : { + "cpus" : { + "resourceName" : "cpus", + "amount" : 1.0 + }, + "gpu" : { + "resourceName" : "gpu", + "amount" : 1.0 + } + } + }, { + "id" : 2, + "executorResources" : { + "cores" : { + "resourceName" : "cores", + "amount" : 2, + "discoveryScript" : "", + "vendor" : "" + } + }, + "taskResources" : { + "cpus" : { + "resourceName" : "cpus", + "amount" : 2.0 + } + } + }, { + "id" : 3, + "executorResources" : { + "cores" : { + "resourceName" : "cores", + "amount" : 4, + "discoveryScript" : "", + "vendor" : "" + }, + "gpu" : { + "resourceName" : "gpu", + "amount" : 1, + "discoveryScript" : "./getGpus", + "vendor" : "" + } + }, + "taskResources" : { + "cpus" : { + "resourceName" : "cpus", + "amount" : 2.0 + }, + "gpu" : { + "resourceName" : "gpu", + "amount" : 1.0 + } + } + } ] +} diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json index 791907045e500..3db7d551b6130 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json @@ -462,5 +462,6 @@ "isBlacklistedForStage" : false } }, - "killedTasksSummary" : { } + "killedTasksSummary" : { }, + "resourceProfileId" : 0 } diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json index 50d3f74ae775f..8ef3769c1ca6b 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json @@ -462,5 +462,6 @@ "isBlacklistedForStage" : false } }, - "killedTasksSummary" : { } + "killedTasksSummary" : { }, + "resourceProfileId" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json index edbac7127039d..a31c907221388 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json @@ -41,7 +41,8 @@ "schedulingPool" : "default", "rddIds" : [ 6, 5 ], "accumulatorUpdates" : [ ], - "killedTasksSummary" : { } + "killedTasksSummary" : { }, + "resourceProfileId" : 0 }, { "status" : "FAILED", "stageId" : 2, @@ -86,7 +87,8 @@ "schedulingPool" : "default", "rddIds" : [ 3, 2 ], "accumulatorUpdates" : [ ], - "killedTasksSummary" : { } + "killedTasksSummary" : { }, + "resourceProfileId" : 0 }, { "status" : "COMPLETE", "stageId" : 1, @@ -130,7 +132,8 @@ "schedulingPool" : "default", "rddIds" : [ 1, 0 ], "accumulatorUpdates" : [ ], - "killedTasksSummary" : { } + "killedTasksSummary" : { }, + "resourceProfileId" : 0 }, { "status" : "COMPLETE", "stageId" : 0, @@ -174,5 +177,6 @@ "schedulingPool" : "default", "rddIds" : [ 0 ], "accumulatorUpdates" : [ ], - "killedTasksSummary" : { } + "killedTasksSummary" : { }, + "resourceProfileId" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json index 836f2cb095097..08089d4f3f65b 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json @@ -45,5 +45,6 @@ "name" : "my counter", "value" : "5050" } ], - "killedTasksSummary" : { } + "killedTasksSummary" : { }, + "resourceProfileId" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json index 735a8257fc343..3b5476ae8b160 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json @@ -506,5 +506,6 @@ "isBlacklistedForStage" : false } }, - "killedTasksSummary" : { } + "killedTasksSummary" : { }, + "resourceProfileId" : 0 } diff --git a/core/src/test/resources/spark-events/application_1578436911597_0052 b/core/src/test/resources/spark-events/application_1578436911597_0052 new file mode 100644 index 0000000000000..c57481a348a89 --- /dev/null +++ b/core/src/test/resources/spark-events/application_1578436911597_0052 @@ -0,0 +1,27 @@ +{"Event":"SparkListenerLogStart","Spark Version":"3.0.0-SNAPSHOT"} +{"Event":"SparkListenerResourceProfileAdded","Resource Profile Id":0,"Executor Resource Requests":{"cores":{"Resource Name":"cores","Amount":1,"Discovery Script":"","Vendor":""},"memory":{"Resource Name":"memory","Amount":1024,"Discovery Script":"","Vendor":""},"gpu":{"Resource Name":"gpu","Amount":1,"Discovery Script":"/home/tgraves/getGpus","Vendor":""}},"Task Resource Requests":{"cpus":{"Resource Name":"cpus","Amount":1.0},"gpu":{"Resource Name":"gpu","Amount":1.0}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"10.10.10.10","Port":32957},"Maximum Memory":428762726,"Timestamp":1578764671818,"Maximum Onheap Memory":428762726,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-8-openjdk-amd64/jre","Java Version":"1.8.0_232 (Private Build)","Scala Version":"version 2.12.10"},"Spark Properties":{},"Hadoop Properties":{},"System Properties":{}, "Classpath Entries": {}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"application_1578436911597_0052","Timestamp":1578764662851,"User":"tgraves"} +{"Event":"SparkListenerResourceProfileAdded","Resource Profile Id":1,"Executor Resource Requests":{"cores":{"Resource Name":"cores","Amount":4,"Discovery Script":"","Vendor":""},"gpu":{"Resource Name":"gpu","Amount":1,"Discovery Script":"./getGpus","Vendor":""}},"Task Resource Requests":{"cpus":{"Resource Name":"cpus","Amount":1.0},"gpu":{"Resource Name":"gpu","Amount":1.0}}} +{"Event":"SparkListenerResourceProfileAdded","Resource Profile Id":2,"Executor Resource Requests":{"cores":{"Resource Name":"cores","Amount":2,"Discovery Script":"","Vendor":""}},"Task Resource Requests":{"cpus":{"Resource Name":"cpus","Amount":2.0}}} +{"Event":"SparkListenerResourceProfileAdded","Resource Profile Id":3,"Executor Resource Requests":{"cores":{"Resource Name":"cores","Amount":4,"Discovery Script":"","Vendor":""},"gpu":{"Resource Name":"gpu","Amount":1,"Discovery Script":"./getGpus","Vendor":""}},"Task Resource Requests":{"cpus":{"Resource Name":"cpus","Amount":2.0},"gpu":{"Resource Name":"gpu","Amount":1.0}}} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1578764765274,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"collect at :29","Number of Tasks":6,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at :31","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at :31","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:1004)\n$line37.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:29)\n$line37.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:33)\n$line37.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:35)\n$line37.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:37)\n$line37.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:39)\n$line37.$read$$iw$$iw$$iw$$iw$$iw.(:41)\n$line37.$read$$iw$$iw$$iw$$iw.(:43)\n$line37.$read$$iw$$iw$$iw.(:45)\n$line37.$read$$iw$$iw.(:47)\n$line37.$read$$iw.(:49)\n$line37.$read.(:51)\n$line37.$read$.(:55)\n$line37.$read$.()\n$line37.$eval$.$print$lzycompute(:7)\n$line37.$eval$.$print(:6)\n$line37.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)","Accumulables":[],"Resource Profile Id":3}],"Stage IDs":[0],"Properties":{"spark.rdd.scope":"{\"id\":\"2\",\"name\":\"collect\"}","spark.rdd.scope.noOverride":"true"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"collect at :29","Number of Tasks":6,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at :31","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at :31","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:1004)\n$line37.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:29)\n$line37.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:33)\n$line37.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:35)\n$line37.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:37)\n$line37.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:39)\n$line37.$read$$iw$$iw$$iw$$iw$$iw.(:41)\n$line37.$read$$iw$$iw$$iw$$iw.(:43)\n$line37.$read$$iw$$iw$$iw.(:45)\n$line37.$read$$iw$$iw.(:47)\n$line37.$read$$iw.(:49)\n$line37.$read.(:51)\n$line37.$read$.(:55)\n$line37.$read$.()\n$line37.$eval$.$print$lzycompute(:7)\n$line37.$eval$.$print(:6)\n$line37.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)","Submission Time":1578764765293,"Accumulables":[],"Resource Profile Id":3},"Properties":{"spark.rdd.scope":"{\"id\":\"2\",\"name\":\"collect\"}","spark.rdd.scope.noOverride":"true"}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1578764769706,"Executor ID":"1","Executor Info":{"Host":"host1","Total Cores":4,"Log Urls":{"stdout":"http://host1:8042/node/containerlogs/container_1578436911597_0052_01_000002/tgraves/stdout?start=-4096","stderr":"http://host1:8042/node/containerlogs/container_1578436911597_0052_01_000002/tgraves/stderr?start=-4096"},"Attributes":{"NM_HTTP_ADDRESS":"host1:8042","USER":"tgraves","LOG_FILES":"stderr,stdout","NM_HTTP_PORT":"8042","CLUSTER_ID":"","NM_PORT":"37783","HTTP_SCHEME":"http://","NM_HOST":"host1","CONTAINER_ID":"container_1578436911597_0052_01_000002"},"Resources":{"gpu":{"name":"gpu","addresses":["0","1","2"]}},"Resource Profile Id":3}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"host1","Port":40787},"Maximum Memory":384093388,"Timestamp":1578764769796,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1578764769858,"Executor ID":"1","Host":"host1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1578764769877,"Executor ID":"1","Host":"host1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1578764770507,"Executor ID":"1","Host":"host1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1578764770509,"Executor ID":"1","Host":"host1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1578764769858,"Executor ID":"1","Host":"host1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1578764770512,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":49,"Value":49,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":3706,"Value":3706,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":20740892,"Value":20740892,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":32,"Value":32,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":250921658,"Value":250921658,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":555,"Value":555,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":555,"Executor Deserialize CPU Time":250921658,"Executor Run Time":32,"Executor CPU Time":20740892,"Peak Execution Memory":0,"Result Size":3706,"JVM GC Time":49,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1578764769877,"Executor ID":"1","Host":"host1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1578764770515,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":49,"Value":98,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":3722,"Value":7428,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":25185125,"Value":45926017,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":32,"Value":64,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":416274503,"Value":667196161,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":555,"Value":1110,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":555,"Executor Deserialize CPU Time":416274503,"Executor Run Time":32,"Executor CPU Time":25185125,"Peak Execution Memory":0,"Result Size":3722,"JVM GC Time":49,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1578764770525,"Executor ID":"1","Host":"host1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1578764770507,"Executor ID":"1","Host":"host1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1578764770526,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":3636,"Value":11064,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":2203515,"Value":48129532,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":66,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2733237,"Value":669929398,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":1112,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2733237,"Executor Run Time":2,"Executor CPU Time":2203515,"Peak Execution Memory":0,"Result Size":3636,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1578764770527,"Executor ID":"1","Host":"host1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1578764770509,"Executor ID":"1","Host":"host1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1578764770529,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":3620,"Value":14684,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":2365599,"Value":50495131,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":68,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3387884,"Value":673317282,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":1115,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3387884,"Executor Run Time":2,"Executor CPU Time":2365599,"Peak Execution Memory":0,"Result Size":3620,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1578764770525,"Executor ID":"1","Host":"host1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1578764770542,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":3636,"Value":18320,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":2456346,"Value":52951477,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":70,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3502860,"Value":676820142,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":1118,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3502860,"Executor Run Time":2,"Executor CPU Time":2456346,"Peak Execution Memory":0,"Result Size":3636,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1578764770527,"Executor ID":"1","Host":"host1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1578764770542,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":3636,"Value":21956,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":2162370,"Value":55113847,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":2,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3622437,"Value":680442579,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":1121,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3622437,"Executor Run Time":2,"Executor CPU Time":2162370,"Peak Execution Memory":0,"Result Size":3636,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"collect at :29","Number of Tasks":6,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at :31","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at :31","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":6,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:1004)\n$line37.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:29)\n$line37.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:33)\n$line37.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:35)\n$line37.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:37)\n$line37.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:39)\n$line37.$read$$iw$$iw$$iw$$iw$$iw.(:41)\n$line37.$read$$iw$$iw$$iw$$iw.(:43)\n$line37.$read$$iw$$iw$$iw.(:45)\n$line37.$read$$iw$$iw.(:47)\n$line37.$read$$iw.(:49)\n$line37.$read.(:51)\n$line37.$read$.(:55)\n$line37.$read$.()\n$line37.$eval$.$print$lzycompute(:7)\n$line37.$eval$.$print(:6)\n$line37.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)","Submission Time":1578764765293,"Completion Time":1578764770543,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Value":72,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Value":98,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Value":21956,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Value":680442579,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Value":55113847,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Value":4,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Value":1121,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":3}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1578764770546,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1578764802615} diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 99f3e3b2e478d..065c17fde942f 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests.TEST_SCHEDULE_INTERVAL import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfile, ResourceProfileBuilder, ResourceProfileManager, TaskResourceRequests} import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -45,6 +46,9 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { private val managers = new mutable.ListBuffer[ExecutorAllocationManager]() private var listenerBus: LiveListenerBus = _ private var client: ExecutorAllocationClient = _ + private val clock = new SystemClock() + private var rpManager: ResourceProfileManager = _ + override def beforeEach(): Unit = { super.beforeEach() @@ -108,65 +112,174 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { test("starting state") { val manager = createManager(createConf()) - assert(numExecutorsTarget(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) assert(executorsPendingToRemove(manager).isEmpty) assert(addTime(manager) === ExecutorAllocationManager.NOT_SET) } - test("add executors") { + test("add executors default profile") { val manager = createManager(createConf(1, 10, 1)) post(SparkListenerStageSubmitted(createStageInfo(0, 1000))) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + // Keep adding until the limit is reached - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 2) - assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 4) - assert(numExecutorsToAdd(manager) === 4) - assert(addExecutors(manager) === 4) - assert(numExecutorsTarget(manager) === 8) - assert(numExecutorsToAdd(manager) === 8) - assert(addExecutors(manager) === 2) // reached the limit of 10 - assert(numExecutorsTarget(manager) === 10) - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 0) - assert(numExecutorsTarget(manager) === 10) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 4) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 4) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) + assert(numExecutorsToAddForDefaultProfile(manager) === 8) + // reached the limit of 10 + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) // Register previously requested executors - onExecutorAdded(manager, "first") - assert(numExecutorsTarget(manager) === 10) - onExecutorAdded(manager, "second") - onExecutorAdded(manager, "third") - onExecutorAdded(manager, "fourth") - assert(numExecutorsTarget(manager) === 10) - onExecutorAdded(manager, "first") // duplicates should not count - onExecutorAdded(manager, "second") - assert(numExecutorsTarget(manager) === 10) + onExecutorAddedDefaultProfile(manager, "first") + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + onExecutorAddedDefaultProfile(manager, "second") + onExecutorAddedDefaultProfile(manager, "third") + onExecutorAddedDefaultProfile(manager, "fourth") + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + onExecutorAddedDefaultProfile(manager, "first") // duplicates should not count + onExecutorAddedDefaultProfile(manager, "second") + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) // Try adding again // This should still fail because the number pending + running is still at the limit - assert(addExecutors(manager) === 0) - assert(numExecutorsTarget(manager) === 10) - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 0) - assert(numExecutorsTarget(manager) === 10) - assert(numExecutorsToAdd(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) } - def testAllocationRatio(cores: Int, divisor: Double, expected: Int): Unit = { + test("add executors multiple profiles") { + val manager = createManager(createConf(1, 10, 1)) + post(SparkListenerStageSubmitted(createStageInfo(0, 1000, rp = defaultProfile))) + val rp1 = new ResourceProfileBuilder() + val execReqs = new ExecutorResourceRequests().cores(4).resource("gpu", 4) + val taskReqs = new TaskResourceRequests().cpus(1).resource("gpu", 1) + rp1.require(execReqs).require(taskReqs) + + val rprof1 = rp1.build + rpManager.addResourceProfile(rprof1) + post(SparkListenerStageSubmitted(createStageInfo(1, 1000, rp = rprof1))) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + // Keep adding until the limit is reached + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + assert(numExecutorsToAdd(manager, rprof1) === 1) + assert(numExecutorsTarget(manager, rprof1.id) === 1) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + assert(numExecutorsToAdd(manager, rprof1) === 2) + assert(numExecutorsTarget(manager, rprof1.id) === 2) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 4) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 4) + assert(numExecutorsToAdd(manager, rprof1) === 4) + assert(numExecutorsTarget(manager, rprof1.id) === 4) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 4) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) + assert(numExecutorsToAddForDefaultProfile(manager) === 8) + // reached the limit of 10 + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + assert(numExecutorsToAdd(manager, rprof1) === 8) + assert(numExecutorsTarget(manager, rprof1.id) === 8) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + assert(numExecutorsToAdd(manager, rprof1) === 1) + assert(numExecutorsTarget(manager, rprof1.id) === 10) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(numExecutorsToAdd(manager, rprof1) === 1) + assert(numExecutorsTarget(manager, rprof1.id) === 10) + + // Register previously requested executors + onExecutorAddedDefaultProfile(manager, "first") + onExecutorAdded(manager, "firstrp1", rprof1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsTarget(manager, rprof1.id) === 10) + onExecutorAddedDefaultProfile(manager, "second") + onExecutorAddedDefaultProfile(manager, "third") + onExecutorAddedDefaultProfile(manager, "fourth") + onExecutorAdded(manager, "secondrp1", rprof1) + onExecutorAdded(manager, "thirdrp1", rprof1) + onExecutorAdded(manager, "fourthrp1", rprof1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsTarget(manager, rprof1.id) === 10) + onExecutorAddedDefaultProfile(manager, "first") // duplicates should not count + onExecutorAddedDefaultProfile(manager, "second") + onExecutorAdded(manager, "firstrp1", rprof1) + onExecutorAdded(manager, "secondrp1", rprof1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsTarget(manager, rprof1.id) === 10) + + // Try adding again + // This should still fail because the number pending + running is still at the limit + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(numExecutorsToAdd(manager, rprof1) === 1) + assert(numExecutorsTarget(manager, rprof1.id) === 10) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(numExecutorsToAdd(manager, rprof1) === 1) + assert(numExecutorsTarget(manager, rprof1.id) === 10) + } + + def testAllocationRatio(cores: Int, divisor: Double, expected: Int): Unit = { + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] val conf = createConf(3, 15) .set(config.DYN_ALLOCATION_EXECUTOR_ALLOCATION_RATIO, divisor) .set(config.EXECUTOR_CORES, cores) + ResourceProfile.reInitDefaultProfile(conf) val manager = createManager(conf) post(SparkListenerStageSubmitted(createStageInfo(0, 20))) for (i <- 0 to 5) { - addExecutors(manager) + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) } - assert(numExecutorsTarget(manager) === expected) + assert(numExecutorsTargetForDefaultProfileId(manager) === expected) } test("executionAllocationRatio is correctly handled") { @@ -185,83 +298,104 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val manager = createManager(createConf(0, 10, 0)) post(SparkListenerStageSubmitted(createStageInfo(0, 5))) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + // Verify that we're capped at number of tasks in the stage - assert(numExecutorsTarget(manager) === 0) - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 3) - assert(numExecutorsToAdd(manager) === 4) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 5) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) // Verify that running a task doesn't affect the target post(SparkListenerStageSubmitted(createStageInfo(1, 3))) post(SparkListenerExecutorAdded( 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) post(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) - assert(numExecutorsTarget(manager) === 5) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 6) - assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 8) - assert(numExecutorsToAdd(manager) === 4) - assert(addExecutors(manager) === 0) - assert(numExecutorsTarget(manager) === 8) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 6) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) // Verify that re-running a task doesn't blow things up post(SparkListenerStageSubmitted(createStageInfo(2, 3))) post(SparkListenerTaskStart(2, 0, createTaskInfo(0, 0, "executor-1"))) post(SparkListenerTaskStart(2, 0, createTaskInfo(1, 0, "executor-1"))) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 9) - assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 10) - assert(numExecutorsToAdd(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 9) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) // Verify that running a task once we're at our limit doesn't blow things up post(SparkListenerTaskStart(2, 0, createTaskInfo(0, 1, "executor-1"))) - assert(addExecutors(manager) === 0) - assert(numExecutorsTarget(manager) === 10) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) } test("add executors when speculative tasks added") { val manager = createManager(createConf(0, 10, 0)) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + post(SparkListenerStageSubmitted(createStageInfo(1, 2))) // Verify that we're capped at number of tasks including the speculative ones in the stage post(SparkListenerSpeculativeTaskSubmitted(1)) - assert(numExecutorsTarget(manager) === 0) - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) post(SparkListenerSpeculativeTaskSubmitted(1)) post(SparkListenerSpeculativeTaskSubmitted(1)) - post(SparkListenerStageSubmitted(createStageInfo(1, 2))) - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 3) - assert(numExecutorsToAdd(manager) === 4) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 5) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) // Verify that running a task doesn't affect the target post(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) - assert(numExecutorsTarget(manager) === 5) - assert(addExecutors(manager) === 0) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) // Verify that running a speculative task doesn't affect the target post(SparkListenerTaskStart(1, 0, createTaskInfo(1, 0, "executor-2", true))) - assert(numExecutorsTarget(manager) === 5) - assert(addExecutors(manager) === 0) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) } test("properly handle task end events from completed stages") { @@ -282,43 +416,49 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { post(SparkListenerStageCompleted(stage)) // There are still two tasks that belong to the zombie stage running. - assert(totalRunningTasks(manager) === 2) + assert(totalRunningTasksPerResourceProfile(manager) === 2) // submit another attempt for the stage. We count completions from the first zombie attempt val stageAttempt1 = createStageInfo(stage.stageId, 5, attemptId = 1) post(SparkListenerStageSubmitted(stageAttempt1)) post(SparkListenerTaskEnd(0, 0, null, Success, taskInfo1, new ExecutorMetrics, null)) - assert(totalRunningTasks(manager) === 1) + assert(totalRunningTasksPerResourceProfile(manager) === 1) val attemptTaskInfo1 = createTaskInfo(3, 0, "executor-1") val attemptTaskInfo2 = createTaskInfo(4, 1, "executor-1") post(SparkListenerTaskStart(0, 1, attemptTaskInfo1)) post(SparkListenerTaskStart(0, 1, attemptTaskInfo2)) - assert(totalRunningTasks(manager) === 3) + assert(totalRunningTasksPerResourceProfile(manager) === 3) post(SparkListenerTaskEnd(0, 1, null, Success, attemptTaskInfo1, new ExecutorMetrics, null)) - assert(totalRunningTasks(manager) === 2) + assert(totalRunningTasksPerResourceProfile(manager) === 2) post(SparkListenerTaskEnd(0, 0, null, Success, taskInfo2, new ExecutorMetrics, null)) - assert(totalRunningTasks(manager) === 1) + assert(totalRunningTasksPerResourceProfile(manager) === 1) post(SparkListenerTaskEnd(0, 1, null, Success, attemptTaskInfo2, new ExecutorMetrics, null)) - assert(totalRunningTasks(manager) === 0) + assert(totalRunningTasksPerResourceProfile(manager) === 0) } testRetry("cancel pending executors when no longer needed") { val manager = createManager(createConf(0, 10, 0)) post(SparkListenerStageSubmitted(createStageInfo(2, 5))) - assert(numExecutorsTarget(manager) === 0) - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 3) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) val task1Info = createTaskInfo(0, 0, "executor-1") post(SparkListenerTaskStart(2, 0, task1Info)) - assert(numExecutorsToAdd(manager) === 4) - assert(addExecutors(manager) === 2) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) val task2Info = createTaskInfo(1, 0, "executor-1") post(SparkListenerTaskStart(2, 0, task2Info)) @@ -334,7 +474,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { test("remove executors") { val manager = createManager(createConf(5, 10, 5)) - (1 to 10).map(_.toString).foreach { id => onExecutorAdded(manager, id) } + (1 to 10).map(_.toString).foreach { id => onExecutorAddedDefaultProfile(manager, id) } // Keep removing until the limit is reached assert(executorsPendingToRemove(manager).isEmpty) @@ -381,7 +521,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { test("remove multiple executors") { val manager = createManager(createConf(5, 10, 5)) - (1 to 10).map(_.toString).foreach { id => onExecutorAdded(manager, id) } + (1 to 10).map(_.toString).foreach { id => onExecutorAddedDefaultProfile(manager, id) } // Keep removing until the limit is reached assert(executorsPendingToRemove(manager).isEmpty) @@ -425,63 +565,76 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { assert(executorsPendingToRemove(manager).isEmpty) } - test ("Removing with various numExecutorsTarget condition") { + test ("Removing with various numExecutorsTargetForDefaultProfileId condition") { val manager = createManager(createConf(5, 12, 5)) post(SparkListenerStageSubmitted(createStageInfo(0, 8))) - // Remove when numExecutorsTarget is the same as the current number of executors - assert(addExecutors(manager) === 1) - assert(addExecutors(manager) === 2) - (1 to 8).foreach(execId => onExecutorAdded(manager, execId.toString)) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + // Remove when numExecutorsTargetForDefaultProfileId is the same as the current + // number of executors + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + (1 to 8).foreach(execId => onExecutorAddedDefaultProfile(manager, execId.toString)) (1 to 8).map { i => createTaskInfo(i, i, s"$i") }.foreach { info => post(SparkListenerTaskStart(0, 0, info)) } assert(manager.executorMonitor.executorCount === 8) - assert(numExecutorsTarget(manager) === 8) - assert(maxNumExecutorsNeeded(manager) == 8) - assert(!removeExecutor(manager, "1")) // won't work since numExecutorsTarget == numExecutors + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 8) + // won't work since numExecutorsTargetForDefaultProfileId == numExecutors + assert(!removeExecutor(manager, "1")) - // Remove executors when numExecutorsTarget is lower than current number of executors + // Remove executors when numExecutorsTargetForDefaultProfileId is lower than + // current number of executors (1 to 3).map { i => createTaskInfo(i, i, s"$i") }.foreach { info => post(SparkListenerTaskEnd(0, 0, null, Success, info, new ExecutorMetrics, null)) } adjustRequestedExecutors(manager) assert(manager.executorMonitor.executorCount === 8) - assert(numExecutorsTarget(manager) === 5) - assert(maxNumExecutorsNeeded(manager) == 5) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 5) assert(removeExecutor(manager, "1")) assert(removeExecutors(manager, Seq("2", "3"))=== Seq("2", "3")) onExecutorRemoved(manager, "1") onExecutorRemoved(manager, "2") onExecutorRemoved(manager, "3") - // numExecutorsTarget is lower than minNumExecutors + // numExecutorsTargetForDefaultProfileId is lower than minNumExecutors post(SparkListenerTaskEnd(0, 0, null, Success, createTaskInfo(4, 4, "4"), new ExecutorMetrics, null)) assert(manager.executorMonitor.executorCount === 5) - assert(numExecutorsTarget(manager) === 5) - assert(maxNumExecutorsNeeded(manager) == 4) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 4) assert(!removeExecutor(manager, "4")) // lower limit - assert(addExecutors(manager) === 0) // upper limit + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) // upper limit } test ("interleaving add and remove") { val manager = createManager(createConf(5, 12, 5)) post(SparkListenerStageSubmitted(createStageInfo(0, 1000))) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + // Add a few executors - assert(addExecutors(manager) === 1) - assert(addExecutors(manager) === 2) - onExecutorAdded(manager, "1") - onExecutorAdded(manager, "2") - onExecutorAdded(manager, "3") - onExecutorAdded(manager, "4") - onExecutorAdded(manager, "5") - onExecutorAdded(manager, "6") - onExecutorAdded(manager, "7") - onExecutorAdded(manager, "8") + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + onExecutorAddedDefaultProfile(manager, "1") + onExecutorAddedDefaultProfile(manager, "2") + onExecutorAddedDefaultProfile(manager, "3") + onExecutorAddedDefaultProfile(manager, "4") + onExecutorAddedDefaultProfile(manager, "5") + onExecutorAddedDefaultProfile(manager, "6") + onExecutorAddedDefaultProfile(manager, "7") + onExecutorAddedDefaultProfile(manager, "8") assert(manager.executorMonitor.executorCount === 8) - assert(numExecutorsTarget(manager) === 8) + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) // Remove when numTargetExecutors is equal to the current number of executors @@ -489,12 +642,12 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { assert(removeExecutors(manager, Seq("2", "3")) !== Seq("2", "3")) // Remove until limit - onExecutorAdded(manager, "9") - onExecutorAdded(manager, "10") - onExecutorAdded(manager, "11") - onExecutorAdded(manager, "12") + onExecutorAddedDefaultProfile(manager, "9") + onExecutorAddedDefaultProfile(manager, "10") + onExecutorAddedDefaultProfile(manager, "11") + onExecutorAddedDefaultProfile(manager, "12") assert(manager.executorMonitor.executorCount === 12) - assert(numExecutorsTarget(manager) === 8) + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) assert(removeExecutor(manager, "1")) assert(removeExecutors(manager, Seq("2", "3", "4")) === Seq("2", "3", "4")) @@ -509,10 +662,10 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { // Add until limit assert(!removeExecutor(manager, "7")) // still at lower limit assert((manager, Seq("8")) !== Seq("8")) - onExecutorAdded(manager, "13") - onExecutorAdded(manager, "14") - onExecutorAdded(manager, "15") - onExecutorAdded(manager, "16") + onExecutorAddedDefaultProfile(manager, "13") + onExecutorAddedDefaultProfile(manager, "14") + onExecutorAddedDefaultProfile(manager, "15") + onExecutorAddedDefaultProfile(manager, "16") assert(manager.executorMonitor.executorCount === 12) // Remove succeeds again, now that we are no longer at the lower limit @@ -522,18 +675,21 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { onExecutorRemoved(manager, "5") onExecutorRemoved(manager, "6") assert(manager.executorMonitor.executorCount === 10) - assert(numExecutorsToAdd(manager) === 4) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) onExecutorRemoved(manager, "9") onExecutorRemoved(manager, "10") - assert(addExecutors(manager) === 4) // at upper limit - onExecutorAdded(manager, "17") - onExecutorAdded(manager, "18") + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 4) // at upper limit + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + onExecutorAddedDefaultProfile(manager, "17") + onExecutorAddedDefaultProfile(manager, "18") assert(manager.executorMonitor.executorCount === 10) - assert(addExecutors(manager) === 0) // still at upper limit - onExecutorAdded(manager, "19") - onExecutorAdded(manager, "20") + // still at upper limit + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + onExecutorAddedDefaultProfile(manager, "19") + onExecutorAddedDefaultProfile(manager, "20") assert(manager.executorMonitor.executorCount === 12) - assert(numExecutorsTarget(manager) === 12) + assert(numExecutorsTargetForDefaultProfileId(manager) === 12) } test("starting/canceling add timer") { @@ -571,22 +727,22 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val manager = createManager(createConf(0, 20, 0), clock = clock) // No events - we should not be adding or removing - assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) schedule(manager) - assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) clock.advance(100L) schedule(manager) - assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) clock.advance(1000L) schedule(manager) - assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) clock.advance(10000L) schedule(manager) - assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) } @@ -599,43 +755,43 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { onSchedulerBacklogged(manager) clock.advance(schedulerBacklogTimeout * 1000 / 2) schedule(manager) - assert(numExecutorsTarget(manager) === 0) // timer not exceeded yet + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) // timer not exceeded yet clock.advance(schedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 1) // first timer exceeded + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) // first timer exceeded clock.advance(sustainedSchedulerBacklogTimeout * 1000 / 2) schedule(manager) - assert(numExecutorsTarget(manager) === 1) // second timer not exceeded yet + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) // second timer not exceeded yet clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 1 + 2) // second timer exceeded + assert(numExecutorsTargetForDefaultProfileId(manager) === 1 + 2) // second timer exceeded clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 1 + 2 + 4) // third timer exceeded + assert(numExecutorsTargetForDefaultProfileId(manager) === 1 + 2 + 4) // third timer exceeded // Scheduler queue drained onSchedulerQueueEmpty(manager) clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 7) // timer is canceled + assert(numExecutorsTargetForDefaultProfileId(manager) === 7) // timer is canceled clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 7) + assert(numExecutorsTargetForDefaultProfileId(manager) === 7) // Scheduler queue backlogged again onSchedulerBacklogged(manager) clock.advance(schedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 7 + 1) // timer restarted + assert(numExecutorsTargetForDefaultProfileId(manager) === 7 + 1) // timer restarted clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 7 + 1 + 2) + assert(numExecutorsTargetForDefaultProfileId(manager) === 7 + 1 + 2) clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 7 + 1 + 2 + 4) + assert(numExecutorsTargetForDefaultProfileId(manager) === 7 + 1 + 2 + 4) clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 20) // limit reached + assert(numExecutorsTargetForDefaultProfileId(manager) === 20) // limit reached } test("mock polling loop remove behavior") { @@ -643,9 +799,9 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val manager = createManager(createConf(1, 20, 1), clock = clock) // Remove idle executors on timeout - onExecutorAdded(manager, "executor-1") - onExecutorAdded(manager, "executor-2") - onExecutorAdded(manager, "executor-3") + onExecutorAddedDefaultProfile(manager, "executor-1") + onExecutorAddedDefaultProfile(manager, "executor-2") + onExecutorAddedDefaultProfile(manager, "executor-3") assert(executorsPendingToRemove(manager).isEmpty) // idle threshold not reached yet @@ -661,10 +817,10 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { assert(executorsPendingToRemove(manager).size === 2) // limit reached (1 executor remaining) // Mark a subset as busy - only idle executors should be removed - onExecutorAdded(manager, "executor-4") - onExecutorAdded(manager, "executor-5") - onExecutorAdded(manager, "executor-6") - onExecutorAdded(manager, "executor-7") + onExecutorAddedDefaultProfile(manager, "executor-4") + onExecutorAddedDefaultProfile(manager, "executor-5") + onExecutorAddedDefaultProfile(manager, "executor-6") + onExecutorAddedDefaultProfile(manager, "executor-7") assert(manager.executorMonitor.executorCount === 7) assert(executorsPendingToRemove(manager).size === 2) // 2 pending to be removed onExecutorBusy(manager, "executor-4") @@ -729,23 +885,31 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val stage1 = createStageInfo(0, 1000) post(SparkListenerStageSubmitted(stage1)) - assert(addExecutors(manager) === 1) - assert(addExecutors(manager) === 2) - assert(addExecutors(manager) === 4) - assert(addExecutors(manager) === 8) - assert(numExecutorsTarget(manager) === 15) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 4) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 8) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 15) (0 until 15).foreach { i => - onExecutorAdded(manager, s"executor-$i") + onExecutorAddedDefaultProfile(manager, s"executor-$i") } assert(manager.executorMonitor.executorCount === 15) post(SparkListenerStageCompleted(stage1)) adjustRequestedExecutors(manager) - assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) post(SparkListenerStageSubmitted(createStageInfo(1, 1000))) - addExecutors(manager) - assert(numExecutorsTarget(manager) === 16) + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 16) } test("avoid ramp down initial executors until first job is submitted") { @@ -753,19 +917,19 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val manager = createManager(createConf(2, 5, 3), clock = clock) // Verify the initial number of executors - assert(numExecutorsTarget(manager) === 3) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) schedule(manager) // Verify whether the initial number of executors is kept with no pending tasks - assert(numExecutorsTarget(manager) === 3) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) post(SparkListenerStageSubmitted(createStageInfo(1, 2))) clock.advance(100L) - assert(maxNumExecutorsNeeded(manager) === 2) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 2) schedule(manager) // Verify that current number of executors should be ramp down when first job is submitted - assert(numExecutorsTarget(manager) === 2) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) } test("avoid ramp down initial executors until idle executor is timeout") { @@ -773,20 +937,20 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val manager = createManager(createConf(2, 5, 3), clock = clock) // Verify the initial number of executors - assert(numExecutorsTarget(manager) === 3) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) schedule(manager) // Verify the initial number of executors is kept when no pending tasks - assert(numExecutorsTarget(manager) === 3) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) (0 until 3).foreach { i => - onExecutorAdded(manager, s"executor-$i") + onExecutorAddedDefaultProfile(manager, s"executor-$i") } clock.advance(executorIdleTimeout * 1000) - assert(maxNumExecutorsNeeded(manager) === 0) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 0) schedule(manager) - // Verify executor is timeout,numExecutorsTarget is recalculated - assert(numExecutorsTarget(manager) === 2) + // Verify executor is timeout,numExecutorsTargetForDefaultProfileId is recalculated + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) } test("get pending task number and related locality preference") { @@ -802,7 +966,8 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val stageInfo1 = createStageInfo(1, 5, localityPreferences1) post(SparkListenerStageSubmitted(stageInfo1)) - assert(localityAwareTasks(manager) === 3) + assert(localityAwareTasksForDefaultProfile(manager) === 3) + val hostToLocal = hostToLocalTaskCount(manager) assert(hostToLocalTaskCount(manager) === Map("host1" -> 2, "host2" -> 3, "host3" -> 2, "host4" -> 2)) @@ -814,67 +979,76 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val stageInfo2 = createStageInfo(2, 3, localityPreferences2) post(SparkListenerStageSubmitted(stageInfo2)) - assert(localityAwareTasks(manager) === 5) + assert(localityAwareTasksForDefaultProfile(manager) === 5) assert(hostToLocalTaskCount(manager) === Map("host1" -> 2, "host2" -> 4, "host3" -> 4, "host4" -> 3, "host5" -> 2)) post(SparkListenerStageCompleted(stageInfo1)) - assert(localityAwareTasks(manager) === 2) + assert(localityAwareTasksForDefaultProfile(manager) === 2) assert(hostToLocalTaskCount(manager) === Map("host2" -> 1, "host3" -> 2, "host4" -> 1, "host5" -> 2)) } - test("SPARK-8366: maxNumExecutorsNeeded should properly handle failed tasks") { + test("SPARK-8366: maxNumExecutorsNeededPerResourceProfile should properly handle failed tasks") { val manager = createManager(createConf()) - assert(maxNumExecutorsNeeded(manager) === 0) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 0) post(SparkListenerStageSubmitted(createStageInfo(0, 1))) - assert(maxNumExecutorsNeeded(manager) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 1) val taskInfo = createTaskInfo(1, 1, "executor-1") post(SparkListenerTaskStart(0, 0, taskInfo)) - assert(maxNumExecutorsNeeded(manager) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 1) // If the task is failed, we expect it to be resubmitted later. val taskEndReason = ExceptionFailure(null, null, null, null, None) post(SparkListenerTaskEnd(0, 0, null, taskEndReason, taskInfo, new ExecutorMetrics, null)) - assert(maxNumExecutorsNeeded(manager) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 1) } test("reset the state of allocation manager") { val manager = createManager(createConf()) - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] // Allocation manager is reset when adding executor requests are sent without reporting back // executor added. post(SparkListenerStageSubmitted(createStageInfo(0, 10))) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 2) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 4) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 5) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) manager.reset() - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) assert(manager.executorMonitor.executorCount === 0) // Allocation manager is reset when executors are added. post(SparkListenerStageSubmitted(createStageInfo(0, 10))) - addExecutors(manager) - addExecutors(manager) - addExecutors(manager) - assert(numExecutorsTarget(manager) === 5) - - onExecutorAdded(manager, "first") - onExecutorAdded(manager, "second") - onExecutorAdded(manager, "third") - onExecutorAdded(manager, "fourth") - onExecutorAdded(manager, "fifth") + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + + onExecutorAddedDefaultProfile(manager, "first") + onExecutorAddedDefaultProfile(manager, "second") + onExecutorAddedDefaultProfile(manager, "third") + onExecutorAddedDefaultProfile(manager, "fourth") + onExecutorAddedDefaultProfile(manager, "fifth") assert(manager.executorMonitor.executorCount === 5) // Cluster manager lost will make all the live executors lost, so here simulate this behavior @@ -885,24 +1059,27 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { onExecutorRemoved(manager, "fifth") manager.reset() - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) assert(manager.executorMonitor.executorCount === 0) // Allocation manager is reset when executors are pending to remove - addExecutors(manager) - addExecutors(manager) - addExecutors(manager) - assert(numExecutorsTarget(manager) === 5) - - onExecutorAdded(manager, "first") - onExecutorAdded(manager, "second") - onExecutorAdded(manager, "third") - onExecutorAdded(manager, "fourth") - onExecutorAdded(manager, "fifth") - onExecutorAdded(manager, "sixth") - onExecutorAdded(manager, "seventh") - onExecutorAdded(manager, "eighth") + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + + onExecutorAddedDefaultProfile(manager, "first") + onExecutorAddedDefaultProfile(manager, "second") + onExecutorAddedDefaultProfile(manager, "third") + onExecutorAddedDefaultProfile(manager, "fourth") + onExecutorAddedDefaultProfile(manager, "fifth") + onExecutorAddedDefaultProfile(manager, "sixth") + onExecutorAddedDefaultProfile(manager, "seventh") + onExecutorAddedDefaultProfile(manager, "eighth") assert(manager.executorMonitor.executorCount === 8) removeExecutor(manager, "first") @@ -920,8 +1097,8 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { manager.reset() - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) assert(executorsPendingToRemove(manager) === Set.empty) assert(manager.executorMonitor.executorCount === 0) } @@ -932,31 +1109,31 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { createConf(1, 2, 1).set(config.DYN_ALLOCATION_TESTING, false), clock = clock) - when(client.requestTotalExecutors(meq(2), any(), any())).thenReturn(true) + when(client.requestTotalExecutors(any(), any(), any())).thenReturn(true) // test setup -- job with 2 tasks, scale up to two executors - assert(numExecutorsTarget(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) post(SparkListenerExecutorAdded( clock.getTimeMillis(), "executor-1", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) post(SparkListenerStageSubmitted(createStageInfo(0, 2))) clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 2) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) val taskInfo0 = createTaskInfo(0, 0, "executor-1") post(SparkListenerTaskStart(0, 0, taskInfo0)) post(SparkListenerExecutorAdded( clock.getTimeMillis(), "executor-2", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) val taskInfo1 = createTaskInfo(1, 1, "executor-2") post(SparkListenerTaskStart(0, 0, taskInfo1)) - assert(numExecutorsTarget(manager) === 2) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) // have one task finish -- we should adjust the target number of executors down // but we should *not* kill any executors yet post(SparkListenerTaskEnd(0, 0, null, Success, taskInfo0, new ExecutorMetrics, null)) - assert(maxNumExecutorsNeeded(manager) === 1) - assert(numExecutorsTarget(manager) === 2) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) verify(client, never).killExecutors(any(), any(), any(), any()) // now we cross the idle timeout for executor-1, so we kill it. the really important @@ -966,8 +1143,8 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { .thenReturn(Seq("executor-1")) clock.advance(3000) schedule(manager) - assert(maxNumExecutorsNeeded(manager) === 1) - assert(numExecutorsTarget(manager) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) // here's the important verify -- we did kill the executors, but did not adjust the target count verify(client).killExecutors(Seq("executor-1"), false, false, false) } @@ -975,7 +1152,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { test("SPARK-26758 check executor target number after idle time out ") { val clock = new ManualClock(10000L) val manager = createManager(createConf(1, 5, 3), clock = clock) - assert(numExecutorsTarget(manager) === 3) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) post(SparkListenerExecutorAdded( clock.getTimeMillis(), "executor-1", new ExecutorInfo("host1", 1, Map.empty))) post(SparkListenerExecutorAdded( @@ -986,14 +1163,14 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { clock.advance(executorIdleTimeout * 1000) schedule(manager) // once the schedule is run target executor number should be 1 - assert(numExecutorsTarget(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) } private def createConf( minExecutors: Int = 1, maxExecutors: Int = 5, initialExecutors: Int = 1): SparkConf = { - new SparkConf() + val sparkConf = new SparkConf() .set(config.DYN_ALLOCATION_ENABLED, true) .set(config.DYN_ALLOCATION_MIN_EXECUTORS, minExecutors) .set(config.DYN_ALLOCATION_MAX_EXECUTORS, maxExecutors) @@ -1008,12 +1185,16 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { // SPARK-22864: effectively disable the allocation schedule by setting the period to a // really long value. .set(TEST_SCHEDULE_INTERVAL, 10000L) + ResourceProfile.reInitDefaultProfile(sparkConf) + sparkConf } private def createManager( conf: SparkConf, clock: Clock = new SystemClock()): ExecutorAllocationManager = { - val manager = new ExecutorAllocationManager(client, listenerBus, conf, clock = clock) + rpManager = new ResourceProfileManager(conf, listenerBus) + val manager = new ExecutorAllocationManager(client, listenerBus, conf, clock = clock, + resourceProfileManager = rpManager) managers += manager manager.start() manager @@ -1022,7 +1203,18 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { private val execInfo = new ExecutorInfo("host1", 1, Map.empty, Map.empty, Map.empty, DEFAULT_RESOURCE_PROFILE_ID) - private def onExecutorAdded(manager: ExecutorAllocationManager, id: String): Unit = { + private def onExecutorAddedDefaultProfile( + manager: ExecutorAllocationManager, + id: String): Unit = { + post(SparkListenerExecutorAdded(0L, id, execInfo)) + } + + private def onExecutorAdded( + manager: ExecutorAllocationManager, + id: String, + rp: ResourceProfile): Unit = { + val cores = rp.getExecutorCores.getOrElse(1) + val execInfo = new ExecutorInfo("host1", cores, Map.empty, Map.empty, Map.empty, rp.id) post(SparkListenerExecutorAdded(0L, id, execInfo)) } @@ -1064,10 +1256,11 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { stageId: Int, numTasks: Int, taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty, - attemptId: Int = 0 + attemptId: Int = 0, + rp: ResourceProfile = defaultProfile ): StageInfo = { new StageInfo(stageId, attemptId, "name", numTasks, Seq.empty, Seq.empty, "no details", - taskLocalityPreferences = taskLocalityPreferences) + taskLocalityPreferences = taskLocalityPreferences, resourceProfileId = rp.id) } private def createTaskInfo( @@ -1082,47 +1275,101 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { | Helper methods for accessing private methods and fields | * ------------------------------------------------------- */ - private val _numExecutorsToAdd = PrivateMethod[Int](Symbol("numExecutorsToAdd")) - private val _numExecutorsTarget = PrivateMethod[Int](Symbol("numExecutorsTarget")) - private val _maxNumExecutorsNeeded = PrivateMethod[Int](Symbol("maxNumExecutorsNeeded")) + private val _numExecutorsToAddPerResourceProfileId = + PrivateMethod[mutable.HashMap[Int, Int]]( + Symbol("numExecutorsToAddPerResourceProfileId")) + private val _numExecutorsTargetPerResourceProfileId = + PrivateMethod[mutable.HashMap[Int, Int]]( + Symbol("numExecutorsTargetPerResourceProfileId")) + private val _maxNumExecutorsNeededPerResourceProfile = + PrivateMethod[Int](Symbol("maxNumExecutorsNeededPerResourceProfile")) private val _addTime = PrivateMethod[Long](Symbol("addTime")) private val _schedule = PrivateMethod[Unit](Symbol("schedule")) - private val _addExecutors = PrivateMethod[Int](Symbol("addExecutors")) + private val _doUpdateRequest = PrivateMethod[Unit](Symbol("doUpdateRequest")) private val _updateAndSyncNumExecutorsTarget = PrivateMethod[Int](Symbol("updateAndSyncNumExecutorsTarget")) + private val _addExecutorsToTarget = PrivateMethod[Int](Symbol("addExecutorsToTarget")) private val _removeExecutors = PrivateMethod[Seq[String]](Symbol("removeExecutors")) private val _onSchedulerBacklogged = PrivateMethod[Unit](Symbol("onSchedulerBacklogged")) private val _onSchedulerQueueEmpty = PrivateMethod[Unit](Symbol("onSchedulerQueueEmpty")) - private val _localityAwareTasks = PrivateMethod[Int](Symbol("localityAwareTasks")) - private val _hostToLocalTaskCount = - PrivateMethod[Map[String, Int]](Symbol("hostToLocalTaskCount")) + private val _localityAwareTasksPerResourceProfileId = + PrivateMethod[mutable.HashMap[Int, Int]](Symbol("numLocalityAwareTasksPerResourceProfileId")) + private val _rpIdToHostToLocalTaskCount = + PrivateMethod[Map[Int, Map[String, Int]]](Symbol("rpIdToHostToLocalTaskCount")) private val _onSpeculativeTaskSubmitted = PrivateMethod[Unit](Symbol("onSpeculativeTaskSubmitted")) - private val _totalRunningTasks = PrivateMethod[Int](Symbol("totalRunningTasks")) + private val _totalRunningTasksPerResourceProfile = + PrivateMethod[Int](Symbol("totalRunningTasksPerResourceProfile")) + + private val defaultProfile = ResourceProfile.getOrCreateDefaultProfile(new SparkConf) + + private def numExecutorsToAddForDefaultProfile(manager: ExecutorAllocationManager): Int = { + numExecutorsToAdd(manager, defaultProfile) + } + + private def numExecutorsToAdd( + manager: ExecutorAllocationManager, + rp: ResourceProfile): Int = { + val nmap = manager invokePrivate _numExecutorsToAddPerResourceProfileId() + nmap(rp.id) + } + + private def updateAndSyncNumExecutorsTarget( + manager: ExecutorAllocationManager, + now: Long): Unit = { + manager invokePrivate _updateAndSyncNumExecutorsTarget(now) + } - private def numExecutorsToAdd(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _numExecutorsToAdd() + private def numExecutorsTargetForDefaultProfileId(manager: ExecutorAllocationManager): Int = { + numExecutorsTarget(manager, defaultProfile.id) } - private def numExecutorsTarget(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _numExecutorsTarget() + private def numExecutorsTarget( + manager: ExecutorAllocationManager, + rpId: Int): Int = { + val numMap = manager invokePrivate _numExecutorsTargetPerResourceProfileId() + numMap(rpId) + } + + private def addExecutorsToTargetForDefaultProfile( + manager: ExecutorAllocationManager, + updatesNeeded: mutable.HashMap[ResourceProfile, + ExecutorAllocationManager.TargetNumUpdates] + ): Int = { + addExecutorsToTarget(manager, updatesNeeded, defaultProfile) + } + + private def addExecutorsToTarget( + manager: ExecutorAllocationManager, + updatesNeeded: mutable.HashMap[ResourceProfile, + ExecutorAllocationManager.TargetNumUpdates], + rp: ResourceProfile + ): Int = { + val maxNumExecutorsNeeded = + manager invokePrivate _maxNumExecutorsNeededPerResourceProfile(rp.id) + manager invokePrivate + _addExecutorsToTarget(maxNumExecutorsNeeded, rp.id, updatesNeeded) } private def addTime(manager: ExecutorAllocationManager): Long = { manager invokePrivate _addTime() } - private def schedule(manager: ExecutorAllocationManager): Unit = { - manager invokePrivate _schedule() + private def doUpdateRequest( + manager: ExecutorAllocationManager, + updates: Map[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates], + now: Long): Unit = { + manager invokePrivate _doUpdateRequest(updates, now) } - private def maxNumExecutorsNeeded(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _maxNumExecutorsNeeded() + private def schedule(manager: ExecutorAllocationManager): Unit = { + manager invokePrivate _schedule() } - private def addExecutors(manager: ExecutorAllocationManager): Int = { - val maxNumExecutorsNeeded = manager invokePrivate _maxNumExecutorsNeeded() - manager invokePrivate _addExecutors(maxNumExecutorsNeeded) + private def maxNumExecutorsNeededPerResourceProfile( + manager: ExecutorAllocationManager, + rp: ResourceProfile): Int = { + manager invokePrivate _maxNumExecutorsNeededPerResourceProfile(rp.id) } private def adjustRequestedExecutors(manager: ExecutorAllocationManager): Int = { @@ -1145,15 +1392,22 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { manager invokePrivate _onSpeculativeTaskSubmitted(id) } - private def localityAwareTasks(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _localityAwareTasks() + private def localityAwareTasksForDefaultProfile(manager: ExecutorAllocationManager): Int = { + val localMap = manager invokePrivate _localityAwareTasksPerResourceProfileId() + localMap(defaultProfile.id) + } + + private def totalRunningTasksPerResourceProfile(manager: ExecutorAllocationManager): Int = { + manager invokePrivate _totalRunningTasksPerResourceProfile(defaultProfile.id) } - private def totalRunningTasks(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _totalRunningTasks() + private def hostToLocalTaskCount( + manager: ExecutorAllocationManager): Map[String, Int] = { + val rpIdToHostLocal = manager invokePrivate _rpIdToHostToLocalTaskCount() + rpIdToHostLocal(defaultProfile.id) } - private def hostToLocalTaskCount(manager: ExecutorAllocationManager): Map[String, Int] = { - manager invokePrivate _hostToLocalTaskCount() + private def getResourceProfileIdOfExecutor(manager: ExecutorAllocationManager): Int = { + defaultProfile.id } } diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index ff0f2f9134ed3..a7bac592aecf1 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -30,7 +30,7 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.config.DYN_ALLOCATION_TESTING -import org.apache.spark.resource.ResourceProfile +import org.apache.spark.resource.{ResourceProfile, ResourceProfileManager} import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -61,6 +61,7 @@ class HeartbeatReceiverSuite PrivateMethod[collection.Map[String, Long]](Symbol("executorLastSeen")) private val _executorTimeoutMs = PrivateMethod[Long](Symbol("executorTimeoutMs")) private val _killExecutorThread = PrivateMethod[ExecutorService](Symbol("killExecutorThread")) + var conf: SparkConf = _ /** * Before each test, set up the SparkContext and a custom [[HeartbeatReceiver]] @@ -68,7 +69,7 @@ class HeartbeatReceiverSuite */ override def beforeEach(): Unit = { super.beforeEach() - val conf = new SparkConf() + conf = new SparkConf() .setMaster("local[2]") .setAppName("test") .set(DYN_ALLOCATION_TESTING, true) @@ -76,7 +77,6 @@ class HeartbeatReceiverSuite scheduler = mock(classOf[TaskSchedulerImpl]) when(sc.taskScheduler).thenReturn(scheduler) when(scheduler.nodeBlacklist).thenReturn(Predef.Set[String]()) - when(scheduler.resourcesReqsPerTask).thenReturn(Seq.empty) when(scheduler.sc).thenReturn(sc) heartbeatReceiverClock = new ManualClock heartbeatReceiver = new HeartbeatReceiver(sc, heartbeatReceiverClock) @@ -164,9 +164,10 @@ class HeartbeatReceiverSuite test("expire dead hosts should kill executors with replacement (SPARK-8119)") { // Set up a fake backend and cluster manager to simulate killing executors val rpcEnv = sc.env.rpcEnv - val fakeClusterManager = new FakeClusterManager(rpcEnv) + val fakeClusterManager = new FakeClusterManager(rpcEnv, conf) val fakeClusterManagerRef = rpcEnv.setupEndpoint("fake-cm", fakeClusterManager) - val fakeSchedulerBackend = new FakeSchedulerBackend(scheduler, rpcEnv, fakeClusterManagerRef) + val fakeSchedulerBackend = + new FakeSchedulerBackend(scheduler, rpcEnv, fakeClusterManagerRef, sc.resourceProfileManager) when(sc.schedulerBackend).thenReturn(fakeSchedulerBackend) // Register fake executors with our fake scheduler backend @@ -282,12 +283,15 @@ private class FakeExecutorEndpoint(override val rpcEnv: RpcEnv) extends RpcEndpo private class FakeSchedulerBackend( scheduler: TaskSchedulerImpl, rpcEnv: RpcEnv, - clusterManagerEndpoint: RpcEndpointRef) + clusterManagerEndpoint: RpcEndpointRef, + resourceProfileManager: ResourceProfileManager) extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { - protected override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { + protected override def doRequestTotalExecutors( + resourceProfileToTotalExecs: Map[ResourceProfile, Int]): Future[Boolean] = { clusterManagerEndpoint.ask[Boolean]( - RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount, Set.empty)) + RequestExecutors(numLocalityAwareTasksPerResourceProfileId, + rpHostToLocalTaskCount, Set.empty, resourceProfileToTotalExecs)) } protected override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = { @@ -298,7 +302,7 @@ private class FakeSchedulerBackend( /** * Dummy cluster manager to simulate responses to executor allocation requests. */ -private class FakeClusterManager(override val rpcEnv: RpcEnv) extends RpcEndpoint { +private class FakeClusterManager(override val rpcEnv: RpcEnv, conf: SparkConf) extends RpcEndpoint { private var targetNumExecutors = 0 private val executorIdsToKill = new mutable.HashSet[String] @@ -306,8 +310,9 @@ private class FakeClusterManager(override val rpcEnv: RpcEnv) extends RpcEndpoin def getExecutorIdsToKill: Set[String] = executorIdsToKill.toSet override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case RequestExecutors(requestedTotal, _, _, _) => - targetNumExecutors = requestedTotal + case RequestExecutors(_, _, _, resourceProfileToTotalExecs) => + targetNumExecutors = + resourceProfileToTotalExecs(ResourceProfile.getOrCreateDefaultProfile(conf)) context.reply(true) case KillExecutors(executorIds) => executorIdsToKill ++= executorIds diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala index 1fe12e116d96e..599ea8955491f 100644 --- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala @@ -44,7 +44,7 @@ trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self def resetSparkContext(): Unit = { LocalSparkContext.stop(sc) - ResourceProfile.clearDefaultProfile + ResourceProfile.clearDefaultProfile() sc = null } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index df9c7c5eaa368..b6dfa69015c28 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -36,6 +36,7 @@ import org.scalatest.concurrent.Eventually import org.apache.spark.TestUtils._ import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests._ import org.apache.spark.internal.config.UI._ import org.apache.spark.resource.ResourceAllocation import org.apache.spark.resource.ResourceUtils._ @@ -784,7 +785,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } test(s"Avoid setting ${CPUS_PER_TASK.key} unreasonably (SPARK-27192)") { - val FAIL_REASON = s"has to be >= the task config: ${CPUS_PER_TASK.key}" + val FAIL_REASON = " has to be >= the number of cpus per task" Seq( ("local", 2, None), ("local[2]", 3, None), @@ -864,9 +865,8 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc = new SparkContext(conf) }.getMessage() - assert(error.contains("The executor resource config: spark.executor.resource.gpu.amount " + - "needs to be specified since a task requirement config: spark.task.resource.gpu.amount " + - "was specified")) + assert(error.contains("No executor resource configs were not specified for the following " + + "task configs: gpu")) } test("Test parsing resources executor config < task requirements") { @@ -880,15 +880,15 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc = new SparkContext(conf) }.getMessage() - assert(error.contains("The executor resource config: spark.executor.resource.gpu.amount = 1 " + - "has to be >= the requested amount in task resource config: " + - "spark.task.resource.gpu.amount = 2")) + assert(error.contains("The executor resource: gpu, amount: 1 needs to be >= the task " + + "resource request amount of 2.0")) } test("Parse resources executor config not the same multiple numbers of the task requirements") { val conf = new SparkConf() .setMaster("local-cluster[1, 1, 1024]") .setAppName("test-cluster") + conf.set(RESOURCES_WARNING_TESTING, true) conf.set(TASK_GPU_ID.amountConf, "2") conf.set(EXECUTOR_GPU_ID.amountConf, "4") @@ -897,25 +897,9 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu }.getMessage() assert(error.contains( - "The configuration of resource: gpu (exec = 4, task = 2, runnable tasks = 2) will result " + - "in wasted resources due to resource CPU limiting the number of runnable tasks per " + - "executor to: 1. Please adjust your configuration.")) - } - - test("Parse resources executor config cpus not limiting resource") { - val conf = new SparkConf() - .setMaster("local-cluster[1, 8, 1024]") - .setAppName("test-cluster") - conf.set(TASK_GPU_ID.amountConf, "2") - conf.set(EXECUTOR_GPU_ID.amountConf, "4") - - var error = intercept[IllegalArgumentException] { - sc = new SparkContext(conf) - }.getMessage() - - assert(error.contains("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")) + "The configuration of resource: gpu (exec = 4, task = 2.0/1, runnable tasks = 2) will " + + "result in wasted resources due to resource cpus limiting the number of runnable " + + "tasks per executor to: 1. Please adjust your configuration.")) } test("test resource scheduling under local-cluster mode") { diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index f8b99302c4ad5..3712e5dfe803e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -511,7 +511,6 @@ class StandaloneDynamicAllocationSuite val taskScheduler = mock(classOf[TaskSchedulerImpl]) when(taskScheduler.nodeBlacklist()).thenReturn(Set("blacklisted-host")) when(taskScheduler.resourceOffers(any())).thenReturn(Nil) - when(taskScheduler.resourcesReqsPerTask).thenReturn(Seq.empty) when(taskScheduler.sc).thenReturn(sc) val rpcEnv = RpcEnv.create("test-rpcenv", "localhost", 0, conf, securityManager) @@ -631,6 +630,7 @@ class StandaloneDynamicAllocationSuite when(endpointRef.address).thenReturn(mockAddress) val message = RegisterExecutor(id, endpointRef, "localhost", 10, Map.empty, Map.empty, Map.empty, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val backend = sc.schedulerBackend.asInstanceOf[CoarseGrainedSchedulerBackend] backend.driverEndpoint.askSync[Boolean](message) backend.driverEndpoint.send(LaunchedExecutor(id)) } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala index 86511ae08784a..c905797bf1287 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.history import org.apache.spark.{SparkFunSuite, Success, TaskResultLost, TaskState} import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler._ import org.apache.spark.status.ListenerEventsTestHelper @@ -141,7 +142,8 @@ class BasicEventFilterBuilderSuite extends SparkFunSuite { // - Re-submit stage 1, all tasks, and succeed them and the stage. val oldS1 = stages.last val newS1 = new StageInfo(oldS1.stageId, oldS1.attemptNumber + 1, oldS1.name, oldS1.numTasks, - oldS1.rddInfos, oldS1.parentIds, oldS1.details, oldS1.taskMetrics) + oldS1.rddInfos, oldS1.parentIds, oldS1.details, oldS1.taskMetrics, + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) time += 1 newS1.submissionTime = Some(time) diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 06554e570a758..940ee3276459d 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -171,6 +171,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "executor node blacklisting unblacklisting" -> "applications/app-20161115172038-0000/executors", "executor memory usage" -> "applications/app-20161116163331-0000/executors", "executor resource information" -> "applications/application_1555004656427_0144/executors", + "multiple resource profiles" -> "applications/application_1578436911597_0052/environment", "app environment" -> "applications/app-20161116163331-0000/environment", diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala index a996fc4a0b648..cfe018a12a53c 100644 --- a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -300,8 +300,8 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite val taskId = 1000000 // We don't really verify the data, just pass it around. val data = ByteBuffer.wrap(Array[Byte](1, 2, 3, 4)) - val taskDescription = new TaskDescription(taskId, 2, "1", "TASK 1000000", - 19, 1, mutable.Map.empty, mutable.Map.empty, new Properties, + val taskDescription = new TaskDescription(taskId, 2, "1", "TASK 1000000", 19, 1, + 1, mutable.Map.empty, mutable.Map.empty, new Properties, Map(GPU -> new ResourceInformation(GPU, Array("0", "1"))), data) val serializedTaskDescription = TaskDescription.encode(taskDescription) backend.executor = mock[Executor] @@ -311,15 +311,17 @@ class CoarseGrainedExecutorBackendSuite extends SparkFunSuite backend.self.send(LaunchTask(new SerializableBuffer(serializedTaskDescription))) eventually(timeout(10.seconds)) { assert(backend.taskResources.size == 1) - val resources = backend.taskResources(taskId) + val (taskcpus, resources) = backend.taskResources(taskId) assert(resources(GPU).addresses sameElements Array("0", "1")) + assert(taskcpus === 1) } // Update the status of a running task shall not affect `taskResources` map. backend.statusUpdate(taskId, TaskState.RUNNING, data) assert(backend.taskResources.size == 1) - val resources = backend.taskResources(taskId) + val (taskcpus, resources) = backend.taskResources(taskId) assert(resources(GPU).addresses sameElements Array("0", "1")) + assert(taskcpus === 1) // Update the status of a finished task shall remove the entry from `taskResources` map. backend.statusUpdate(taskId, TaskState.FINISHED, data) diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 7272a98c9770b..9bcd9767bc44b 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -445,6 +445,7 @@ class ExecutorSuite extends SparkFunSuite name = "", index = 0, partitionId = 0, + cpus = 1, addedFiles = Map[String, Long](), addedJars = Map[String, Long](), properties = new Properties, diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 18154d861a731..2d9adcac2825c 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -33,7 +33,9 @@ import org.scalatest.concurrent.Eventually import org.apache.spark._ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.internal.config.RDD_PARALLEL_LISTING_THRESHOLD +import org.apache.spark.internal.config.Tests.RESOURCE_PROFILE_MANAGER_TESTING import org.apache.spark.rdd.RDDSuiteUtils._ +import org.apache.spark.resource._ import org.apache.spark.util.{ThreadUtils, Utils} class RDDSuite extends SparkFunSuite with SharedSparkContext with Eventually { @@ -170,6 +172,20 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext with Eventually { } } + test("withResources raises exception in local mode") { + val rp = new ResourceProfileBuilder() + val ereqs = new ExecutorResourceRequests().resource("gpu", 1) + val treqs = new TaskResourceRequests().resource("gpu", 1) + rp.require(ereqs).require(treqs) + sc.conf.set(RESOURCE_PROFILE_MANAGER_TESTING, true) + val error = intercept[SparkException] { + val rdd = sc.parallelize(Seq(1 -> true)).withResources(rp.build) + }.getMessage() + + assert(error.contains("ResourceProfiles are only supported on YARN with dynamic " + + "allocation enabled")) + } + test("SPARK-23778: empty RDD in union should not produce a UnionRDD") { val rddWithPartitioner = sc.parallelize(Seq(1 -> true)).partitionBy(new HashPartitioner(1)) val emptyRDD = sc.emptyRDD[(Int, Boolean)] diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala new file mode 100644 index 0000000000000..5d3c944bf3e5d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala @@ -0,0 +1,106 @@ +/* + * 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 org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests._ +import org.apache.spark.scheduler.LiveListenerBus + +class ResourceProfileManagerSuite extends SparkFunSuite { + + override def beforeAll() { + try { + ResourceProfile.clearDefaultProfile + } finally { + super.beforeAll() + } + } + + override def afterEach() { + try { + ResourceProfile.clearDefaultProfile + } finally { + super.afterEach() + } + } + + val listenerBus = new LiveListenerBus(new SparkConf()) + + test("ResourceProfileManager") { + val conf = new SparkConf().set(EXECUTOR_CORES, 4) + val rpmanager = new ResourceProfileManager(conf, listenerBus) + val defaultProf = rpmanager.defaultResourceProfile + assert(defaultProf.id === ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + assert(defaultProf.executorResources.size === 2, + "Executor resources should contain cores and memory by default") + assert(defaultProf.executorResources(ResourceProfile.CORES).amount === 4, + s"Executor resources should have 4 cores") + } + + test("isSupported yarn no dynamic allocation") { + val conf = new SparkConf().setMaster("yarn").set(EXECUTOR_CORES, 4) + conf.set(RESOURCE_PROFILE_MANAGER_TESTING.key, "true") + val rpmanager = new ResourceProfileManager(conf, listenerBus) + // default profile should always work + val defaultProf = rpmanager.defaultResourceProfile + val rprof = new ResourceProfileBuilder() + val gpuExecReq = + new ExecutorResourceRequests().resource("gpu", 2, "someScript") + val immrprof = rprof.require(gpuExecReq).build + var error = intercept[SparkException] { + rpmanager.isSupported(immrprof) + }.getMessage() + + assert(error.contains("ResourceProfiles are only supported on YARN with dynamic allocation")) + } + + test("isSupported yarn with dynamic allocation") { + val conf = new SparkConf().setMaster("yarn").set(EXECUTOR_CORES, 4) + conf.set(DYN_ALLOCATION_ENABLED, true) + conf.set(RESOURCE_PROFILE_MANAGER_TESTING.key, "true") + val rpmanager = new ResourceProfileManager(conf, listenerBus) + // default profile should always work + val defaultProf = rpmanager.defaultResourceProfile + val rprof = new ResourceProfileBuilder() + val gpuExecReq = + new ExecutorResourceRequests().resource("gpu", 2, "someScript") + val immrprof = rprof.require(gpuExecReq).build + assert(rpmanager.isSupported(immrprof) == true) + } + + test("isSupported yarn with local mode") { + val conf = new SparkConf().setMaster("local").set(EXECUTOR_CORES, 4) + conf.set(RESOURCE_PROFILE_MANAGER_TESTING.key, "true") + val rpmanager = new ResourceProfileManager(conf, listenerBus) + // default profile should always work + val defaultProf = rpmanager.defaultResourceProfile + val rprof = new ResourceProfileBuilder() + val gpuExecReq = + new ExecutorResourceRequests().resource("gpu", 2, "someScript") + val immrprof = rprof.require(gpuExecReq).build + var error = intercept[SparkException] { + rpmanager.isSupported(immrprof) + }.getMessage() + + assert(error.contains("ResourceProfiles are only supported on YARN with dynamic allocation")) + } + + + +} diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala index c0637eeeacaba..652b99df8fe20 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala @@ -18,11 +18,20 @@ package org.apache.spark.resource import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.internal.config.{EXECUTOR_CORES, EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD, SPARK_EXECUTOR_PREFIX} +import org.apache.spark.internal.config.{EXECUTOR_CORES, EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD, SPARK_EXECUTOR_PREFIX, SPARK_TASK_PREFIX} import org.apache.spark.internal.config.Python.PYSPARK_EXECUTOR_MEMORY +import org.apache.spark.resource.TestResourceIDs._ class ResourceProfileSuite extends SparkFunSuite { + override def beforeAll() { + try { + ResourceProfile.clearDefaultProfile + } finally { + super.beforeAll() + } + } + override def afterEach() { try { ResourceProfile.clearDefaultProfile @@ -59,18 +68,19 @@ class ResourceProfileSuite extends SparkFunSuite { conf.set(EXECUTOR_MEMORY_OVERHEAD.key, "1g") conf.set(EXECUTOR_MEMORY.key, "4g") conf.set(EXECUTOR_CORES.key, "4") - conf.set("spark.task.resource.gpu.amount", "1") - conf.set(s"$SPARK_EXECUTOR_PREFIX.resource.gpu.amount", "1") - conf.set(s"$SPARK_EXECUTOR_PREFIX.resource.gpu.discoveryScript", "nameOfScript") + conf.set(TASK_GPU_ID.amountConf, "1") + conf.set(EXECUTOR_GPU_ID.amountConf, "1") + conf.set(EXECUTOR_GPU_ID.discoveryScriptConf, "nameOfScript") val rprof = ResourceProfile.getOrCreateDefaultProfile(conf) assert(rprof.id === ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val execResources = rprof.executorResources - assert(execResources.size === 5, - "Executor resources should contain cores, memory, and gpu " + execResources) + assert(execResources.size === 5, s"Executor resources should contain cores, pyspark " + + s"memory, memory overhead, memory, and gpu $execResources") assert(execResources.contains("gpu"), "Executor resources should have gpu") assert(rprof.executorResources(ResourceProfile.CORES).amount === 4, "Executor resources should have 4 core") - assert(rprof.getExecutorCores.get === 4, "Executor resources should have 4 core") + assert(rprof.getExecutorCores.get === 4, + "Executor resources should have 4 core") assert(rprof.executorResources(ResourceProfile.MEMORY).amount === 4096, "Executor resources should have 1024 memory") assert(rprof.executorResources(ResourceProfile.PYSPARK_MEM).amount == 2048, @@ -84,12 +94,118 @@ class ResourceProfileSuite extends SparkFunSuite { test("test default profile task gpus fractional") { val sparkConf = new SparkConf() - .set("spark.executor.resource.gpu.amount", "2") - .set("spark.task.resource.gpu.amount", "0.33") + .set(EXECUTOR_GPU_ID.amountConf, "2") + .set(TASK_GPU_ID.amountConf, "0.33") val immrprof = ResourceProfile.getOrCreateDefaultProfile(sparkConf) assert(immrprof.taskResources.get("gpu").get.amount == 0.33) } + test("Internal pyspark memory confs") { + val rprof = new ResourceProfileBuilder() + val gpuExecReq = + new ExecutorResourceRequests().cores(2).pysparkMemory("2g").resource("gpu", 2, "someScript") + rprof.require(gpuExecReq) + val immrprof = new ResourceProfile(rprof.executorResources, rprof.taskResources) + val pysparkConfs = immrprof.getInternalPysparkMemoryConfs + val sparkConf = new SparkConf + pysparkConfs.foreach { case(key, value) => sparkConf.set(key, value) } + val pysparkmemory = + ResourceProfile.getPysparkMemoryFromInternalConfs(sparkConf, immrprof.id) + assert(pysparkmemory.get === 2048) + } + + test("maxTasksPerExecutor cpus") { + val sparkConf = new SparkConf() + .set(EXECUTOR_CORES, 1) + val rprof = new ResourceProfileBuilder() + val taskReq = new TaskResourceRequests().resource("gpu", 1) + val execReq = + new ExecutorResourceRequests().resource("gpu", 2, "myscript", "nvidia") + rprof.require(taskReq).require(execReq) + val immrprof = new ResourceProfile(rprof.executorResources, rprof.taskResources) + assert(immrprof.limitingResource(sparkConf) == "cpus") + assert(immrprof.maxTasksPerExecutor(sparkConf) == 1) + } + + test("maxTasksPerExecutor gpus") { + val sparkConf = new SparkConf() + .set(EXECUTOR_CORES, 6) + val rprof = new ResourceProfileBuilder() + val taskReq = new TaskResourceRequests().resource("gpu", 2) + val execReq = + new ExecutorResourceRequests().resource("gpu", 4, "myscript", "nvidia") + rprof.require(taskReq).require(execReq) + val immrprof = new ResourceProfile(rprof.executorResources, rprof.taskResources) + assert(immrprof.limitingResource(sparkConf) == "gpu") + assert(immrprof.maxTasksPerExecutor(sparkConf) == 2) + assert(immrprof.getNumSlotsPerAddress("gpu", sparkConf) == 1) + } + + test("maxTasksPerExecutor gpus fractional") { + val sparkConf = new SparkConf() + .set(EXECUTOR_CORES, 6) + val rprof = new ResourceProfileBuilder() + val taskReq = new TaskResourceRequests().resource("gpu", 0.5) + val execReq = new ExecutorResourceRequests().resource("gpu", 2, "myscript", "nvidia") + rprof.require(taskReq).require(execReq) + val immrprof = new ResourceProfile(rprof.executorResources, rprof.taskResources) + assert(immrprof.limitingResource(sparkConf) == "gpu") + assert(immrprof.maxTasksPerExecutor(sparkConf) == 4) + assert(immrprof.getNumSlotsPerAddress("gpu", sparkConf) == 2) + } + + test("maxTasksPerExecutor multiple resources") { + val sparkConf = new SparkConf() + .set(EXECUTOR_CORES, 6) + val rprof = new ResourceProfileBuilder() + val taskReqs = new TaskResourceRequests() + val execReqs = new ExecutorResourceRequests() + taskReqs.resource("gpu", 1) + execReqs.resource("gpu", 6, "myscript", "nvidia") + taskReqs.resource("fpga", 1) + execReqs.resource("fpga", 4, "myscript", "nvidia") + rprof.require(taskReqs).require(execReqs) + val immrprof = new ResourceProfile(rprof.executorResources, rprof.taskResources) + assert(immrprof.limitingResource(sparkConf) == "fpga") + assert(immrprof.maxTasksPerExecutor(sparkConf) == 4) + assert(immrprof.getNumSlotsPerAddress("gpu", sparkConf) == 1) + assert(immrprof.getNumSlotsPerAddress("fpga", sparkConf) == 1) + } + + test("maxTasksPerExecutor/limiting no executor cores") { + val sparkConf = new SparkConf().setMaster("spark://testing") + val rprof = new ResourceProfileBuilder() + val taskReq = new TaskResourceRequests().resource("gpu", 1) + val execReq = + new ExecutorResourceRequests().resource("gpu", 2, "myscript", "nvidia") + rprof.require(taskReq).require(execReq) + val immrprof = new ResourceProfile(rprof.executorResources, rprof.taskResources) + assert(immrprof.limitingResource(sparkConf) == "gpu") + assert(immrprof.maxTasksPerExecutor(sparkConf) == 2) + assert(immrprof.isCoresLimitKnown == false) + } + + test("maxTasksPerExecutor/limiting no other resource no executor cores") { + val sparkConf = new SparkConf().setMaster("spark://testing") + val immrprof = ResourceProfile.getOrCreateDefaultProfile(sparkConf) + assert(immrprof.limitingResource(sparkConf) == "") + assert(immrprof.maxTasksPerExecutor(sparkConf) == 1) + assert(immrprof.isCoresLimitKnown == false) + } + + test("maxTasksPerExecutor/limiting executor cores") { + val sparkConf = new SparkConf().setMaster("spark://testing").set(EXECUTOR_CORES, 2) + val rprof = new ResourceProfileBuilder() + val taskReq = new TaskResourceRequests().resource("gpu", 1) + val execReq = + new ExecutorResourceRequests().resource("gpu", 2, "myscript", "nvidia") + rprof.require(taskReq).require(execReq) + val immrprof = new ResourceProfile(rprof.executorResources, rprof.taskResources) + assert(immrprof.limitingResource(sparkConf) == ResourceProfile.CPUS) + assert(immrprof.maxTasksPerExecutor(sparkConf) == 2) + assert(immrprof.isCoresLimitKnown == true) + } + test("Create ResourceProfile") { val rprof = new ResourceProfileBuilder() val taskReq = new TaskResourceRequests().resource("gpu", 1) diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala index b809469fd7163..33944b0d2ad9f 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala @@ -25,13 +25,17 @@ import org.json4s.{DefaultFormats, Extraction} import org.apache.spark.{LocalSparkContext, SparkConf, SparkException, SparkFunSuite} import org.apache.spark.TestUtils._ import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests._ import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ +import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.util.Utils class ResourceUtilsSuite extends SparkFunSuite with LocalSparkContext { + val listenerBus = new LiveListenerBus(new SparkConf()) + test("ResourceID") { val componentName = "spark.test" val resourceName = "p100" @@ -310,4 +314,125 @@ class ResourceUtilsSuite extends SparkFunSuite assert(error.contains("User is expecting to use resource: gpu, but " + "didn't specify a discovery script!")) } + + private def testWarningsClusterManagerWithCores(conf: SparkConf) = { + // this should pass as default case with just 1 core + var rpmanager = new ResourceProfileManager(conf, listenerBus) + + ResourceProfile.clearDefaultProfile + conf.set(EXECUTOR_CORES, 4) + conf.set("spark.executor.resource.gpu.amount", "1") + conf.set("spark.task.resource.gpu.amount", "1") + var error = intercept[SparkException] { + rpmanager = new ResourceProfileManager(conf, listenerBus) + }.getMessage() + + assert(error.contains( + "The configuration of cores (exec = 4 task = 1, runnable tasks = 4) will result in " + + "wasted resources due to resource gpu limiting the number of runnable tasks per " + + "executor to: 1. Please adjust your configuration.")) + + ResourceProfile.clearDefaultProfile + conf.set(EXECUTOR_CORES, 1) + conf.set("spark.executor.resource.gpu.amount", "4") + conf.set("spark.task.resource.gpu.amount", "1") + error = intercept[SparkException] { + rpmanager = new ResourceProfileManager(conf, listenerBus) + }.getMessage() + + assert(error.contains( + "The configuration of resource: gpu (exec = 4, task = 1.0/1, runnable tasks = 4) will " + + "result in wasted resources due to resource cpus limiting the number of runnable " + + "tasks per executor to: 1. Please adjust your configuration.")) + + ResourceProfile.clearDefaultProfile + // multiple resources + conf.set("spark.executor.resource.fpga.amount", "6") + conf.set("spark.task.resource.fpga.amount", "1") + error = intercept[SparkException] { + rpmanager = new ResourceProfileManager(conf, listenerBus) + }.getMessage() + + assert(error.contains( + "The configuration of resource: fpga (exec = 6, task = 1.0/1, runnable tasks = 6) will " + + "result in wasted resources due to resource cpus limiting the number of runnable " + + "tasks per executor to: 1. Please adjust your configuration.")) + } + + test("default profile warn on wasted resources yarn") { + val conf = new SparkConf().setMaster("yarn") + conf.set(RESOURCES_WARNING_TESTING, true) + testWarningsClusterManagerWithCores(conf) + } + + test("default profile warn on wasted resources k8s") { + val conf = new SparkConf().setMaster("k8s://foo") + conf.set(RESOURCES_WARNING_TESTING, true) + testWarningsClusterManagerWithCores(conf) + } + + test("default profile warn on wasted resources standalone") { + val conf = new SparkConf().setMaster("spark://testing") + conf.set(RESOURCES_WARNING_TESTING, true) + // this should pass as default case with just 1 core + var rpmanager = new ResourceProfileManager(conf, listenerBus) + // cores only resource + warnOnWastedResources(rpmanager.defaultResourceProfile, conf, Some(4)) + + ResourceProfile.clearDefaultProfile + conf.set("spark.executor.resource.gpu.amount", "4") + conf.set("spark.task.resource.gpu.amount", "1") + // doesn't error because cores unknown + rpmanager = new ResourceProfileManager(conf, listenerBus) + + ResourceProfile.clearDefaultProfile + conf.set("spark.executor.resource.gpu.amount", "1") + conf.set("spark.task.resource.gpu.amount", "1") + rpmanager = new ResourceProfileManager(conf, listenerBus) + + var error = intercept[SparkException] { + warnOnWastedResources(rpmanager.defaultResourceProfile, conf, Some(4)) + }.getMessage() + + assert(error.contains( + "The configuration of cores (exec = 4 task = 1, runnable tasks = 4) will result " + + "in wasted resources due to resource gpu limiting the number of runnable tasks per " + + "executor to: 1. Please adjust your configuration.")) + + ResourceProfile.clearDefaultProfile + conf.set("spark.executor.resource.gpu.amount", "4") + conf.set("spark.task.resource.gpu.amount", "1") + rpmanager = new ResourceProfileManager(conf, listenerBus) + + error = intercept[SparkException] { + warnOnWastedResources(rpmanager.defaultResourceProfile, conf, Some(1)) + }.getMessage() + + assert(error.contains( + "The configuration of resource: gpu (exec = 4, task = 1.0/1, runnable tasks = 4) will " + + "result in wasted resources due to resource cpus limiting the number of runnable " + + "tasks per executor to: 1. Please adjust your configuration.")) + + ResourceProfile.clearDefaultProfile + conf.set("spark.executor.resource.gpu.amount", "4") + conf.set("spark.task.resource.gpu.amount", "1") + // specify cores should work + conf.set(EXECUTOR_CORES, 4) + rpmanager = new ResourceProfileManager(conf, listenerBus) + + ResourceProfile.clearDefaultProfile + conf.set("spark.executor.resource.gpu.amount", "2") + conf.set("spark.task.resource.gpu.amount", "1") + // specify cores that has extra + conf.set(EXECUTOR_CORES, 4) + + error = intercept[SparkException] { + rpmanager = new ResourceProfileManager(conf, listenerBus) + }.getMessage() + + assert(error.contains( + "The configuration of cores (exec = 4 task = 1, runnable tasks = 4) will result in wasted " + + "resources due to resource gpu limiting the number of runnable tasks per " + + "executor to: 2. Please adjust your configuration")) + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index c063301673598..afb6e29c8f8fb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark._ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Network.RPC_MESSAGE_MAX_SIZE import org.apache.spark.rdd.RDD -import org.apache.spark.resource.{ResourceInformation, ResourceProfile} +import org.apache.spark.resource.{ExecutorResourceRequests, ResourceInformation, ResourceProfile, TaskResourceRequests} import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} @@ -72,7 +72,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo // Ensure all executors have been launched. assert(sc.getExecutorIds().length == 4) } - assert(sc.maxNumConcurrentTasks() == 12) + assert(sc.maxNumConcurrentTasks(ResourceProfile.getOrCreateDefaultProfile(conf)) == 12) } test("compute max number of concurrent tasks can be launched when spark.task.cpus > 1") { @@ -86,7 +86,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo assert(sc.getExecutorIds().length == 4) } // Each executor can only launch one task since `spark.task.cpus` is 2. - assert(sc.maxNumConcurrentTasks() == 4) + assert(sc.maxNumConcurrentTasks(ResourceProfile.getOrCreateDefaultProfile(conf)) == 4) } test("compute max number of concurrent tasks can be launched when some executors are busy") { @@ -126,7 +126,8 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo assert(taskStarted.get()) assert(taskEnded.get() == false) // Assert we count in slots on both busy and free executors. - assert(sc.maxNumConcurrentTasks() == 4) + assert( + sc.maxNumConcurrentTasks(ResourceProfile.getOrCreateDefaultProfile(conf)) == 4) } } finally { sc.removeSparkListener(listener) @@ -189,8 +190,10 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo test("extra resources from executor") { import TestUtils._ + val execCores = 3 + val taskCpus = 1 val conf = new SparkConf() - .set(EXECUTOR_CORES, 1) + .set(EXECUTOR_CORES, execCores) .set(SCHEDULER_REVIVE_INTERVAL.key, "1m") // don't let it auto revive during test .set(EXECUTOR_INSTANCES, 0) // avoid errors about duplicate executor registrations .setMaster( @@ -200,6 +203,11 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo conf.set(EXECUTOR_GPU_ID.amountConf, "1") sc = new SparkContext(conf) + val execGpu = new ExecutorResourceRequests().cores(1).resource(GPU, 3) + val taskGpu = new TaskResourceRequests().cpus(1).resource(GPU, 1) + val rp = new ResourceProfile(execGpu.requests, taskGpu.requests) + sc.resourceProfileManager.addResourceProfile(rp) + assert(rp.id > ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val backend = sc.schedulerBackend.asInstanceOf[TestCoarseGrainedSchedulerBackend] val mockEndpointRef = mock[RpcEndpointRef] val mockAddress = mock[RpcAddress] @@ -224,7 +232,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) backend.driverEndpoint.askSync[Boolean]( RegisterExecutor("3", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources, - 5)) + rp.id)) val frameSize = RpcUtils.maxMessageSizeBytes(sc.conf) val bytebuffer = java.nio.ByteBuffer.allocate(frameSize - 100) @@ -234,11 +242,11 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo assert(execResources(GPU).availableAddrs.sorted === Array("0", "1", "3")) var exec3ResourceProfileId = backend.getExecutorResourceProfileId("3") - assert(exec3ResourceProfileId === 5) + assert(exec3ResourceProfileId === rp.id) val taskResources = Map(GPU -> new ResourceInformation(GPU, Array("0"))) var taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", - "t1", 0, 1, mutable.Map.empty[String, Long], mutable.Map.empty[String, Long], + "t1", 0, 1, taskCpus, mutable.Map.empty[String, Long], mutable.Map.empty[String, Long], new Properties(), taskResources, bytebuffer))) val ts = backend.getTaskSchedulerImpl() when(ts.resourceOffers(any[IndexedSeq[WorkerOffer]])).thenReturn(taskDescs) @@ -252,7 +260,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo } backend.driverEndpoint.send( - StatusUpdate("1", 1, TaskState.FINISHED, buffer, taskResources)) + StatusUpdate("1", 1, TaskState.FINISHED, buffer, taskCpus, taskResources)) eventually(timeout(5 seconds)) { execResources = backend.getExecutorAvailableResources("1") @@ -291,7 +299,6 @@ private class CSMockExternalClusterManager extends ExternalClusterManager { when(ts.applicationAttemptId()).thenReturn(Some("attempt1")) when(ts.schedulingMode).thenReturn(SchedulingMode.FIFO) when(ts.nodeBlacklist()).thenReturn(Set.empty[String]) - when(ts.resourcesReqsPerTask).thenReturn(Seq.empty) ts } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 101e60c73e9f8..a8cb14656e42a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -34,6 +34,8 @@ import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config import org.apache.spark.rdd.{DeterministicLevel, RDD} +import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfile, ResourceProfileBuilder, TaskResourceRequests} +import org.apache.spark.resource.ResourceUtils.{FPGA, GPU} import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.shuffle.{FetchFailedException, MetadataFetchFailedException} import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} @@ -3092,6 +3094,121 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assertDataStructuresEmpty() } + test("test default resource profiles") { + val rdd = sc.parallelize(1 to 10).map(x => (x, x)) + val rp = scheduler.mergeResourceProfilesForStage(rdd) + assert(rp.id == scheduler.sc.resourceProfileManager.defaultResourceProfile.id) + } + + test("test 1 resource profiles") { + val ereqs = new ExecutorResourceRequests().cores(4) + val treqs = new TaskResourceRequests().cpus(1) + val rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build + + val rdd = sc.parallelize(1 to 10).map(x => (x, x)).withResources(rp1) + val rpMerged = scheduler.mergeResourceProfilesForStage(rdd) + val expectedid = Option(rdd.getResourceProfile).map(_.id) + assert(expectedid.isDefined) + assert(expectedid.get != ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + assert(rpMerged.id == expectedid.get) + } + + test("test 2 resource profiles errors by default") { + import org.apache.spark.resource._ + val ereqs = new ExecutorResourceRequests().cores(4) + val treqs = new TaskResourceRequests().cpus(1) + val rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build + + val ereqs2 = new ExecutorResourceRequests().cores(2) + val treqs2 = new TaskResourceRequests().cpus(2) + val rp2 = new ResourceProfileBuilder().require(ereqs2).require(treqs2).build + + val rdd = sc.parallelize(1 to 10).withResources(rp1).map(x => (x, x)).withResources(rp2) + val error = intercept[IllegalArgumentException] { + scheduler.mergeResourceProfilesForStage(rdd) + }.getMessage() + + assert(error.contains("Multiple ResourceProfile's specified in the RDDs")) + } + + test("test 2 resource profile with merge conflict config true") { + afterEach() + val conf = new SparkConf() + conf.set(config.RESOURCE_PROFILE_MERGE_CONFLICTS.key, "true") + init(conf) + + val ereqs = new ExecutorResourceRequests().cores(4) + val treqs = new TaskResourceRequests().cpus(1) + val rp1 = new ResourceProfileBuilder().require(ereqs).require(treqs).build + + val ereqs2 = new ExecutorResourceRequests().cores(2) + val treqs2 = new TaskResourceRequests().cpus(2) + val rp2 = new ResourceProfileBuilder().require(ereqs2).require(treqs2).build + + val rdd = sc.parallelize(1 to 10).withResources(rp1).map(x => (x, x)).withResources(rp2) + val mergedRp = scheduler.mergeResourceProfilesForStage(rdd) + assert(mergedRp.getTaskCpus.get == 2) + assert(mergedRp.getExecutorCores.get == 4) + } + + test("test merge 2 resource profiles") { + val ereqs = new ExecutorResourceRequests().cores(4) + val treqs = new TaskResourceRequests().cpus(1) + val rp1 = new ResourceProfile(ereqs.requests, treqs.requests) + + val ereqs2 = new ExecutorResourceRequests().cores(2) + val treqs2 = new TaskResourceRequests().cpus(2) + val rp2 = new ResourceProfile(ereqs2.requests, treqs2.requests) + + var mergedRp = scheduler.mergeResourceProfiles(rp1, rp2) + + assert(mergedRp.getTaskCpus.get == 2) + assert(mergedRp.getExecutorCores.get == 4) + + val ereqs3 = new ExecutorResourceRequests().cores(1).resource(GPU, 1, "disc") + val treqs3 = new TaskResourceRequests().cpus(1).resource(GPU, 1) + val rp3 = new ResourceProfile(ereqs3.requests, treqs3.requests) + + val ereqs4 = new ExecutorResourceRequests().cores(2) + val treqs4 = new TaskResourceRequests().cpus(2) + val rp4 = new ResourceProfile(ereqs4.requests, treqs4.requests) + + mergedRp = scheduler.mergeResourceProfiles(rp3, rp4) + + assert(mergedRp.getTaskCpus.get == 2) + assert(mergedRp.getExecutorCores.get == 2) + assert(mergedRp.executorResources.size == 2) + assert(mergedRp.taskResources.size == 2) + assert(mergedRp.executorResources.get(GPU).get.amount == 1) + assert(mergedRp.executorResources.get(GPU).get.discoveryScript == "disc") + assert(mergedRp.taskResources.get(GPU).get.amount == 1) + + val ereqs5 = new ExecutorResourceRequests().cores(1).memory("3g") + .memoryOverhead("1g").pysparkMemory("2g").resource(GPU, 1, "disc") + val treqs5 = new TaskResourceRequests().cpus(1).resource(GPU, 1) + val rp5 = new ResourceProfile(ereqs5.requests, treqs5.requests) + + val ereqs6 = new ExecutorResourceRequests().cores(8).resource(FPGA, 2, "fdisc") + val treqs6 = new TaskResourceRequests().cpus(2).resource(FPGA, 1) + val rp6 = new ResourceProfile(ereqs6.requests, treqs6.requests) + + mergedRp = scheduler.mergeResourceProfiles(rp5, rp6) + + assert(mergedRp.getTaskCpus.get == 2) + assert(mergedRp.getExecutorCores.get == 8) + assert(mergedRp.executorResources.size == 6) + assert(mergedRp.taskResources.size == 3) + assert(mergedRp.executorResources.get(GPU).get.amount == 1) + assert(mergedRp.executorResources.get(GPU).get.discoveryScript == "disc") + assert(mergedRp.taskResources.get(GPU).get.amount == 1) + assert(mergedRp.executorResources.get(FPGA).get.amount == 2) + assert(mergedRp.executorResources.get(FPGA).get.discoveryScript == "fdisc") + assert(mergedRp.taskResources.get(FPGA).get.amount == 1) + assert(mergedRp.executorResources.get(ResourceProfile.MEMORY).get.amount == 3072) + assert(mergedRp.executorResources.get(ResourceProfile.PYSPARK_MEM).get.amount == 2048) + assert(mergedRp.executorResources.get(ResourceProfile.OVERHEAD_MEM).get.amount == 1024) + } + /** * Assert that the supplied TaskSet has exactly the given hosts as its preferred locations. * Note that this checks only the host and not the executor ID. diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 286924001e920..61ea21fa86c5a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -38,6 +38,7 @@ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.io._ import org.apache.spark.metrics.{ExecutorMetricType, MetricsSystem} +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.{JsonProtocol, Utils} @@ -438,12 +439,14 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit private def createStageSubmittedEvent(stageId: Int) = { SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0, - Seq.empty, Seq.empty, "details")) + Seq.empty, Seq.empty, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) } private def createStageCompletedEvent(stageId: Int) = { SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0, - Seq.empty, Seq.empty, "details")) + Seq.empty, Seq.empty, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) } private def createExecutorAddedEvent(executorId: Int) = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala index 4e71ec1ea7b37..586e69b63355d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -21,6 +21,7 @@ import scala.collection.mutable.Map import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.executor.ExecutorMetrics +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AccumulatorV2 @@ -71,7 +72,7 @@ private class DummySchedulerBackend extends SchedulerBackend { def stop(): Unit = {} def reviveOffers(): Unit = {} def defaultParallelism(): Int = 1 - def maxNumConcurrentTasks(): Int = 0 + def maxNumConcurrentTasks(rp: ResourceProfile): Int = 0 } private class DummyTaskScheduler extends TaskScheduler { diff --git a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala index 8cb6268f85d36..9ec088aaddddd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala @@ -21,6 +21,7 @@ import java.util.Properties import org.apache.spark.{Partition, SparkEnv, TaskContext} import org.apache.spark.executor.TaskMetrics +import org.apache.spark.resource.ResourceProfile class FakeTask( stageId: Int, @@ -42,7 +43,12 @@ object FakeTask { * locations for each task (given as varargs) if this sequence is not empty. */ def createTaskSet(numTasks: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { - createTaskSet(numTasks, stageId = 0, stageAttemptId = 0, priority = 0, prefLocs: _*) + createTaskSet(numTasks, stageId = 0, stageAttemptId = 0, priority = 0, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID, prefLocs: _*) + } + + def createTaskSet(numTasks: Int, rpId: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { + createTaskSet(numTasks, stageId = 0, stageAttemptId = 0, priority = 0, rpId, prefLocs: _*) } def createTaskSet( @@ -50,7 +56,8 @@ object FakeTask { stageId: Int, stageAttemptId: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { - createTaskSet(numTasks, stageId, stageAttemptId, priority = 0, prefLocs: _*) + createTaskSet(numTasks, stageId, stageAttemptId, priority = 0, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID, prefLocs: _*) } def createTaskSet( @@ -58,6 +65,7 @@ object FakeTask { stageId: Int, stageAttemptId: Int, priority: Int, + rpId: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { if (prefLocs.size != 0 && prefLocs.size != numTasks) { throw new IllegalArgumentException("Wrong number of task locations") @@ -65,7 +73,7 @@ object FakeTask { val tasks = Array.tabulate[Task[_]](numTasks) { i => new FakeTask(stageId, i, if (prefLocs.size != 0) prefLocs(i) else Nil) } - new TaskSet(tasks, stageId, stageAttemptId, priority = priority, null) + new TaskSet(tasks, stageId, stageAttemptId, priority = priority, null, rpId) } def createShuffleMapTaskSet( @@ -91,11 +99,21 @@ object FakeTask { }, prefLocs(i), new Properties, SparkEnv.get.closureSerializer.newInstance().serialize(TaskMetrics.registered).array()) } - new TaskSet(tasks, stageId, stageAttemptId, priority = priority, null) + new TaskSet(tasks, stageId, stageAttemptId, priority = priority, null, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) } def createBarrierTaskSet(numTasks: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { - createBarrierTaskSet(numTasks, stageId = 0, stageAttemptId = 0, priority = 0, prefLocs: _*) + createBarrierTaskSet(numTasks, stageId = 0, stageAttemptId = 0, priority = 0, + rpId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID, prefLocs: _*) + } + + def createBarrierTaskSet( + numTasks: Int, + rpId: Int, + prefLocs: Seq[TaskLocation]*): TaskSet = { + createBarrierTaskSet(numTasks, stageId = 0, stageAttemptId = 0, priority = 0, + rpId = rpId, prefLocs: _*) } def createBarrierTaskSet( @@ -103,6 +121,7 @@ object FakeTask { stageId: Int, stageAttemptId: Int, priority: Int, + rpId: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { if (prefLocs.size != 0 && prefLocs.size != numTasks) { throw new IllegalArgumentException("Wrong number of task locations") @@ -110,6 +129,6 @@ object FakeTask { val tasks = Array.tabulate[Task[_]](numTasks) { i => new FakeTask(stageId, i, if (prefLocs.size != 0) prefLocs(i) else Nil, isBarrier = true) } - new TaskSet(tasks, stageId, stageAttemptId, priority = priority, null) + new TaskSet(tasks, stageId, stageAttemptId, priority = priority, null, rpId) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala index b953add9d58cb..d9de976c789d4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala @@ -22,6 +22,7 @@ import java.util.Properties import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.internal.config.SCHEDULER_ALLOCATION_FILE +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler.SchedulingMode._ /** @@ -39,7 +40,8 @@ class PoolSuite extends SparkFunSuite with LocalSparkContext { val tasks = Array.tabulate[Task[_]](numTasks) { i => new FakeTask(stageId, i, Nil) } - new TaskSetManager(taskScheduler, new TaskSet(tasks, stageId, 0, 0, null), 0) + new TaskSetManager(taskScheduler, new TaskSet(tasks, stageId, 0, 0, null, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID), 0) } def scheduleTaskAndVerifyId(taskId: Int, rootPool: Pool, expectedStageId: Int): Unit = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index dff8975a4fe49..e6d06ad6efd16 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -36,6 +36,7 @@ import org.apache.spark.TaskState._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.SCHEDULER_REVIVE_INTERVAL import org.apache.spark.rdd.RDD +import org.apache.spark.resource.ResourceProfile import org.apache.spark.util.{CallSite, ThreadUtils, Utils} /** @@ -336,8 +337,8 @@ private[spark] abstract class MockBackend( if (TaskState.isFinished(state)) { synchronized { runningTasks -= task.taskId - executorIdToExecutor(task.executorId).freeCores += taskScheduler.CPUS_PER_TASK - freeCores += taskScheduler.CPUS_PER_TASK + executorIdToExecutor(task.executorId).freeCores += task.cpus + freeCores += task.cpus } reviveOffers() } @@ -385,7 +386,7 @@ private[spark] abstract class MockBackend( }.toIndexedSeq } - override def maxNumConcurrentTasks(): Int = 0 + override def maxNumConcurrentTasks(rp: ResourceProfile): Int = 0 /** * This is called by the scheduler whenever it has tasks it would like to schedule, when a tasks @@ -406,9 +407,9 @@ private[spark] abstract class MockBackend( (taskDescription, task) } newTasks.foreach { case (taskDescription, _) => - executorIdToExecutor(taskDescription.executorId).freeCores -= taskScheduler.CPUS_PER_TASK + freeCores -= taskDescription.cpus + executorIdToExecutor(taskDescription.executorId).freeCores -= taskDescription.cpus } - freeCores -= newTasks.size * taskScheduler.CPUS_PER_TASK assignedTasksWaitingToRun ++= newTasks } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala index 5839532f11666..a4a3ff338060b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala @@ -68,6 +68,7 @@ class TaskDescriptionSuite extends SparkFunSuite { name = "task for test", index = 19, partitionId = 1, + cpus = 1, originalFiles, originalJars, originalProperties, diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index e7ecf847ff4f4..4c3931677df9d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -31,6 +31,7 @@ import org.scalatestplus.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config +import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfile, TaskResourceRequests} import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.util.ManualClock @@ -40,7 +41,7 @@ class FakeSchedulerBackend extends SchedulerBackend { def stop(): Unit = {} def reviveOffers(): Unit = {} def defaultParallelism(): Int = 1 - def maxNumConcurrentTasks(): Int = 0 + def maxNumConcurrentTasks(rp: ResourceProfile): Int = 0 } class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfterEach @@ -202,7 +203,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B config.CPUS_PER_TASK.key -> taskCpus.toString) val numFreeCores = 1 val taskSet = new TaskSet( - Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) + Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), + 0, 0, 0, null, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val multiCoreWorkerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", taskCpus), new WorkerOffer("executor1", "host1", numFreeCores)) taskScheduler.submitTasks(taskSet) @@ -216,7 +218,8 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // still be processed without error taskScheduler.submitTasks(FakeTask.createTaskSet(1)) val taskSet2 = new TaskSet( - Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 1, 0, 0, null) + Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), + 1, 0, 0, null, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) taskScheduler.submitTasks(taskSet2) taskDescriptions = taskScheduler.resourceOffers(multiCoreWorkerOffers).flatten assert(taskDescriptions.map(_.executorId) === Seq("executor0")) @@ -1135,6 +1138,99 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(0 === taskDescriptions.length) } + test("don't schedule for a barrier taskSet if available slots are less than " + + "pending tasks gpus limiting") { + val taskCpus = 1 + val taskScheduler = setupSchedulerWithMaster( + s"local[$taskCpus]", config.CPUS_PER_TASK.key -> taskCpus.toString, + "spark.executor.resource.gpu.amount" -> "1", "spark.task.resource.gpu.amount" -> "1") + + val numFreeCores = 3 + val workerOffers = IndexedSeq( + new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625"), + Map("gpu" -> Seq("0").toBuffer)), + new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627"), + Map("gpu" -> Seq("0").toBuffer))) + val attempt1 = FakeTask.createBarrierTaskSet(3) + + taskScheduler.submitTasks(attempt1) + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(0 === taskDescriptions.length) + } + + test("schedule tasks for a barrier taskSet if all tasks can be launched together gpus") { + val taskCpus = 1 + val taskScheduler = setupSchedulerWithMaster( + s"local[$taskCpus]", config.CPUS_PER_TASK.key -> taskCpus.toString, + "spark.executor.resource.gpu.amount" -> "1", "spark.task.resource.gpu.amount" -> "1") + + val numFreeCores = 3 + val workerOffers = IndexedSeq( + new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625"), + Map("gpu" -> Seq("0").toBuffer)), + new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627"), + Map("gpu" -> Seq("0").toBuffer)), + new WorkerOffer("executor2", "host2", numFreeCores, Some("192.168.0.101:49629"), + Map("gpu" -> Seq("0").toBuffer))) + val attempt1 = FakeTask.createBarrierTaskSet(3) + + taskScheduler.submitTasks(attempt1) + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(3 === taskDescriptions.length) + } + + + // barrier scheduling doesn't yet work with dynamic allocation but test it with another + // ResourceProfile anyway to make sure code path works when it is supported + test("schedule tasks for a barrier taskSet if all tasks can be launched together " + + "diff ResourceProfile") { + val taskCpus = 1 + val taskScheduler = setupSchedulerWithMaster( + s"local[$taskCpus]", config.CPUS_PER_TASK.key -> taskCpus.toString) + val execReqs = new ExecutorResourceRequests().cores(2).resource("gpu", 2) + val taskReqs = new TaskResourceRequests().cpus(1).resource("gpu", 1) + val rp = new ResourceProfile(execReqs.requests, taskReqs.requests) + taskScheduler.sc.resourceProfileManager.addResourceProfile(rp) + + val numFreeCores = 2 + val workerOffers = IndexedSeq( + new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625"), + Map("gpu" -> Seq("0", "1").toBuffer), rp.id), + new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627"), + Map("gpu" -> Seq("0", "1").toBuffer), rp.id)) + val attempt1 = FakeTask.createBarrierTaskSet(3, rpId = rp.id) + + taskScheduler.submitTasks(attempt1) + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(3 === taskDescriptions.length) + } + + + test("schedule tasks for a barrier taskSet if all tasks can be launched together " + + "diff ResourceProfile, but not enough gpus") { + val taskCpus = 1 + val taskScheduler = setupSchedulerWithMaster( + s"local[$taskCpus]", config.CPUS_PER_TASK.key -> taskCpus.toString) + val execReqs = new ExecutorResourceRequests().cores(2).resource("gpu", 2) + val taskReqs = new TaskResourceRequests().cpus(1).resource("gpu", 1) + val rp = new ResourceProfile(execReqs.requests, taskReqs.requests) + taskScheduler.sc.resourceProfileManager.addResourceProfile(rp) + + val numFreeCores = 2 + // make each of the worker offers only have 1 GPU, thus making it not enough + val workerOffers = IndexedSeq( + new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625"), + Map("gpu" -> Seq("0").toBuffer), rp.id), + new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627"), + Map("gpu" -> Seq("0").toBuffer), rp.id)) + val attempt1 = FakeTask.createBarrierTaskSet(3, rpId = rp.id) + + taskScheduler.submitTasks(attempt1) + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(0 === taskDescriptions.length) + } + + test("schedule tasks for a barrier taskSet if all tasks can be launched together") { val taskCpus = 2 val taskScheduler = setupSchedulerWithMaster( @@ -1165,8 +1261,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B new WorkerOffer("executor0", "host0", numFreeCores, Some("192.168.0.101:49625")), new WorkerOffer("executor1", "host1", numFreeCores, Some("192.168.0.101:49627")), new WorkerOffer("executor2", "host2", numFreeCores, Some("192.168.0.101:49629"))) - val barrier = FakeTask.createBarrierTaskSet(3, stageId = 0, stageAttemptId = 0, priority = 1) - val highPrio = FakeTask.createTaskSet(1, stageId = 1, stageAttemptId = 0, priority = 0) + val barrier = FakeTask.createBarrierTaskSet(3, stageId = 0, stageAttemptId = 0, priority = 1, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val highPrio = FakeTask.createTaskSet(1, stageId = 1, stageAttemptId = 0, priority = 0, + rpId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) // submit highPrio and barrier taskSet taskScheduler.submitTasks(highPrio) @@ -1289,6 +1387,68 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B assert(ArrayBuffer("1") === taskDescriptions(1).resources.get(GPU).get.addresses) } + test("Scheduler works with multiple ResourceProfiles and gpus") { + val taskCpus = 1 + val taskGpus = 1 + val executorGpus = 4 + val executorCpus = 4 + + val taskScheduler = setupScheduler(numCores = executorCpus, + config.CPUS_PER_TASK.key -> taskCpus.toString, + TASK_GPU_ID.amountConf -> taskGpus.toString, + EXECUTOR_GPU_ID.amountConf -> executorGpus.toString, + config.EXECUTOR_CORES.key -> executorCpus.toString) + + val ereqs = new ExecutorResourceRequests().cores(6).resource(GPU, 6) + val treqs = new TaskResourceRequests().cpus(2).resource(GPU, 2) + val rp = new ResourceProfile(ereqs.requests, treqs.requests) + taskScheduler.sc.resourceProfileManager.addResourceProfile(rp) + val taskSet = FakeTask.createTaskSet(3) + val rpTaskSet = FakeTask.createTaskSet(5, stageId = 1, stageAttemptId = 0, + priority = 0, rpId = rp.id) + + val resourcesDefaultProf = Map(GPU -> ArrayBuffer("0", "1", "2", "3")) + val resources = Map(GPU -> ArrayBuffer("4", "5", "6", "7", "8", "9")) + + val workerOffers = + IndexedSeq(new WorkerOffer("executor0", "host0", 2, None, resourcesDefaultProf), + new WorkerOffer("executor1", "host1", 6, None, resources, rp.id)) + taskScheduler.submitTasks(taskSet) + taskScheduler.submitTasks(rpTaskSet) + // should have 2 for default profile and 2 for additional resource profile + var taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(5 === taskDescriptions.length) + var has2Gpus = 0 + var has1Gpu = 0 + for (tDesc <- taskDescriptions) { + assert(tDesc.resources.contains(GPU)) + if (tDesc.resources(GPU).addresses.size == 2) { + has2Gpus += 1 + assert(2 === tDesc.cpus) + } + if (tDesc.resources(GPU).addresses.size == 1) { + has1Gpu += 1 + assert(1 === tDesc.cpus) + } + } + assert(has2Gpus == 3) + assert(has1Gpu == 2) + + val resources3 = Map(GPU -> ArrayBuffer("14", "15", "16", "17", "18", "19")) + + // clear the first 2 worker offers so they don't have any room and add a third + // for the resource profile + val workerOffers3 = IndexedSeq( + new WorkerOffer("executor0", "host0", 0, None, Map.empty), + new WorkerOffer("executor1", "host1", 0, None, Map.empty, rp.id), + new WorkerOffer("executor2", "host2", 6, None, resources3, rp.id)) + taskDescriptions = taskScheduler.resourceOffers(workerOffers3).flatten + assert(2 === taskDescriptions.length) + assert(taskDescriptions.head.resources.contains(GPU)) + assert(2 == taskDescriptions.head.resources(GPU).addresses.size) + assert(2 === taskDescriptions.head.cpus) + } + /** * Used by tests to simulate a task failure. This calls the failure handler explicitly, to ensure * that all the state is updated when this method returns. Otherwise, there's no way to know when diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index b740e357903a2..8b9f46e40b98b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -34,6 +34,8 @@ import org.scalatest.concurrent.Eventually import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config +import org.apache.spark.internal.config.Tests.TASKSET_MANAGER_SPECULATION_TESTING +import org.apache.spark.resource.{ResourceInformation, ResourceProfile} import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend @@ -213,7 +215,6 @@ class TaskSetManagerSuite super.afterEach() } - test("TaskSet with no preferences") { sc = new SparkContext("local", "test") sched = new FakeTaskScheduler(sc, ("exec1", "host1")) @@ -224,7 +225,7 @@ class TaskSetManagerSuite // Offer a host with NO_PREF as the constraint, // we should get a nopref task immediately since that's what we only have - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, 1) assert(taskOption.isDefined) clock.advance(1) @@ -245,7 +246,7 @@ class TaskSetManagerSuite // First three offers should all find tasks for (i <- 0 until 3) { - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, 1) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === "exec1") @@ -253,7 +254,7 @@ class TaskSetManagerSuite assert(sched.startedTasks.toSet === Set(0, 1, 2)) // Re-offer the host -- now we should get no more tasks - assert(manager.resourceOffer("exec1", "host1", NO_PREF) === None) + assert(manager.resourceOffer("exec1", "host1", NO_PREF, 1) === None) // Finish the first two tasks manager.handleSuccessfulTask(0, createTaskResult(0, accumUpdatesByTask(0))) @@ -276,12 +277,12 @@ class TaskSetManagerSuite val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // An executor that is not NODE_LOCAL should be rejected. - assert(manager.resourceOffer("execC", "host2", ANY) === None) + assert(manager.resourceOffer("execC", "host2", ANY, 1) === None) // Because there are no alive PROCESS_LOCAL executors, the base locality level should be // NODE_LOCAL. So, we should schedule the task on this offered NODE_LOCAL executor before // any of the locality wait timers expire. - assert(manager.resourceOffer("execA", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("execA", "host1", ANY, 1).get.index === 0) } test("basic delay scheduling") { @@ -296,22 +297,22 @@ class TaskSetManagerSuite val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1, exec1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) == None) + assert(manager.resourceOffer("exec1", "host1", ANY, 1).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, 1) == None) clock.advance(LOCALITY_WAIT_MS) // Offer host1, exec1 again, at NODE_LOCAL level: the node local (task 3) should // get chosen before the noPref task - assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL).get.index == 2) + assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL, 1).get.index == 2) // Offer host2, exec2, at NODE_LOCAL level: we should choose task 2 - assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL).get.index == 1) + assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL, 1).get.index == 1) // Offer host2, exec2 again, at NODE_LOCAL level: we should get noPref task // after failing to find a node_Local task - assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL) == None) + assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL, 1) == None) clock.advance(LOCALITY_WAIT_MS) - assert(manager.resourceOffer("exec2", "host2", NO_PREF).get.index == 3) + assert(manager.resourceOffer("exec2", "host2", NO_PREF, 1).get.index == 3) } test("we do not need to delay scheduling when we only have noPref tasks in the queue") { @@ -325,10 +326,10 @@ class TaskSetManagerSuite val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1, exec1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).get.index === 0) - assert(manager.resourceOffer("exec3", "host2", PROCESS_LOCAL).get.index === 1) - assert(manager.resourceOffer("exec3", "host2", NODE_LOCAL) == None) - assert(manager.resourceOffer("exec3", "host2", NO_PREF).get.index === 2) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, 1).get.index === 0) + assert(manager.resourceOffer("exec3", "host2", PROCESS_LOCAL, 1).get.index === 1) + assert(manager.resourceOffer("exec3", "host2", NODE_LOCAL, 1) == None) + assert(manager.resourceOffer("exec3", "host2", NO_PREF, 1).get.index === 2) } test("delay scheduling with fallback") { @@ -346,29 +347,29 @@ class TaskSetManagerSuite val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY, 1).get.index === 0) // Offer host1 again: nothing should get chosen - assert(manager.resourceOffer("exec1", "host1", ANY) === None) + assert(manager.resourceOffer("exec1", "host1", ANY, 1) === None) clock.advance(LOCALITY_WAIT_MS) // Offer host1 again: second task (on host2) should get chosen - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) + assert(manager.resourceOffer("exec1", "host1", ANY, 1).get.index === 1) // Offer host1 again: third task (on host2) should get chosen - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 2) + assert(manager.resourceOffer("exec1", "host1", ANY, 1).get.index === 2) // Offer host2: fifth task (also on host2) should get chosen - assert(manager.resourceOffer("exec2", "host2", ANY).get.index === 4) + assert(manager.resourceOffer("exec2", "host2", ANY, 1).get.index === 4) // Now that we've launched a local task, we should no longer launch the task for host3 - assert(manager.resourceOffer("exec2", "host2", ANY) === None) + assert(manager.resourceOffer("exec2", "host2", ANY, 1) === None) clock.advance(LOCALITY_WAIT_MS) // After another delay, we can go ahead and launch that task non-locally - assert(manager.resourceOffer("exec2", "host2", ANY).get.index === 3) + assert(manager.resourceOffer("exec2", "host2", ANY, 1).get.index === 3) } test("delay scheduling with failed hosts") { @@ -384,28 +385,28 @@ class TaskSetManagerSuite val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // First offer host1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY, 1).get.index === 0) // After this, nothing should get chosen, because we have separated tasks with unavailable // preference from the noPrefPendingTasks - assert(manager.resourceOffer("exec1", "host1", ANY) === None) + assert(manager.resourceOffer("exec1", "host1", ANY, 1) === None) // Now mark host2 as dead sched.removeExecutor("exec2") manager.executorLost("exec2", "host2", SlaveLost()) // nothing should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY) === None) + assert(manager.resourceOffer("exec1", "host1", ANY, 1) === None) clock.advance(LOCALITY_WAIT_MS * 2) // task 1 and 2 would be scheduled as nonLocal task - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 2) + assert(manager.resourceOffer("exec1", "host1", ANY, 1).get.index === 1) + assert(manager.resourceOffer("exec1", "host1", ANY, 1).get.index === 2) // all finished - assert(manager.resourceOffer("exec1", "host1", ANY) === None) - assert(manager.resourceOffer("exec2", "host2", ANY) === None) + assert(manager.resourceOffer("exec1", "host1", ANY, 1) === None) + assert(manager.resourceOffer("exec2", "host2", ANY, 1) === None) } test("task result lost") { @@ -416,14 +417,14 @@ class TaskSetManagerSuite clock.advance(1) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY, 1).get.index === 0) // Tell it the task has finished but the result was lost. manager.handleFailedTask(0, TaskState.FINISHED, TaskResultLost) assert(sched.endedTasks(0) === TaskResultLost) // Re-offer the host -- now we should get task 0 again. - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY, 1).get.index === 0) } test("repeated failures lead to task set abortion") { @@ -437,7 +438,7 @@ class TaskSetManagerSuite // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted // after the last failure. (1 to manager.maxTaskFailures).foreach { index => - val offerResult = manager.resourceOffer("exec1", "host1", ANY) + val offerResult = manager.resourceOffer("exec1", "host1", ANY, 1) assert(offerResult.isDefined, "Expect resource offer on iteration %s to return a task".format(index)) assert(offerResult.get.index === 0) @@ -473,7 +474,7 @@ class TaskSetManagerSuite val manager = new TaskSetManager(sched, taskSet, 4, blacklistTrackerOpt, clock) { - val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) + val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, 1) assert(offerResult.isDefined, "Expect resource offer to return a task") assert(offerResult.get.index === 0) @@ -484,15 +485,15 @@ class TaskSetManagerSuite assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec1 fails after failure 1 due to blacklist - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).isEmpty) - assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL).isEmpty) - assert(manager.resourceOffer("exec1", "host1", RACK_LOCAL).isEmpty) - assert(manager.resourceOffer("exec1", "host1", ANY).isEmpty) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, 1).isEmpty) + assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL, 1).isEmpty) + assert(manager.resourceOffer("exec1", "host1", RACK_LOCAL, 1).isEmpty) + assert(manager.resourceOffer("exec1", "host1", ANY, 1).isEmpty) } // Run the task on exec1.1 - should work, and then fail it on exec1.1 { - val offerResult = manager.resourceOffer("exec1.1", "host1", NODE_LOCAL) + val offerResult = manager.resourceOffer("exec1.1", "host1", NODE_LOCAL, 1) assert(offerResult.isDefined, "Expect resource offer to return a task for exec1.1, offerResult = " + offerResult) @@ -504,12 +505,12 @@ class TaskSetManagerSuite assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec1.1 fails after failure 2 due to blacklist - assert(manager.resourceOffer("exec1.1", "host1", NODE_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1.1", "host1", NODE_LOCAL, 1).isEmpty) } // Run the task on exec2 - should work, and then fail it on exec2 { - val offerResult = manager.resourceOffer("exec2", "host2", ANY) + val offerResult = manager.resourceOffer("exec2", "host2", ANY, 1) assert(offerResult.isDefined, "Expect resource offer to return a task") assert(offerResult.get.index === 0) @@ -520,7 +521,7 @@ class TaskSetManagerSuite assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec2 fails after failure 3 due to blacklist - assert(manager.resourceOffer("exec2", "host2", ANY).isEmpty) + assert(manager.resourceOffer("exec2", "host2", ANY, 1).isEmpty) } // Despite advancing beyond the time for expiring executors from within the blacklist, @@ -528,17 +529,17 @@ class TaskSetManagerSuite clock.advance(rescheduleDelay) { - val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) + val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL, 1) assert(offerResult.isEmpty) } { - val offerResult = manager.resourceOffer("exec3", "host3", ANY) + val offerResult = manager.resourceOffer("exec3", "host3", ANY, 1) assert(offerResult.isDefined) assert(offerResult.get.index === 0) assert(offerResult.get.executorId === "exec3") - assert(manager.resourceOffer("exec3", "host3", ANY).isEmpty) + assert(manager.resourceOffer("exec3", "host3", ANY, 1).isEmpty) // Cause exec3 to fail : failure 4 manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost) @@ -597,14 +598,14 @@ class TaskSetManagerSuite manager.executorAdded() sched.addExecutor("execC", "host2") manager.executorAdded() - assert(manager.resourceOffer("exec1", "host1", ANY).isDefined) + assert(manager.resourceOffer("exec1", "host1", ANY, 1).isDefined) sched.removeExecutor("execA") manager.executorLost( "execA", "host1", ExecutorExited(143, false, "Terminated for reason unrelated to running tasks")) assert(!sched.taskSetsFailed.contains(taskSet.id)) - assert(manager.resourceOffer("execC", "host2", ANY).isDefined) + assert(manager.resourceOffer("execC", "host2", ANY, 1).isDefined) sched.removeExecutor("execC") manager.executorLost( "execC", "host2", ExecutorExited(1, true, "Terminated due to issue with running tasks")) @@ -632,12 +633,12 @@ class TaskSetManagerSuite clock.advance(LOCALITY_WAIT_MS * 3) // Offer host3 // No task is scheduled if we restrict locality to RACK_LOCAL - assert(manager.resourceOffer("execC", "host3", RACK_LOCAL) === None) + assert(manager.resourceOffer("execC", "host3", RACK_LOCAL, 1) === None) // Task 0 can be scheduled with ANY - assert(manager.resourceOffer("execC", "host3", ANY).get.index === 0) + assert(manager.resourceOffer("execC", "host3", ANY, 1).get.index === 0) // Offer host2 // Task 1 can be scheduled with RACK_LOCAL - assert(manager.resourceOffer("execB", "host2", RACK_LOCAL).get.index === 1) + assert(manager.resourceOffer("execB", "host2", RACK_LOCAL, 1).get.index === 1) } test("do not emit warning when serialized task is small") { @@ -648,7 +649,7 @@ class TaskSetManagerSuite assert(!manager.emittedTaskSizeWarning) - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY, 1).get.index === 0) assert(!manager.emittedTaskSizeWarning) } @@ -657,12 +658,13 @@ class TaskSetManagerSuite sc = new SparkContext("local", "test") sched = new FakeTaskScheduler(sc, ("exec1", "host1")) - val taskSet = new TaskSet(Array(new LargeTask(0)), 0, 0, 0, null) + val taskSet = new TaskSet(Array(new LargeTask(0)), 0, 0, 0, + null, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) assert(!manager.emittedTaskSizeWarning) - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY, 1).get.index === 0) assert(manager.emittedTaskSizeWarning) } @@ -672,11 +674,12 @@ class TaskSetManagerSuite sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = new TaskSet( - Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) + Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), + 0, 0, 0, null, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) intercept[TaskNotSerializableException] { - manager.resourceOffer("exec1", "host1", ANY) + manager.resourceOffer("exec1", "host1", ANY, 1) } assert(manager.isZombie) } @@ -743,18 +746,19 @@ class TaskSetManagerSuite val singleTask = new ShuffleMapTask(0, 0, null, new Partition { override def index: Int = 0 }, Seq(TaskLocation("host1", "execA")), new Properties, null) - val taskSet = new TaskSet(Array(singleTask), 0, 0, 0, null) + val taskSet = new TaskSet(Array(singleTask), 0, 0, 0, + null, ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) // Offer host1, which should be accepted as a PROCESS_LOCAL location // by the one task in the task set - val task1 = manager.resourceOffer("execA", "host1", TaskLocality.PROCESS_LOCAL).get + val task1 = manager.resourceOffer("execA", "host1", TaskLocality.PROCESS_LOCAL, 1).get // Mark the task as available for speculation, and then offer another resource, // which should be used to launch a speculative copy of the task. manager.speculatableTasks += singleTask.partitionId manager.addPendingTask(singleTask.partitionId, speculatable = true) - val task2 = manager.resourceOffer("execB", "host2", TaskLocality.ANY).get + val task2 = manager.resourceOffer("execB", "host2", TaskLocality.ANY, 1).get assert(manager.runningTasks === 2) assert(manager.isZombie === false) @@ -836,11 +840,11 @@ class TaskSetManagerSuite } // Offer resources for 4 tasks to start for ((exec, host) <- Seq( - "exec1" -> "host1", - "exec1" -> "host1", - "exec3" -> "host3", - "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(exec, host, NO_PREF) + "exec1" -> "host1", + "exec1" -> "host1", + "exec3" -> "host3", + "exec2" -> "host2")) { + val taskOption = manager.resourceOffer(exec, host, NO_PREF, 1) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === exec) @@ -866,7 +870,7 @@ class TaskSetManagerSuite assert(sched.speculativeTasks.toSet === Set(2, 3)) // Offer resource to start the speculative attempt for the running task 2.0 - val taskOption = manager.resourceOffer("exec2", "host2", ANY) + val taskOption = manager.resourceOffer("exec2", "host2", ANY, 1) assert(taskOption.isDefined) val task4 = taskOption.get assert(task4.index === 2) @@ -895,20 +899,20 @@ class TaskSetManagerSuite val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) - assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 0) - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) - assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index == 1) + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL, 1).get.index === 0) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, 1) == None) + assert(manager.resourceOffer("execA", "host1", NO_PREF, 1).get.index == 1) manager.speculatableTasks += 1 manager.addPendingTask(1, speculatable = true) clock.advance(LOCALITY_WAIT_MS) // schedule the nonPref task - assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index === 2) + assert(manager.resourceOffer("execA", "host1", NO_PREF, 1).get.index === 2) // schedule the speculative task - assert(manager.resourceOffer("execB", "host2", NO_PREF).get.index === 1) + assert(manager.resourceOffer("execB", "host2", NO_PREF, 1).get.index === 1) clock.advance(LOCALITY_WAIT_MS * 3) // schedule non-local tasks - assert(manager.resourceOffer("execB", "host2", ANY).get.index === 3) + assert(manager.resourceOffer("execB", "host2", ANY, 1).get.index === 3) } test("node-local tasks should be scheduled right away " + @@ -925,13 +929,13 @@ class TaskSetManagerSuite val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // node-local tasks are scheduled without delay - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 0) - assert(manager.resourceOffer("execA", "host2", NODE_LOCAL).get.index === 1) - assert(manager.resourceOffer("execA", "host3", NODE_LOCAL).get.index === 3) - assert(manager.resourceOffer("execA", "host3", NODE_LOCAL) === None) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, 1).get.index === 0) + assert(manager.resourceOffer("execA", "host2", NODE_LOCAL, 1).get.index === 1) + assert(manager.resourceOffer("execA", "host3", NODE_LOCAL, 1).get.index === 3) + assert(manager.resourceOffer("execA", "host3", NODE_LOCAL, 1) === None) // schedule no-preference after node local ones - assert(manager.resourceOffer("execA", "host3", NO_PREF).get.index === 2) + assert(manager.resourceOffer("execA", "host3", NO_PREF, 1).get.index === 2) } test("SPARK-4939: node-local tasks should be scheduled right after process-local tasks finished") @@ -947,13 +951,13 @@ class TaskSetManagerSuite val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // process-local tasks are scheduled first - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 2) - assert(manager.resourceOffer("execB", "host2", NODE_LOCAL).get.index === 3) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, 1).get.index === 2) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL, 1).get.index === 3) // node-local tasks are scheduled without delay - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 0) - assert(manager.resourceOffer("execB", "host2", NODE_LOCAL).get.index === 1) - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) - assert(manager.resourceOffer("execB", "host2", NODE_LOCAL) == None) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, 1).get.index === 0) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL, 1).get.index === 1) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, 1) == None) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL, 1) == None) } test("SPARK-4939: no-pref tasks should be scheduled after process-local tasks finished") { @@ -967,13 +971,13 @@ class TaskSetManagerSuite val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) // process-local tasks are scheduled first - assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 1) - assert(manager.resourceOffer("execB", "host2", PROCESS_LOCAL).get.index === 2) + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL, 1).get.index === 1) + assert(manager.resourceOffer("execB", "host2", PROCESS_LOCAL, 1).get.index === 2) // no-pref tasks are scheduled without delay - assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL) == None) - assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) - assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index === 0) - assert(manager.resourceOffer("execA", "host1", ANY) == None) + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL, 1) == None) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL, 1) == None) + assert(manager.resourceOffer("execA", "host1", NO_PREF, 1).get.index === 0) + assert(manager.resourceOffer("execA", "host1", ANY, 1) == None) } test("Ensure TaskSetManager is usable after addition of levels") { @@ -994,9 +998,9 @@ class TaskSetManagerSuite assert(manager.pendingTasks.noPrefs.size === 0) // Valid locality should contain PROCESS_LOCAL, NODE_LOCAL and ANY assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) - assert(manager.resourceOffer("execA", "host1", ANY) !== None) + assert(manager.resourceOffer("execA", "host1", ANY, 1) !== None) clock.advance(LOCALITY_WAIT_MS * 4) - assert(manager.resourceOffer("execB.2", "host2", ANY) !== None) + assert(manager.resourceOffer("execB.2", "host2", ANY, 1) !== None) sched.removeExecutor("execA") sched.removeExecutor("execB.2") manager.executorLost("execA", "host1", SlaveLost()) @@ -1005,7 +1009,7 @@ class TaskSetManagerSuite sched.addExecutor("execC", "host3") manager.executorAdded() // Prior to the fix, this line resulted in an ArrayIndexOutOfBoundsException: - assert(manager.resourceOffer("execC", "host3", ANY) !== None) + assert(manager.resourceOffer("execC", "host3", ANY, 1) !== None) } test("Test that locations with HDFSCacheTaskLocation are treated as PROCESS_LOCAL.") { @@ -1057,7 +1061,7 @@ class TaskSetManagerSuite "exec1" -> "host1", "exec2" -> "host2", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(k, v, NO_PREF) + val taskOption = manager.resourceOffer(k, v, NO_PREF, 1) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === k) @@ -1078,7 +1082,7 @@ class TaskSetManagerSuite assert(sched.speculativeTasks.toSet === Set(3)) // Offer resource to start the speculative attempt for the running task - val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF, 1) assert(taskOption5.isDefined) val task5 = taskOption5.get assert(task5.index === 3) @@ -1117,7 +1121,7 @@ class TaskSetManagerSuite "exec1" -> "host1", "exec2" -> "host2", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(k, v, NO_PREF) + val taskOption = manager.resourceOffer(k, v, NO_PREF, 1) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === k) @@ -1150,7 +1154,7 @@ class TaskSetManagerSuite manager.handleFailedTask(task.taskId, TaskState.FAILED, endReason) sched.endedTasks(task.taskId) = endReason assert(!manager.isZombie) - val nextTask = manager.resourceOffer(s"exec2", s"host2", NO_PREF) + val nextTask = manager.resourceOffer(s"exec2", s"host2", NO_PREF, 1) assert(nextTask.isDefined, s"no offer for attempt $attempt of $index") tasks += nextTask.get } @@ -1166,7 +1170,7 @@ class TaskSetManagerSuite assert(manager.checkSpeculatableTasks(0)) assert(sched.speculativeTasks.toSet === Set(3, 4)) // Offer resource to start the speculative attempt for the running task - val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF, 1) assert(taskOption5.isDefined) val speculativeTask = taskOption5.get assert(speculativeTask.index === 3 || speculativeTask.index === 4) @@ -1191,7 +1195,7 @@ class TaskSetManagerSuite assert(!manager.isZombie) // now run another speculative task - val taskOpt6 = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOpt6 = manager.resourceOffer("exec1", "host1", NO_PREF, 1) assert(taskOpt6.isDefined) val speculativeTask2 = taskOpt6.get assert(speculativeTask2.index === 3 || speculativeTask2.index === 4) @@ -1222,7 +1226,7 @@ class TaskSetManagerSuite val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = new ManualClock(1)) when(mockDAGScheduler.taskEnded(any(), any(), any(), any(), any(), any())).thenAnswer( (invocationOnMock: InvocationOnMock) => assert(manager.isZombie)) - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, 1) assert(taskOption.isDefined) // this would fail, inside our mock dag scheduler, if it calls dagScheduler.taskEnded() too soon manager.handleSuccessfulTask(0, createTaskResult(0)) @@ -1267,7 +1271,7 @@ class TaskSetManagerSuite "exec2" -> "host1" ).flatMap { case (exec, host) => // offer each executor twice (simulating 2 cores per executor) - (0 until 2).flatMap{ _ => tsmSpy.resourceOffer(exec, host, TaskLocality.ANY)} + (0 until 2).flatMap{ _ => tsmSpy.resourceOffer(exec, host, TaskLocality.ANY, 1)} } assert(taskDescs.size === 4) @@ -1304,7 +1308,7 @@ class TaskSetManagerSuite "exec2" -> "host2" ).flatMap { case (exec, host) => // offer each executor twice (simulating 2 cores per executor) - (0 until 2).flatMap{ _ => tsm.resourceOffer(exec, host, TaskLocality.ANY)} + (0 until 2).flatMap{ _ => tsm.resourceOffer(exec, host, TaskLocality.ANY, 1)} } assert(taskDescs.size === 4) @@ -1340,7 +1344,7 @@ class TaskSetManagerSuite val taskSetManager = new TaskSetManager(sched, taskSet, 1, Some(blacklistTracker)) val taskSetManagerSpy = spy(taskSetManager) - val taskDesc = taskSetManagerSpy.resourceOffer(exec, host, TaskLocality.ANY) + val taskDesc = taskSetManagerSpy.resourceOffer(exec, host, TaskLocality.ANY, 1) // Assert the task has been black listed on the executor it was last executed on. when(taskSetManagerSpy.addPendingTask(anyInt(), anyBoolean(), anyBoolean())).thenAnswer( @@ -1368,9 +1372,9 @@ class TaskSetManagerSuite val manager1 = new TaskSetManager(sched, taskSet1, MAX_TASK_FAILURES, clock = new ManualClock) // all tasks from the first taskset have the same jars - val taskOption1 = manager1.resourceOffer("exec1", "host1", NO_PREF) + val taskOption1 = manager1.resourceOffer("exec1", "host1", NO_PREF, 1) assert(taskOption1.get.addedJars === addedJarsPreTaskSet) - val taskOption2 = manager1.resourceOffer("exec1", "host1", NO_PREF) + val taskOption2 = manager1.resourceOffer("exec1", "host1", NO_PREF, 1) assert(taskOption2.get.addedJars === addedJarsPreTaskSet) // even with a jar added mid-TaskSet @@ -1378,7 +1382,7 @@ class TaskSetManagerSuite sc.addJar(jarPath.toString) val addedJarsMidTaskSet = Map[String, Long](sc.addedJars.toSeq: _*) assert(addedJarsPreTaskSet !== addedJarsMidTaskSet) - val taskOption3 = manager1.resourceOffer("exec1", "host1", NO_PREF) + val taskOption3 = manager1.resourceOffer("exec1", "host1", NO_PREF, 1) // which should have the old version of the jars list assert(taskOption3.get.addedJars === addedJarsPreTaskSet) @@ -1386,7 +1390,7 @@ class TaskSetManagerSuite val taskSet2 = FakeTask.createTaskSet(1) val manager2 = new TaskSetManager(sched, taskSet2, MAX_TASK_FAILURES, clock = new ManualClock) - val taskOption4 = manager2.resourceOffer("exec1", "host1", NO_PREF) + val taskOption4 = manager2.resourceOffer("exec1", "host1", NO_PREF, 1) assert(taskOption4.get.addedJars === addedJarsMidTaskSet) } @@ -1480,11 +1484,11 @@ class TaskSetManagerSuite } // Offer resources for 4 tasks to start for ((exec, host) <- Seq( - "exec1" -> "host1", - "exec1" -> "host1", - "exec3" -> "host3", - "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(exec, host, NO_PREF) + "exec1" -> "host1", + "exec1" -> "host1", + "exec3" -> "host3", + "exec2" -> "host2")) { + val taskOption = manager.resourceOffer(exec, host, NO_PREF, 1) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === exec) @@ -1510,7 +1514,7 @@ class TaskSetManagerSuite assert(sched.speculativeTasks.toSet === Set(2, 3)) // Offer resource to start the speculative attempt for the running task 2.0 - val taskOption = manager.resourceOffer("exec2", "host2", ANY) + val taskOption = manager.resourceOffer("exec2", "host2", ANY, 1) assert(taskOption.isDefined) val task4 = taskOption.get assert(task4.index === 2) @@ -1556,7 +1560,7 @@ class TaskSetManagerSuite "exec1" -> "host1", "exec2" -> "host2", "exec2" -> "host2")) { - val taskOption = manager.resourceOffer(k, v, NO_PREF) + val taskOption = manager.resourceOffer(k, v, NO_PREF, 1) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === k) @@ -1576,7 +1580,7 @@ class TaskSetManagerSuite assert(sched.speculativeTasks.toSet === Set(3)) // Offer resource to start the speculative attempt for the running task - val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF) + val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF, 1) assert(taskOption5.isDefined) val task5 = taskOption5.get assert(task5.index === 3) @@ -1636,16 +1640,16 @@ class TaskSetManagerSuite assert(FakeRackUtil.numBatchInvocation === 1) assert(FakeRackUtil.numSingleHostInvocation === 0) // with rack locality, reject an offer on a host with an unknown rack - assert(manager.resourceOffer("otherExec", "otherHost", TaskLocality.RACK_LOCAL).isEmpty) + assert(manager.resourceOffer("otherExec", "otherHost", TaskLocality.RACK_LOCAL, 1).isEmpty) (0 until 20).foreach { rackIdx => (0 until 5).foreach { offerIdx => // if we offer hosts which are not in preferred locations, // we'll reject them at NODE_LOCAL level, // but accept them at RACK_LOCAL level if they're on OK racks val hostIdx = 100 + rackIdx - assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.NODE_LOCAL) + assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.NODE_LOCAL, 1) .isEmpty) - assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.RACK_LOCAL) + assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.RACK_LOCAL, 1) .isDefined) } } @@ -1655,7 +1659,7 @@ class TaskSetManagerSuite assert(FakeRackUtil.numBatchInvocation === 1) } - test("TaskSetManager allocate resource addresses from available resources") { + test("TaskSetManager passes task resource along") { import TestUtils._ sc = new SparkContext("local", "test") @@ -1664,15 +1668,13 @@ class TaskSetManagerSuite val taskSet = FakeTask.createTaskSet(1) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) - val availableResources = Map(GPU -> ArrayBuffer("0", "1", "2", "3")) - val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF, availableResources) + val taskResourceAssignments = Map(GPU -> new ResourceInformation(GPU, Array("0", "1"))) + val taskOption = + manager.resourceOffer("exec1", "host1", NO_PREF, 1, taskResourceAssignments) assert(taskOption.isDefined) val allocatedResources = taskOption.get.resources assert(allocatedResources.size == 1) assert(allocatedResources(GPU).addresses sameElements Array("0", "1")) - // Allocated resource addresses should still present in `availableResources`, they will only - // get removed inside TaskSchedulerImpl later. - assert(availableResources(GPU) sameElements Array("0", "1", "2", "3")) } test("SPARK-26755 Ensure that a speculative task is submitted only once for execution") { @@ -1691,7 +1693,7 @@ class TaskSetManagerSuite // Offer resources for 4 tasks to start, 2 on each exec Seq("exec1" -> "host1", "exec2" -> "host2").foreach { case (exec, host) => (0 until 2).foreach { _ => - val taskOption = manager.resourceOffer(exec, host, NO_PREF) + val taskOption = manager.resourceOffer(exec, host, NO_PREF, 1) assert(taskOption.isDefined) assert(taskOption.get.executorId === exec) } @@ -1715,8 +1717,8 @@ class TaskSetManagerSuite // Offer resource to start the speculative attempt for the running task. We offer more // resources, and ensure that speculative tasks get scheduled appropriately -- only one extra // copy per speculatable task - val taskOption2 = manager.resourceOffer("exec1", "host1", NO_PREF) - val taskOption3 = manager.resourceOffer("exec2", "host2", NO_PREF) + val taskOption2 = manager.resourceOffer("exec1", "host1", NO_PREF, 1) + val taskOption3 = manager.resourceOffer("exec2", "host2", NO_PREF, 1) assert(taskOption2.isDefined) val task2 = taskOption2.get // Ensure that task index 3 is launched on host1 and task index 4 on host2 @@ -1736,9 +1738,9 @@ class TaskSetManagerSuite assert(manager.copiesRunning(1) === 2) assert(manager.copiesRunning(3) === 2) // Offering additional resources should not lead to any speculative tasks being respawned - assert(manager.resourceOffer("exec1", "host1", ANY).isEmpty) - assert(manager.resourceOffer("exec2", "host2", ANY).isEmpty) - assert(manager.resourceOffer("exec3", "host3", ANY).isEmpty) + assert(manager.resourceOffer("exec1", "host1", ANY, 1).isEmpty) + assert(manager.resourceOffer("exec2", "host2", ANY, 1).isEmpty) + assert(manager.resourceOffer("exec3", "host3", ANY, 1).isEmpty) } test("SPARK-26755 Ensure that a speculative task obeys original locality preferences") { @@ -1761,7 +1763,7 @@ class TaskSetManagerSuite } // Offer resources for 3 tasks to start Seq("exec1" -> "host1", "exec2" -> "host2", "exec3" -> "host3").foreach { case (exec, host) => - val taskOption = manager.resourceOffer(exec, host, NO_PREF) + val taskOption = manager.resourceOffer(exec, host, NO_PREF, 1) assert(taskOption.isDefined) assert(taskOption.get.executorId === exec) } @@ -1774,17 +1776,17 @@ class TaskSetManagerSuite assert(manager.checkSpeculatableTasks(0)) assert(sched.speculativeTasks.toSet === Set(0, 1)) // Ensure that the speculatable tasks obey the original locality preferences - assert(manager.resourceOffer("exec4", "host4", NODE_LOCAL).isEmpty) + assert(manager.resourceOffer("exec4", "host4", NODE_LOCAL, 1).isEmpty) // task 1 does have a node-local preference for host2 -- but we've already got a regular // task running there, so we should not schedule a speculative there as well. - assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL).isEmpty) - assert(manager.resourceOffer("exec3", "host3", NODE_LOCAL).isDefined) - assert(manager.resourceOffer("exec4", "host4", ANY).isDefined) + assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL, 1).isEmpty) + assert(manager.resourceOffer("exec3", "host3", NODE_LOCAL, 1).isDefined) + assert(manager.resourceOffer("exec4", "host4", ANY, 1).isDefined) // Since, all speculatable tasks have been launched, making another offer // should not schedule any more tasks - assert(manager.resourceOffer("exec1", "host1", ANY).isEmpty) + assert(manager.resourceOffer("exec1", "host1", ANY, 1).isEmpty) assert(!manager.checkSpeculatableTasks(0)) - assert(manager.resourceOffer("exec1", "host1", ANY).isEmpty) + assert(manager.resourceOffer("exec1", "host1", ANY, 1).isEmpty) } private def testSpeculationDurationSetup( @@ -1793,15 +1795,16 @@ class TaskSetManagerSuite numTasks: Int, numExecutorCores: Int, numCoresPerTask: Int): (TaskSetManager, ManualClock) = { - sc = new SparkContext("local", "test") - sc.conf.set(config.SPECULATION_ENABLED, true) - sc.conf.set(config.SPECULATION_QUANTILE.key, speculationQuantile.toString) + val conf = new SparkConf() + conf.set(config.SPECULATION_ENABLED, true) + conf.set(config.SPECULATION_QUANTILE.key, speculationQuantile.toString) // Set the number of slots per executor - sc.conf.set(config.EXECUTOR_CORES.key, numExecutorCores.toString) - sc.conf.set(config.CPUS_PER_TASK.key, numCoresPerTask.toString) + conf.set(config.EXECUTOR_CORES.key, numExecutorCores.toString) + conf.set(config.CPUS_PER_TASK.key, numCoresPerTask.toString) if (speculationThresholdOpt.isDefined) { - sc.conf.set(config.SPECULATION_TASK_DURATION_THRESHOLD.key, speculationThresholdOpt.get) + conf.set(config.SPECULATION_TASK_DURATION_THRESHOLD.key, speculationThresholdOpt.get) } + sc = new SparkContext("local", "test", conf) sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) // Create a task set with the given number of tasks val taskSet = FakeTask.createTaskSet(numTasks) @@ -1811,7 +1814,7 @@ class TaskSetManagerSuite // Offer resources for the task to start for (i <- 1 to numTasks) { - manager.resourceOffer(s"exec$i", s"host$i", NO_PREF) + manager.resourceOffer(s"exec$i", s"host$i", NO_PREF, numCoresPerTask) } (manager, clock) } @@ -1890,15 +1893,28 @@ class TaskSetManagerSuite test("SPARK-30417 when spark.task.cpus is greater than spark.executor.cores due to " + "standalone settings, speculate if there is only one task in the stage") { - val (manager, clock) = testSpeculationDurationSetup( - Some("60min"), - // Set the quantile to be 1.0 so that regular speculation would not be triggered - speculationQuantile = 1.0, - numTasks = 1, - numExecutorCores = 1, - numCoresPerTask = 2 - ) + val numTasks = 1 + val numCoresPerTask = 2 + val conf = new SparkConf() + // skip throwing exception when cores per task > cores per executor to emulate standalone mode + conf.set(TASKSET_MANAGER_SPECULATION_TESTING, true) + conf.set(config.SPECULATION_ENABLED, true) + conf.set(config.SPECULATION_QUANTILE.key, "1.0") + // Skip setting cores per executor to emulate standalone default mode + conf.set(config.CPUS_PER_TASK.key, numCoresPerTask.toString) + conf.set(config.SPECULATION_TASK_DURATION_THRESHOLD.key, "60min") + sc = new SparkContext("local", "test", conf) + sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) + // Create a task set with the given number of tasks + val taskSet = FakeTask.createTaskSet(numTasks) + val clock = new ManualClock() + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + manager.isZombie = false + // Offer resources for the task to start + for (i <- 1 to numTasks) { + manager.resourceOffer(s"exec$i", s"host$i", NO_PREF, numCoresPerTask) + } clock.advance(1000*60*60) assert(!manager.checkSpeculatableTasks(0)) assert(sched.speculativeTasks.size == 0) @@ -1915,7 +1931,7 @@ class TaskSetManagerSuite val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) assert(sched.taskSetsFailed.isEmpty) - val offerResult = manager.resourceOffer("exec1", "host1", ANY) + val offerResult = manager.resourceOffer("exec1", "host1", ANY, 1) assert(offerResult.isDefined, "Expect resource offer on iteration 0 to return a task") assert(offerResult.get.index === 0) @@ -1942,7 +1958,8 @@ class TaskSetManagerSuite TestUtils.waitUntilExecutorsUp(sc, 2, 60000) val tasks = Array.tabulate[Task[_]](2)(partition => new FakeLongTasks(stageId = 0, partition)) - val taskSet: TaskSet = new TaskSet(tasks, stageId = 0, stageAttemptId = 0, priority = 0, null) + val taskSet: TaskSet = new TaskSet(tasks, stageId = 0, stageAttemptId = 0, priority = 0, null, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val stageId = taskSet.stageId val stageAttemptId = taskSet.stageAttemptId sched.submitTasks(taskSet) diff --git a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala index 615389ae5c2d4..65e0031249de3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark._ import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config._ import org.apache.spark.resource.ResourceProfile.{DEFAULT_RESOURCE_PROFILE_ID, UNKNOWN_RESOURCE_PROFILE_ID} +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage._ @@ -435,7 +436,8 @@ class ExecutorMonitorSuite extends SparkFunSuite { private def stageInfo(id: Int, shuffleId: Int = -1): StageInfo = { new StageInfo(id, 0, s"stage$id", 1, Nil, Nil, "", - shuffleDepId = if (shuffleId >= 0) Some(shuffleId) else None) + shuffleDepId = if (shuffleId >= 0) Some(shuffleId) else None, + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) } private def taskInfo( diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index e7eed7bf4c879..90c5239f4bb48 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -30,6 +30,7 @@ import org.apache.spark._ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.config.Status._ import org.apache.spark.metrics.ExecutorMetricType +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster._ import org.apache.spark.status.ListenerEventsTestHelper._ @@ -151,8 +152,10 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // Start a job with 2 stages / 4 tasks each time += 1 val stages = Seq( - new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1"), - new StageInfo(2, 0, "stage2", 4, Nil, Seq(1), "details2")) + new StageInfo(1, 0, "stage1", 4, Nil, Nil, + "details1", resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID), + new StageInfo(2, 0, "stage2", 4, Nil, Seq(1), + "details2", resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) val jobProps = new Properties() jobProps.setProperty(SparkContext.SPARK_JOB_DESCRIPTION, "jobDescription") @@ -524,7 +527,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // - Re-submit stage 2, all tasks, and succeed them and the stage. val oldS2 = stages.last val newS2 = new StageInfo(oldS2.stageId, oldS2.attemptNumber + 1, oldS2.name, oldS2.numTasks, - oldS2.rddInfos, oldS2.parentIds, oldS2.details, oldS2.taskMetrics) + oldS2.rddInfos, oldS2.parentIds, oldS2.details, oldS2.taskMetrics, + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) time += 1 newS2.submissionTime = Some(time) @@ -575,8 +579,10 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // change the stats of the already finished job. time += 1 val j2Stages = Seq( - new StageInfo(3, 0, "stage1", 4, Nil, Nil, "details1"), - new StageInfo(4, 0, "stage2", 4, Nil, Seq(3), "details2")) + new StageInfo(3, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID), + new StageInfo(4, 0, "stage2", 4, Nil, Seq(3), "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) j2Stages.last.submissionTime = Some(time) listener.onJobStart(SparkListenerJobStart(2, time, j2Stages, null)) assert(store.count(classOf[JobDataWrapper]) === 2) @@ -703,7 +709,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // Submit a stage for the first RDD before it's marked for caching, to make sure later // the listener picks up the correct storage level. val rdd1Info = new RDDInfo(rdd1b1.rddId, "rdd1", 2, StorageLevel.NONE, false, Nil) - val stage0 = new StageInfo(0, 0, "stage0", 4, Seq(rdd1Info), Nil, "details0") + val stage0 = new StageInfo(0, 0, "stage0", 4, Seq(rdd1Info), Nil, "details0", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) listener.onStageSubmitted(SparkListenerStageSubmitted(stage0, new Properties())) listener.onStageCompleted(SparkListenerStageCompleted(stage0)) assert(store.count(classOf[RDDStorageInfoWrapper]) === 0) @@ -711,7 +718,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // Submit a stage and make sure the RDDs are recorded. rdd1Info.storageLevel = level val rdd2Info = new RDDInfo(rdd2b1.rddId, "rdd2", 1, level, false, Nil) - val stage = new StageInfo(1, 0, "stage1", 4, Seq(rdd1Info, rdd2Info), Nil, "details1") + val stage = new StageInfo(1, 0, "stage1", 4, Seq(rdd1Info, rdd2Info), Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) check[RDDStorageInfoWrapper](rdd1b1.rddId) { wrapper => @@ -1018,9 +1026,12 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // data is not deleted. time += 1 val stages = Seq( - new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1"), - new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2"), - new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3")) + new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID), + new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID), + new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) // Graph data is generated by the job start event, so fire it. listener.onJobStart(SparkListenerJobStart(4, time, stages, null)) @@ -1068,7 +1079,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { } assert(store.count(classOf[CachedQuantile], "stage", key(dropped)) === 0) - val attempt2 = new StageInfo(3, 1, "stage3", 4, Nil, Nil, "details3") + val attempt2 = new StageInfo(3, 1, "stage3", 4, Nil, Nil, "details3", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) time += 1 attempt2.submissionTime = Some(time) listener.onStageSubmitted(SparkListenerStageSubmitted(attempt2, new Properties())) @@ -1112,6 +1124,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { test("eviction should respect job completion time") { val testConf = conf.clone().set(MAX_RETAINED_JOBS, 2) + val listener = new AppStatusListener(store, testConf, true) // Start job 1 and job 2 @@ -1139,9 +1152,12 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { val testConf = conf.clone().set(MAX_RETAINED_STAGES, 2) val listener = new AppStatusListener(store, testConf, true) - val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1") - val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2") - val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3") + val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) // Start stage 1 and stage 2 time += 1 @@ -1172,8 +1188,10 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { val testConf = conf.clone().set(MAX_RETAINED_STAGES, 2) val listener = new AppStatusListener(store, testConf, true) - val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1") - val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2") + val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) // Sart job 1 time += 1 @@ -1193,7 +1211,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { listener.onJobEnd(SparkListenerJobEnd(1, time, JobSucceeded)) // Submit stage 3 and verify stage 2 is evicted - val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3") + val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) time += 1 stage3.submissionTime = Some(time) listener.onStageSubmitted(SparkListenerStageSubmitted(stage3, new Properties())) @@ -1208,7 +1227,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { val testConf = conf.clone().set(MAX_RETAINED_TASKS_PER_STAGE, 2) val listener = new AppStatusListener(store, testConf, true) - val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1") + val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) stage1.submissionTime = Some(time) listener.onStageSubmitted(SparkListenerStageSubmitted(stage1, new Properties())) @@ -1243,9 +1263,12 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { val listener = new AppStatusListener(store, testConf, true) val appStore = new AppStatusStore(store) - val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1") - val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2") - val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3") + val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) time += 1 stage1.submissionTime = Some(time) @@ -1274,8 +1297,10 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { test("SPARK-24415: update metrics for tasks that finish late") { val listener = new AppStatusListener(store, conf, true) - val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1") - val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2") + val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) // Start job listener.onJobStart(SparkListenerJobStart(1, time, Seq(stage1, stage2), null)) @@ -1335,12 +1360,12 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { } else { conf.clone().set(LIVE_ENTITY_UPDATE_PERIOD, -1L) } - val listener = new AppStatusListener(store, testConf, live) listener.onExecutorAdded(createExecutorAddedEvent(1)) listener.onExecutorAdded(createExecutorAddedEvent(2)) - val stage = new StageInfo(1, 0, "stage", 4, Nil, Nil, "details") + val stage = new StageInfo(1, 0, "stage", 4, Nil, Nil, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) listener.onJobStart(SparkListenerJobStart(1, time, Seq(stage), null)) listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) @@ -1577,7 +1602,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // Submit a stage and make sure the RDDs are recorded. val rdd1Info = new RDDInfo(rdd1b1.rddId, "rdd1", 2, level, false, Nil) - val stage = new StageInfo(1, 0, "stage1", 4, Seq(rdd1Info), Nil, "details1") + val stage = new StageInfo(1, 0, "stage1", 4, Seq(rdd1Info), Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) // Add partition 1 replicated on two block managers. diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala index 735e51942626f..0cab37d767c12 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.status import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.executor.TaskMetrics +import org.apache.spark.resource.ResourceProfileManager import org.apache.spark.scheduler.{TaskInfo, TaskLocality} import org.apache.spark.util.{Distribution, Utils} import org.apache.spark.util.kvstore._ diff --git a/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala b/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala index 4b3fbacc47f9c..99c0d9593ccae 100644 --- a/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala +++ b/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala @@ -23,6 +23,7 @@ import scala.collection.immutable.Map import org.apache.spark.{AccumulatorSuite, SparkContext, Success, TaskState} import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorAdded, SparkListenerExecutorMetricsUpdate, SparkListenerExecutorRemoved, SparkListenerJobStart, SparkListenerStageCompleted, SparkListenerStageSubmitted, SparkListenerTaskEnd, SparkListenerTaskStart, StageInfo, TaskInfo, TaskLocality} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage.{RDDInfo, StorageLevel} @@ -61,7 +62,8 @@ object ListenerEventsTestHelper { } def createStage(id: Int, rdds: Seq[RDDInfo], parentIds: Seq[Int]): StageInfo = { - new StageInfo(id, 0, s"stage${id}", 4, rdds, parentIds, s"details${id}") + new StageInfo(id, 0, s"stage${id}", 4, rdds, parentIds, s"details${id}", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) } def createStage(rdds: Seq[RDDInfo], parentIds: Seq[Int]): StageInfo = { @@ -96,13 +98,15 @@ object ListenerEventsTestHelper { /** Create a stage submitted event for the specified stage Id. */ def createStageSubmittedEvent(stageId: Int): SparkListenerStageSubmitted = { SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0, - Seq.empty, Seq.empty, "details")) + Seq.empty, Seq.empty, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) } /** Create a stage completed event for the specified stage Id. */ def createStageCompletedEvent(stageId: Int): SparkListenerStageCompleted = { SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0, - Seq.empty, Seq.empty, "details")) + Seq.empty, Seq.empty, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) } def createExecutorAddedEvent(executorId: Int): SparkListenerExecutorAdded = { diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala index bd18e9e628da8..b9e11ed8bd250 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -27,6 +27,7 @@ import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} import org.apache.spark._ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.config.Status._ +import org.apache.spark.resource.{ResourceProfile, ResourceProfileManager} import org.apache.spark.scheduler._ import org.apache.spark.status.AppStatusStore import org.apache.spark.status.api.v1.{AccumulableInfo => UIAccumulableInfo, StageData, StageStatus} @@ -91,7 +92,8 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { accumulatorUpdates = Seq(new UIAccumulableInfo(0L, "acc", None, "value")), tasks = None, executorSummary = None, - killedTasksSummary = Map.empty + killedTasksSummary = Map.empty, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID ) val taskTable = new TaskPagedTable( stageData, @@ -131,7 +133,8 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { val page = new StagePage(tab, statusStore) // Simulate a stage in job progress listener - val stageInfo = new StageInfo(0, 0, "dummy", 1, Seq.empty, Seq.empty, "details") + val stageInfo = new StageInfo(0, 0, "dummy", 1, Seq.empty, Seq.empty, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) // Simulate two tasks to test PEAK_EXECUTION_MEMORY correctness (1 to 2).foreach { taskId => diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index a2a4b3aa974fc..c2903d109abc7 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -32,8 +32,7 @@ import org.apache.spark._ import org.apache.spark.executor._ import org.apache.spark.metrics.ExecutorMetricType import org.apache.spark.rdd.RDDOperationScope -import org.apache.spark.resource.ResourceInformation -import org.apache.spark.resource.ResourceUtils +import org.apache.spark.resource.{ResourceInformation, ResourceProfile, ResourceUtils} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.shuffle.MetadataFetchFailedException @@ -93,7 +92,7 @@ class JsonProtocolSuite extends SparkFunSuite { 42L, "Garfield", Some("appAttempt"), Some(logUrlMap)) val applicationEnd = SparkListenerApplicationEnd(42L) val executorAdded = SparkListenerExecutorAdded(executorAddedTime, "exec1", - new ExecutorInfo("Hostee.awesome.com", 11, logUrlMap, attributes, resources.toMap)) + new ExecutorInfo("Hostee.awesome.com", 11, logUrlMap, attributes, resources.toMap, 4)) val executorRemoved = SparkListenerExecutorRemoved(executorRemovedTime, "exec2", "test reason") val executorBlacklisted = SparkListenerExecutorBlacklisted(executorBlacklistedTime, "exec1", 22) val executorUnblacklisted = @@ -232,6 +231,21 @@ class JsonProtocolSuite extends SparkFunSuite { assert(0 === newInfo.accumulables.size) } + test("StageInfo resourceProfileId") { + val info = makeStageInfo(1, 2, 3, 4L, 5L, 5) + val json = JsonProtocol.stageInfoToJson(info) + + // Fields added after 1.0.0. + assert(info.details.nonEmpty) + assert(info.resourceProfileId === 5) + + val newInfo = JsonProtocol.stageInfoFromJson(json) + + assert(info.name === newInfo.name) + assert(5 === newInfo.resourceProfileId) + } + + test("InputMetrics backward compatibility") { // InputMetrics were added after 1.0.1. val metrics = makeTaskMetrics(1L, 2L, 3L, 4L, 5, 6, hasHadoopInput = true, hasOutput = false) @@ -341,7 +355,8 @@ class JsonProtocolSuite extends SparkFunSuite { val stageIds = Seq[Int](1, 2, 3, 4) val stageInfos = stageIds.map(x => makeStageInfo(x, x * 200, x * 300, x * 400L, x * 500L)) val dummyStageInfos = - stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown")) + stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) val jobStart = SparkListenerJobStart(10, jobSubmissionTime, stageInfos, properties) val oldEvent = JsonProtocol.jobStartToJson(jobStart).removeField({_._1 == "Stage Infos"}) val expectedJobStart = @@ -383,9 +398,11 @@ class JsonProtocolSuite extends SparkFunSuite { test("StageInfo backward compatibility (parent IDs)") { // Prior to Spark 1.4.0, StageInfo did not have the "Parent IDs" property - val stageInfo = new StageInfo(1, 1, "me-stage", 1, Seq.empty, Seq(1, 2, 3), "details") + val stageInfo = new StageInfo(1, 1, "me-stage", 1, Seq.empty, Seq(1, 2, 3), "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val oldStageInfo = JsonProtocol.stageInfoToJson(stageInfo).removeField({ _._1 == "Parent IDs"}) - val expectedStageInfo = new StageInfo(1, 1, "me-stage", 1, Seq.empty, Seq.empty, "details") + val expectedStageInfo = new StageInfo(1, 1, "me-stage", 1, Seq.empty, Seq.empty, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) assertEquals(expectedStageInfo, JsonProtocol.stageInfoFromJson(oldStageInfo)) } @@ -871,9 +888,16 @@ private[spark] object JsonProtocolSuite extends Assertions { r } - private def makeStageInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { + private def makeStageInfo( + a: Int, + b: Int, + c: Int, + d: Long, + e: Long, + rpId: Int = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) = { val rddInfos = (0 until a % 5).map { i => makeRddInfo(a + i, b + i, c + i, d + i, e + i) } - val stageInfo = new StageInfo(a, 0, "greetings", b, rddInfos, Seq(100, 200, 300), "details") + val stageInfo = new StageInfo(a, 0, "greetings", b, rddInfos, Seq(100, 200, 300), "details", + resourceProfileId = rpId) val (acc1, acc2) = (makeAccumulableInfo(1), makeAccumulableInfo(2)) stageInfo.accumulables(acc1.id) = acc1 stageInfo.accumulables(acc2.id) = acc2 @@ -1009,7 +1033,8 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Internal": false, | "Count Failed Values": false | } - | ] + | ], + | "Resource Profile Id" : 0 | }, | "Properties": { | "France": "Paris", @@ -1066,7 +1091,8 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Internal": false, | "Count Failed Values": false | } - | ] + | ], + | "Resource Profile Id" : 0 | } |} """.stripMargin @@ -1588,7 +1614,8 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Internal": false, | "Count Failed Values": false | } - | ] + | ], + | "Resource Profile Id" : 0 | }, | { | "Stage ID": 2, @@ -1648,7 +1675,8 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Internal": false, | "Count Failed Values": false | } - | ] + | ], + | "Resource Profile Id" : 0 | }, | { | "Stage ID": 3, @@ -1724,7 +1752,8 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Internal": false, | "Count Failed Values": false | } - | ] + | ], + | "Resource Profile Id" : 0 | }, | { | "Stage ID": 4, @@ -1816,7 +1845,8 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Internal": false, | "Count Failed Values": false | } - | ] + | ], + | "Resource Profile Id" : 0 | } | ], | "Stage IDs": [ @@ -1963,7 +1993,8 @@ private[spark] object JsonProtocolSuite extends Assertions { | "name" : "gpu", | "addresses" : [ "0", "1" ] | } - | } + | }, + | "Resource Profile Id": 4 | } |} """.stripMargin diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 73f461255de43..be2d59a5cc05e 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -119,3 +119,4 @@ vote.tmpl SessionManager.java SessionHandler.java GangliaReporter.java +application_1578436911597_0052 diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index f1bbe0b10e228..982a90d00272d 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -70,6 +70,9 @@ object MimaExcludes { // [SPARK-29417][CORE] Resource Scheduling - add TaskContext.resource java api ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.resourcesJMap"), + // [SPARK-27495][CORE] Support Stage Level Scheduling + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.TaskContext.resourceProfileId"), + // [SPARK-27410][MLLIB] Remove deprecated / no-op mllib.KMeans getRuns, setRuns ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.KMeans.getRuns"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.KMeans.setRuns"), diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index 76a5bd0d645ba..e3c3254bdf58b 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -55,6 +55,12 @@ from pyspark.accumulators import Accumulator, AccumulatorParam from pyspark.broadcast import Broadcast from pyspark.resourceinformation import ResourceInformation +from pyspark.taskresourcerequest import TaskResourceRequest +from pyspark.executorresourcerequest import ExecutorResourceRequest +from pyspark.taskresourcerequests import TaskResourceRequests +from pyspark.executorresourcerequests import ExecutorResourceRequests +from pyspark.resourceprofilebuilder import ResourceProfileBuilder +from pyspark.resourceprofile import ResourceProfile from pyspark.serializers import MarshalSerializer, PickleSerializer from pyspark.status import * from pyspark.taskcontext import TaskContext, BarrierTaskContext, BarrierTaskInfo @@ -120,4 +126,6 @@ def wrapper(self, *args, **kwargs): "Accumulator", "AccumulatorParam", "MarshalSerializer", "PickleSerializer", "StatusTracker", "SparkJobInfo", "SparkStageInfo", "Profiler", "BasicProfiler", "TaskContext", "RDDBarrier", "BarrierTaskContext", "BarrierTaskInfo", "ResourceInformation", + "TaskResourceRequest", "TaskResourceRequests", "ExecutorResourceRequest", + "ExecutorResourceRequests", "ResourceProfile" ] diff --git a/python/pyspark/executorresourcerequest.py b/python/pyspark/executorresourcerequest.py new file mode 100644 index 0000000000000..79c824f64cb5f --- /dev/null +++ b/python/pyspark/executorresourcerequest.py @@ -0,0 +1,73 @@ +# +# 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. +# + + +class ExecutorResourceRequest(object): + """ + .. note:: Evolving + + An Executor resource request. 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 is used to specify what the resource requirements are for an Executor and how + Spark can find out specific details about those resources. Not all the parameters are + required for every resource type. Resources like GPUs are supported and have same limitations + as using the global spark configs spark.executor.resource.gpu.*. The amount, discoveryScript, + and vendor parameters for resources are all the same parameters a user would specify through the + configs: spark.executor.resource.{resourceName}.{amount, discoveryScript, vendor}. + + For instance, a user wants to allocate an Executor with GPU resources on YARN. The user has + to specify the resource name (gpu), the amount or number of GPUs per Executor, + the discovery script would be specified so that when the Executor starts up it can + discovery what GPU addresses are available for it to use because YARN doesn't tell + Spark that, then vendor would not be used because its specific for Kubernetes. + + See the configuration and cluster specific docs for more details. + + Use ExecutorResourceRequests class as a convenience API. + + :param resourceName: Name of the resource + :param amount: Amount requesting + :param discoveryScript: Optional script used to discover the resources. This is required on some + cluster managers that don't tell Spark the addresses of the resources + allocated. The script runs on Executors startup to discover the addresses + of the resources available. + :param vendor: Vendor, required for some cluster managers + """ + + def __init__(self, resourceName, amount, discoveryScript="", vendor=""): + """Create a new ExecutorResourceRequest that wraps the underlying JVM object.""" + from pyspark.context import SparkContext + self._jExecRequest = SparkContext._jvm.org.apache.spark.resource.ExecutorResourceRequest( + resourceName, amount, discoveryScript, vendor) + + @property + def resourceName(self): + return self._jExecRequest.resourceName() + + @property + def amount(self): + return self._jExecRequest.amount() + + @property + def discoveryScript(self): + return self._jExecRequest.discoveryScript() + + @property + def vendor(self): + return self._jExecRequest.vendor() diff --git a/python/pyspark/executorresourcerequests.py b/python/pyspark/executorresourcerequests.py new file mode 100644 index 0000000000000..761282b38a795 --- /dev/null +++ b/python/pyspark/executorresourcerequests.py @@ -0,0 +1,65 @@ +# +# 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. +# + +from pyspark.executorresourcerequest import ExecutorResourceRequest + + +class ExecutorResourceRequests(object): + + """ + .. note:: Evolving + + A set of Executor resource requests. This is used in conjunction with the + ResourceProfileBuilder to programmatically specify the resources needed for an RDD + that will be applied at the stage level. + """ + + def __init__(self): + """Create a new ExecutorResourceRequests that wraps the underlying JVM object.""" + from pyspark import SparkContext + self._javaExecutorResourceRequests \ + = SparkContext._jvm.org.apache.spark.resource.ExecutorResourceRequests() + + def memory(self, amount): + self._javaExecutorResourceRequests.memory(amount) + return self + + def memoryOverhead(self, amount): + self._javaExecutorResourceRequests.memoryOverhead(amount) + return self + + def pysparkMemory(self, amount): + self._javaExecutorResourceRequests.pysparkMemory(amount) + return self + + def cores(self, amount): + self._javaExecutorResourceRequests.cores(amount) + return self + + def resource(self, resourceName, amount, discoveryScript="", vendor=""): + self._javaExecutorResourceRequests.resource(resourceName, amount, discoveryScript, vendor) + return self + + @property + def requests(self): + execRes = self._javaExecutorResourceRequests.requestsJMap() + result = {} + # convert back to python ExecutorResourceRequest + for k, v in execRes.items(): + result[k] = ExecutorResourceRequest(v.resourceName(), v.amount(), + v.discoveryScript(), v.vendor()) + return result diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 316a5b4d01273..0daf09b17a82a 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -152,6 +152,7 @@ def killChild(): java_import(gateway.jvm, "org.apache.spark.api.python.*") java_import(gateway.jvm, "org.apache.spark.ml.python.*") java_import(gateway.jvm, "org.apache.spark.mllib.api.python.*") + java_import(gateway.jvm, "org.apache.spark.resource.*") # TODO(davies): move into sql java_import(gateway.jvm, "org.apache.spark.sql.*") java_import(gateway.jvm, "org.apache.spark.sql.api.python.*") diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 52ab86c0d88ee..fa59b9d11d6c6 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -47,6 +47,7 @@ from pyspark.statcounter import StatCounter from pyspark.rddsampler import RDDSampler, RDDRangeSampler, RDDStratifiedSampler from pyspark.storagelevel import StorageLevel +from pyspark.resourceprofile import ResourceProfile from pyspark.resultiterable import ResultIterable from pyspark.shuffle import Aggregator, ExternalMerger, \ get_used_memory, ExternalSorter, ExternalGroupBy @@ -255,6 +256,7 @@ class RDD(object): def __init__(self, jrdd, ctx, jrdd_deserializer=AutoBatchedSerializer(PickleSerializer())): self._jrdd = jrdd self.is_cached = False + self.has_resourceProfile = False self.is_checkpointed = False self.ctx = ctx self._jrdd_deserializer = jrdd_deserializer @@ -2477,6 +2479,33 @@ def barrier(self): """ return RDDBarrier(self) + def withResources(self, profile): + """ + .. note:: Experimental + + 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. + + .. versionadded:: 3.0.0 + """ + self.has_resourceProfile = True + self._jrdd.withResources(profile._jResourceProfile) + return self + + def getResourceProfile(self): + """ + .. note:: Experimental + + 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 + + .. versionadded:: 3.0.0 + """ + return ResourceProfile(self._jrdd.getResourceProfile()) + def _is_barrier(self): """ Whether this RDD is in a barrier stage. @@ -2587,6 +2616,7 @@ def pipeline_func(split, iterator): self._prev_jrdd = prev._prev_jrdd # maintain the pipeline self._prev_jrdd_deserializer = prev._prev_jrdd_deserializer self.is_cached = False + self.has_resourceProfile = False self.is_checkpointed = False self.ctx = prev.ctx self.prev = prev @@ -2629,7 +2659,7 @@ def id(self): return self._id def _is_pipelinable(self): - return not (self.is_cached or self.is_checkpointed) + return not (self.is_cached or self.is_checkpointed or self.has_resourceProfile) def _is_barrier(self): return self.is_barrier diff --git a/python/pyspark/resourceinformation.py b/python/pyspark/resourceinformation.py index aaed21374b6ee..79a5e2fe83289 100644 --- a/python/pyspark/resourceinformation.py +++ b/python/pyspark/resourceinformation.py @@ -26,8 +26,8 @@ class ResourceInformation(object): One example is GPUs, where the addresses would be the indices of the GPUs - @param name the name of the resource - @param addresses an array of strings describing the addresses of the resource + :param name: the name of the resource + :param addresses: an array of strings describing the addresses of the resource """ def __init__(self, name, addresses): diff --git a/python/pyspark/resourceprofile.py b/python/pyspark/resourceprofile.py new file mode 100644 index 0000000000000..62e8d70590f21 --- /dev/null +++ b/python/pyspark/resourceprofile.py @@ -0,0 +1,53 @@ + +# 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. +# + +from pyspark.executorresourcerequest import ExecutorResourceRequest +from pyspark.taskresourcerequest import TaskResourceRequest + + +class ResourceProfile(object): + + """ + .. note:: Evolving + + Resource profile to associate with an RDD. A ResourceProfile allows the user to + specify executor and task requirements for an RDD that will get applied during a + stage. This allows the user to change the resource requirements between stages. + This is meant to be immutable so user doesn't change it after building. + """ + + def __init__(self, _jResourceProfile): + self._jResourceProfile = _jResourceProfile + + @property + def taskResources(self): + taskRes = self._jResourceProfile.taskResourcesJMap() + result = {} + # convert back to python TaskResourceRequest + for k, v in taskRes.items(): + result[k] = TaskResourceRequest(v.resourceName(), v.amount()) + return result + + @property + def executorResources(self): + execRes = self._jResourceProfile.executorResourcesJMap() + result = {} + # convert back to python ExecutorResourceRequest + for k, v in execRes.items(): + result[k] = ExecutorResourceRequest(v.resourceName(), v.amount(), + v.discoveryScript(), v.vendor()) + return result diff --git a/python/pyspark/resourceprofilebuilder.py b/python/pyspark/resourceprofilebuilder.py new file mode 100644 index 0000000000000..2fe3eca1a9b62 --- /dev/null +++ b/python/pyspark/resourceprofilebuilder.py @@ -0,0 +1,76 @@ +# +# 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. +# + +from pyspark.executorresourcerequest import ExecutorResourceRequest +from pyspark.resourceprofile import ResourceProfile +from pyspark.taskresourcerequest import TaskResourceRequest +from pyspark.taskresourcerequests import TaskResourceRequests + + +class ResourceProfileBuilder(object): + + """ + .. note:: Evolving + + Resource profile Builder to build a resource profile to associate with an RDD. + A ResourceProfile allows the user to specify executor and task requirements for + an RDD that will get applied during a stage. This allows the user to change the + resource requirements between stages. + """ + + def __init__(self, ): + """Create a new ResourceProfileBuilder that wraps the underlying JVM object.""" + from pyspark.context import SparkContext + self._jResourceProfileBuilder \ + = SparkContext._jvm.org.apache.spark.resource.ResourceProfileBuilder() + + def require(self, resourceRequest): + if isinstance(resourceRequest, TaskResourceRequests): + self._jResourceProfileBuilder.require(resourceRequest._javaTaskResourceRequests) + else: + self._jResourceProfileBuilder.require(resourceRequest._javaExecutorResourceRequests) + return self + + def clearExecutorResourceRequests(self): + self._jResourceProfileBuilder.clearExecutorResourceRequests() + + def clearTaskResourceRequests(self): + self._jResourceProfileBuilder.clearTaskResourceRequests() + + @property + def taskResources(self): + taskRes = self._jResourceProfileBuilder.taskResourcesJMap() + result = {} + # convert back to python TaskResourceRequest + for k, v in taskRes.items(): + result[k] = TaskResourceRequest(v.resourceName(), v.amount()) + return result + + @property + def executorResources(self): + execRes = self._jResourceProfileBuilder.executorResourcesJMap() + result = {} + # convert back to python ExecutorResourceRequest + for k, v in execRes.items(): + result[k] = ExecutorResourceRequest(v.resourceName(), v.amount(), + v.discoveryScript(), v.vendor()) + return result + + @property + def build(self): + jresourceProfile = self._jResourceProfileBuilder.build() + return ResourceProfile(jresourceProfile) diff --git a/python/pyspark/taskresourcerequest.py b/python/pyspark/taskresourcerequest.py new file mode 100644 index 0000000000000..784416ea57056 --- /dev/null +++ b/python/pyspark/taskresourcerequest.py @@ -0,0 +1,45 @@ +# +# 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. +# + + +class TaskResourceRequest(object): + """ + .. note:: Evolving + + A task resource request. 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. The amount is specified as a Double to allow for saying you want more then + 1 task per resource. Valid values are less than or equal to 0.5 or whole numbers. + Use TaskResourceRequests class as a convenience API. + + :param resourceName: Name of the resource + :param amount: Amount requesting as a Double to support fractional resource requests. + Valid values are less than or equal to 0.5 or whole numbers. + """ + def __init__(self, resourceName, amount): + """Create a new TaskResourceRequest that wraps the underlying JVM object.""" + from pyspark.context import SparkContext + self._jTaskRequest = SparkContext._jvm.org.apache.spark.resource.TaskResourceRequest( + resourceName, amount) + + @property + def resourceName(self): + return self._jTaskRequest.resourceName() + + @property + def amount(self): + return self._jTaskRequest.amount() diff --git a/python/pyspark/taskresourcerequests.py b/python/pyspark/taskresourcerequests.py new file mode 100644 index 0000000000000..76cc177c1db90 --- /dev/null +++ b/python/pyspark/taskresourcerequests.py @@ -0,0 +1,52 @@ +# +# 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. +# + +from pyspark.taskresourcerequest import TaskResourceRequest + + +class TaskResourceRequests(object): + + """ + .. note:: Evolving + + A set of task resource requests. This is used in conjuntion with the + ResourceProfileBuilder to programmatically specify the resources needed for + an RDD that will be applied at the stage level. + """ + + def __init__(self): + """Create a new TaskResourceRequests that wraps the underlying JVM object.""" + from pyspark import SparkContext + self._javaTaskResourceRequests \ + = SparkContext._jvm.org.apache.spark.resource.TaskResourceRequests() + + def cpus(self, amount): + self._javaTaskResourceRequests.cpus(amount) + return self + + def resource(self, resourceName, amount): + self._javaTaskResourceRequests.resource(resourceName, float(amount)) + return self + + @property + def requests(self): + taskRes = self._javaTaskResourceRequests.requestsJMap() + result = {} + # convert back to python TaskResourceRequest + for k, v in taskRes.items(): + result[k] = TaskResourceRequest(v.resourceName(), v.amount()) + return result diff --git a/python/pyspark/tests/test_context.py b/python/pyspark/tests/test_context.py index c7f435a582210..edfea42bed71d 100644 --- a/python/pyspark/tests/test_context.py +++ b/python/pyspark/tests/test_context.py @@ -275,9 +275,13 @@ def setUp(self): self.tempFile = tempfile.NamedTemporaryFile(delete=False) self.tempFile.write(b'echo {\\"name\\": \\"gpu\\", \\"addresses\\": [\\"0\\"]}') self.tempFile.close() + # create temporary directory for Worker resources coordination + self.tempdir = tempfile.NamedTemporaryFile(delete=False) + os.unlink(self.tempdir.name) os.chmod(self.tempFile.name, stat.S_IRWXU | stat.S_IXGRP | stat.S_IRGRP | stat.S_IROTH | stat.S_IXOTH) conf = SparkConf().set("spark.test.home", SPARK_HOME) + conf = conf.set("spark.resources.dir", self.tempdir.name) conf = conf.set("spark.driver.resource.gpu.amount", "1") conf = conf.set("spark.driver.resource.gpu.discoveryScript", self.tempFile.name) self.sc = SparkContext('local-cluster[2,1,1024]', class_name, conf=conf) @@ -292,6 +296,7 @@ def test_resources(self): def tearDown(self): os.unlink(self.tempFile.name) + shutil.rmtree(self.tempdir.name) self.sc.stop() diff --git a/python/pyspark/tests/test_rdd.py b/python/pyspark/tests/test_rdd.py index 15cc48ae2153d..10f53841fb69e 100644 --- a/python/pyspark/tests/test_rdd.py +++ b/python/pyspark/tests/test_rdd.py @@ -25,7 +25,8 @@ from py4j.protocol import Py4JJavaError -from pyspark import shuffle, RDD +from pyspark import ExecutorResourceRequests, shuffle, RDD, ResourceProfile,\ + ResourceProfileBuilder, TaskResourceRequests from pyspark.serializers import CloudPickleSerializer, BatchedSerializer, PickleSerializer,\ MarshalSerializer, UTF8Deserializer, NoOpSerializer from pyspark.testing.utils import ReusedPySparkTestCase, SPARK_HOME, QuietTest @@ -783,6 +784,32 @@ def fail_last(x): for i in range(4): self.assertEqual(i, next(it)) + def test_resourceprofile(self): + rp_builder = ResourceProfileBuilder() + ereqs = ExecutorResourceRequests().cores(2).memory("6g").memoryOverhead("1g") + ereqs.pysparkMemory("2g").resource("gpu", 2, "testGpus", "nvidia.com") + treqs = TaskResourceRequests().cpus(2).resource("gpu", 2) + + def assert_request_contents(exec_reqs, task_reqs): + self.assertEqual(len(exec_reqs), 5) + self.assertEqual(exec_reqs["cores"].amount, 2) + self.assertEqual(exec_reqs["memory"].amount, 6144) + self.assertEqual(exec_reqs["memoryOverhead"].amount, 1024) + self.assertEqual(exec_reqs["pyspark.memory"].amount, 2048) + self.assertEqual(exec_reqs["gpu"].amount, 2) + self.assertEqual(exec_reqs["gpu"].discoveryScript, "testGpus") + self.assertEqual(exec_reqs["gpu"].resourceName, "gpu") + self.assertEqual(exec_reqs["gpu"].vendor, "nvidia.com") + self.assertEqual(len(task_reqs), 2) + self.assertEqual(task_reqs["cpus"].amount, 2.0) + self.assertEqual(task_reqs["gpu"].amount, 2.0) + + assert_request_contents(ereqs.requests, treqs.requests) + rp = rp_builder.require(ereqs).require(treqs).build + assert_request_contents(rp.executorResources, rp.taskResources) + rdd = self.sc.parallelize(range(10)).withResources(rp) + return_rp = rdd.getResourceProfile() + assert_request_contents(return_rp.executorResources, return_rp.taskResources) if __name__ == "__main__": import unittest diff --git a/python/pyspark/tests/test_taskcontext.py b/python/pyspark/tests/test_taskcontext.py index 6095a384679af..04f1a8ec3dbf8 100644 --- a/python/pyspark/tests/test_taskcontext.py +++ b/python/pyspark/tests/test_taskcontext.py @@ -16,6 +16,7 @@ # import os import random +import shutil import stat import sys import tempfile @@ -277,9 +278,13 @@ def setUp(self): self.tempFile = tempfile.NamedTemporaryFile(delete=False) self.tempFile.write(b'echo {\\"name\\": \\"gpu\\", \\"addresses\\": [\\"0\\"]}') self.tempFile.close() + # create temporary directory for Worker resources coordination + self.tempdir = tempfile.NamedTemporaryFile(delete=False) + os.unlink(self.tempdir.name) os.chmod(self.tempFile.name, stat.S_IRWXU | stat.S_IXGRP | stat.S_IRGRP | stat.S_IROTH | stat.S_IXOTH) conf = SparkConf().set("spark.test.home", SPARK_HOME) + conf = conf.set("spark.resources.dir", self.tempdir.name) conf = conf.set("spark.worker.resource.gpu.discoveryScript", self.tempFile.name) conf = conf.set("spark.worker.resource.gpu.amount", 1) conf = conf.set("spark.task.resource.gpu.amount", "1") @@ -297,6 +302,7 @@ def test_resources(self): def tearDown(self): os.unlink(self.tempFile.name) + shutil.rmtree(self.tempdir.name) self.sc.stop() if __name__ == "__main__": diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index b9d7a7083f41a..1797dee292fa6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -27,6 +27,7 @@ import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkKubern import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.Logging +import org.apache.spark.resource.ResourceProfileManager import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} import org.apache.spark.util.{SystemClock, ThreadUtils} @@ -34,7 +35,9 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit override def canCreate(masterURL: String): Boolean = masterURL.startsWith("k8s") - override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = { + override def createTaskScheduler( + sc: SparkContext, + masterURL: String): TaskScheduler = { new TaskSchedulerImpl(sc) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 105841ac834b3..16d0ac68560cb 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -27,6 +27,7 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.config.SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO +import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc.RpcAddress import org.apache.spark.scheduler.{ExecutorKilled, ExecutorLossReason, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SchedulerBackendUtils} @@ -116,8 +117,10 @@ private[spark] class KubernetesClusterSchedulerBackend( } } - override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { - podAllocator.setTotalExpectedExecutors(requestedTotal) + override def doRequestTotalExecutors( + resourceProfileToTotalExecs: Map[ResourceProfile, Int]): Future[Boolean] = { + podAllocator.setTotalExpectedExecutors( + resourceProfileToTotalExecs(scheduler.sc.resourceProfileManager.defaultResourceProfile)) Future.successful(true) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 7e1e39c85a183..4d764baa83f8b 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -26,13 +26,14 @@ import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} import org.mockito.ArgumentMatchers.{any, eq => mockitoEq} import org.mockito.Mockito.{mock, never, spy, verify, when} import org.scalatest.BeforeAndAfter - import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkFunSuite} + import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.Fabric8Aliases._ +import org.apache.spark.resource.ResourceProfileManager import org.apache.spark.rpc.{RpcEndpoint, RpcEndpointRef, RpcEnv} -import org.apache.spark.scheduler.{ExecutorKilled, TaskSchedulerImpl} +import org.apache.spark.scheduler.{ExecutorKilled, LiveListenerBus, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RemoveExecutor import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.cluster.k8s.ExecutorLifecycleTestUtils.TEST_SPARK_APP_ID @@ -86,10 +87,14 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn private var driverEndpoint: ArgumentCaptor[RpcEndpoint] = _ private var schedulerBackendUnderTest: KubernetesClusterSchedulerBackend = _ + private val listenerBus = new LiveListenerBus(new SparkConf()) + private val resourceProfileManager = new ResourceProfileManager(sparkConf, listenerBus) + before { MockitoAnnotations.initMocks(this) when(taskScheduler.sc).thenReturn(sc) when(sc.conf).thenReturn(sparkConf) + when(sc.resourceProfileManager).thenReturn(resourceProfileManager) when(sc.env).thenReturn(env) when(env.rpcEnv).thenReturn(rpcEnv) driverEndpoint = ArgumentCaptor.forClass(classOf[RpcEndpoint]) diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala index ed1b3d7a16739..9a62c3854cb15 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler.cluster.mesos import org.apache.spark.SparkContext import org.apache.spark.deploy.mesos.config._ import org.apache.spark.internal.config._ +import org.apache.spark.resource.ResourceProfileManager import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} /** @@ -32,7 +33,9 @@ private[spark] class MesosClusterManager extends ExternalClusterManager { masterURL.startsWith("mesos") } - override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = { + override def createTaskScheduler( + sc: SparkContext, + masterURL: String): TaskScheduler = { new TaskSchedulerImpl(sc) } diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index e916125ffdb67..8fa870fff9bca 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -38,6 +38,7 @@ import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.mesos.MesosExternalBlockStoreClient +import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc.{RpcEndpointAddress, RpcEndpointRef} import org.apache.spark.scheduler.{SlaveLost, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend @@ -595,13 +596,17 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } private def satisfiesLocality(offerHostname: String): Boolean = { + val hostToLocalTaskCount = + rpHostToLocalTaskCount.getOrElse(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID, + Map.empty) if (!Utils.isDynamicAllocationEnabled(conf) || hostToLocalTaskCount.isEmpty) { return true } // Check the locality information val currentHosts = slaves.values.filter(_.taskIDs.nonEmpty).map(_.hostname).toSet - val allDesiredHosts = hostToLocalTaskCount.keys.toSet + val allDesiredHosts = hostToLocalTaskCount.map { case (k, v) => k }.toSet + // Try to match locality for hosts which do not have executors yet, to potentially // increase coverage. val remainingHosts = allDesiredHosts -- currentHosts @@ -759,11 +764,15 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( super.applicationId } - override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future.successful { + override def doRequestTotalExecutors( + resourceProfileToTotalExecs: Map[ResourceProfile, Int] + ): Future[Boolean] = Future.successful { // We don't truly know if we can fulfill the full amount of executors // since at coarse grain it depends on the amount of slaves available. - logInfo("Capping the total amount of executors to " + requestedTotal) - executorLimitOption = Some(requestedTotal) + val rp = sc.resourceProfileManager.defaultResourceProfile + val numExecs = resourceProfileToTotalExecs.getOrElse(rp, 0) + logInfo("Capping the total amount of executors to " + numExecs) + executorLimitOption = Some(numExecs) // Update the locality wait start time to continue trying for locality. localityWaitStartTimeNs = System.nanoTime() true diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala index e2a99148dd799..f1e3fcab7e6af 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala @@ -31,6 +31,7 @@ import org.apache.spark.{SparkContext, SparkException, TaskState} import org.apache.spark.deploy.mesos.{config => mesosConfig} import org.apache.spark.executor.MesosExecutorBackend import org.apache.spark.internal.config +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.Utils @@ -457,7 +458,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( super.applicationId } - override def maxNumConcurrentTasks(): Int = { + override def maxNumConcurrentTasks(rp: ResourceProfile): Int = { // TODO SPARK-25074 support this method for MesosFineGrainedSchedulerBackend 0 } diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala index 1876861700fc0..3e8be24794b33 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -71,8 +71,10 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite offerResources(offers) verifyTaskLaunched(driver, "o1") + val totalExecs = Map(ResourceProfile.getOrCreateDefaultProfile(sparkConf) -> 0) // kills executors - assert(backend.doRequestTotalExecutors(0).futureValue) + val defaultResourceProfile = ResourceProfile.getOrCreateDefaultProfile(sparkConf) + assert(backend.doRequestTotalExecutors(Map(defaultResourceProfile -> 0)).futureValue) assert(backend.doKillExecutors(Seq("0")).futureValue) val taskID0 = createTaskId("0") verify(driver, times(1)).killTask(taskID0) @@ -82,7 +84,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite verifyDeclinedOffer(driver, createOfferId("o2")) // Launches a new task when requested executors is positive - backend.doRequestTotalExecutors(2) + backend.doRequestTotalExecutors(Map(defaultResourceProfile -> 2)) offerResources(offers, 2) verifyTaskLaunched(driver, "o2") } @@ -635,7 +637,11 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite assert(backend.getExecutorIds().isEmpty) - backend.requestTotalExecutors(2, 2, Map("hosts10" -> 1, "hosts11" -> 1)) + val defaultProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID + val defaultProf = ResourceProfile.getOrCreateDefaultProfile(sparkConf) + backend.requestTotalExecutors(Map(defaultProfileId -> 2), + Map(defaultProfileId -> Map("hosts10" -> 1, "hosts11" -> 1)), + Map(defaultProfileId -> 2)) // Offer non-local resources, which should be rejected offerResourcesAndVerify(1, false) @@ -651,7 +657,10 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite offerResourcesAndVerify(1, true) // Update total executors - backend.requestTotalExecutors(3, 3, Map("hosts10" -> 1, "hosts11" -> 1, "hosts12" -> 1)) + backend.requestTotalExecutors(Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 2), + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> + Map("hosts10" -> 1, "hosts11" -> 1, "hosts12" -> 1)), + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 3)) // Offer non-local resources, which should be rejected offerResourcesAndVerify(3, false) @@ -660,8 +669,10 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite Thread.sleep(2000) // Update total executors - backend.requestTotalExecutors(4, 4, Map("hosts10" -> 1, "hosts11" -> 1, "hosts12" -> 1, - "hosts13" -> 1)) + backend.requestTotalExecutors(Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 4), + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> + Map("hosts10" -> 1, "hosts11" -> 1, "hosts12" -> 1, "hosts13" -> 1)), + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 4)) // Offer non-local resources, which should be rejected offerResourcesAndVerify(3, false) diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala index a5bd34888a0a6..5fab9de63933d 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala @@ -262,6 +262,7 @@ class MesosFineGrainedSchedulerBackendSuite name = "n1", index = 0, partitionId = 0, + cpus = 1, addedFiles = mutable.Map.empty[String, Long], addedJars = mutable.Map.empty[String, Long], properties = new Properties(), @@ -373,6 +374,7 @@ class MesosFineGrainedSchedulerBackendSuite name = "n1", index = 0, partitionId = 0, + cpus = 1, addedFiles = mutable.Map.empty[String, Long], addedJars = mutable.Map.empty[String, Long], properties = new Properties(), diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 1e8f4084ef9c7..43cd7458ef55b 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -593,7 +593,7 @@ private[spark] class ApplicationMaster( } } try { - val numPendingAllocate = allocator.getPendingAllocate.size + val numPendingAllocate = allocator.getNumContainersPendingAllocate var sleepStartNs = 0L var sleepInterval = 200L // ms allocatorLock.synchronized { @@ -778,8 +778,11 @@ private[spark] class ApplicationMaster( case r: RequestExecutors => Option(allocator) match { case Some(a) => - if (a.requestTotalExecutorsWithPreferredLocalities(r.requestedTotal, - r.localityAwareTasks, r.hostToLocalTaskCount, r.nodeBlacklist)) { + if (a.requestTotalExecutorsWithPreferredLocalities( + r.resourceProfileToTotalExecs, + r.numLocalityAwareTasksPerResourceProfileId, + r.hostToLocalTaskCount, + r.nodeBlacklist)) { resetAllocatorInterval() } context.reply(true) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterSource.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterSource.scala index 0fec916582602..62ac17cff169a 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterSource.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterSource.scala @@ -40,11 +40,11 @@ private[spark] class ApplicationMasterSource(prefix: String, yarnAllocator: Yarn }) metricRegistry.register(MetricRegistry.name("numLocalityAwareTasks"), new Gauge[Int] { - override def getValue: Int = yarnAllocator.numLocalityAwareTasks + override def getValue: Int = yarnAllocator.getNumLocalityAwareTasks }) metricRegistry.register(MetricRegistry.name("numContainersPendingAllocate"), new Gauge[Int] { - override def getValue: Int = yarnAllocator.numContainersPendingAllocate + override def getValue: Int = yarnAllocator.getNumContainersPendingAllocate }) } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala index 2288bb55d8b47..0212fecaadfb3 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.yarn.api.records.{ContainerId, Resource} import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.spark.SparkConf -import org.apache.spark.internal.config._ +import org.apache.spark.resource.ResourceProfile private[yarn] case class ContainerLocalityPreferences(nodes: Array[String], racks: Array[String]) @@ -82,7 +82,6 @@ private[yarn] case class ContainerLocalityPreferences(nodes: Array[String], rack private[yarn] class LocalityPreferredContainerPlacementStrategy( val sparkConf: SparkConf, val yarnConf: Configuration, - val resource: Resource, resolver: SparkRackResolver) { /** @@ -104,11 +103,13 @@ private[yarn] class LocalityPreferredContainerPlacementStrategy( numLocalityAwareTasks: Int, hostToLocalTaskCount: Map[String, Int], allocatedHostToContainersMap: HashMap[String, Set[ContainerId]], - localityMatchedPendingAllocations: Seq[ContainerRequest] + localityMatchedPendingAllocations: Seq[ContainerRequest], + resource: Resource, + rp: ResourceProfile ): Array[ContainerLocalityPreferences] = { val updatedHostToContainerCount = expectedHostToContainerCount( numLocalityAwareTasks, hostToLocalTaskCount, allocatedHostToContainersMap, - localityMatchedPendingAllocations) + localityMatchedPendingAllocations, resource, rp) val updatedLocalityAwareContainerNum = updatedHostToContainerCount.values.sum // The number of containers to allocate, divided into two groups, one with preferred locality, @@ -152,11 +153,14 @@ private[yarn] class LocalityPreferredContainerPlacementStrategy( } /** - * Calculate the number of executors need to satisfy the given number of pending tasks. + * Calculate the number of executors needed to satisfy the given number of pending tasks. */ - private def numExecutorsPending(numTasksPending: Int): Int = { - val coresPerExecutor = resource.getVirtualCores - (numTasksPending * sparkConf.get(CPUS_PER_TASK) + coresPerExecutor - 1) / coresPerExecutor + private def numExecutorsPending( + numTasksPending: Int, + resource: Resource, + rp: ResourceProfile): Int = { + val tasksPerExec = rp.maxTasksPerExecutor(sparkConf) + math.ceil(numTasksPending / tasksPerExec.toDouble).toInt } /** @@ -175,14 +179,16 @@ private[yarn] class LocalityPreferredContainerPlacementStrategy( localityAwareTasks: Int, hostToLocalTaskCount: Map[String, Int], allocatedHostToContainersMap: HashMap[String, Set[ContainerId]], - localityMatchedPendingAllocations: Seq[ContainerRequest] + localityMatchedPendingAllocations: Seq[ContainerRequest], + resource: Resource, + rp: ResourceProfile ): Map[String, Int] = { val totalLocalTaskNum = hostToLocalTaskCount.values.sum val pendingHostToContainersMap = pendingHostToContainerCount(localityMatchedPendingAllocations) hostToLocalTaskCount.map { case (host, count) => val expectedCount = - count.toDouble * numExecutorsPending(localityAwareTasks) / totalLocalTaskNum + count.toDouble * numExecutorsPending(localityAwareTasks, resource, rp) / totalLocalTaskNum // Take the locality of pending containers into consideration val existedCount = allocatedHostToContainersMap.get(host).map(_.size).getOrElse(0) + pendingHostToContainersMap.getOrElse(host, 0.0) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala index f524962141455..3799883526764 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala @@ -208,8 +208,19 @@ private object ResourceRequestHelper extends Logging { resourceInformation } + def isYarnCustomResourcesNonEmpty(resource: Resource): Boolean = { + try { + // Use reflection as this uses APIs only available in Hadoop 3 + val getResourcesMethod = resource.getClass().getMethod("getResources") + val resources = getResourcesMethod.invoke(resource).asInstanceOf[Array[Any]] + if (resources.nonEmpty) true else false + } catch { + case _: NoSuchMethodException => false + } + } + /** - * Checks whether Hadoop 2.x or 3 is used as a dependency. + * Checks whether Hadoop 2.x or jj3 is used as a dependency. * In case of Hadoop 3 and later, the ResourceInformation class * should be available on the classpath. */ diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 09414cbbe50a4..a7785e6447e69 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -75,7 +75,11 @@ private[yarn] class YarnAllocator( import YarnAllocator._ // Visible for testing. - val allocatedHostToContainersMap = new HashMap[String, collection.mutable.Set[ContainerId]] + val allocatedHostToContainersMapPerRPId = + new HashMap[Int, HashMap[String, collection.mutable.Set[ContainerId]]] + allocatedHostToContainersMapPerRPId(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) = + new HashMap[String, mutable.Set[ContainerId]]() + val allocatedContainerToHostMap = new HashMap[ContainerId, String] // Containers that we no longer care about. We've either already told the RM to release them or @@ -84,10 +88,14 @@ private[yarn] class YarnAllocator( private val releasedContainers = Collections.newSetFromMap[ContainerId]( new ConcurrentHashMap[ContainerId, java.lang.Boolean]) - private val runningExecutors = Collections.newSetFromMap[String]( - new ConcurrentHashMap[String, java.lang.Boolean]()) + private val runningExecutorsPerResourceProfileId = + new ConcurrentHashMap[Int, java.util.Set[String]]() + runningExecutorsPerResourceProfileId.put(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID, + Collections.newSetFromMap[String](new ConcurrentHashMap[String, java.lang.Boolean]())) - private val numExecutorsStarting = new AtomicInteger(0) + private val numExecutorsStartingPerResourceProfileId = new HashMap[Int, AtomicInteger] + numExecutorsStartingPerResourceProfileId(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) = + new AtomicInteger(0) /** * Used to generate a unique ID per executor @@ -110,10 +118,10 @@ private[yarn] class YarnAllocator( private val allocatorBlacklistTracker = new YarnAllocatorBlacklistTracker(sparkConf, amClient, failureTracker) - @volatile private var targetNumExecutors = + private val targetNumExecutorsPerResourceProfileId = new mutable.HashMap[Int, Int] + targetNumExecutorsPerResourceProfileId(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) = SchedulerBackendUtils.getInitialTargetExecutorNumber(sparkConf) - // Executor loss reason requests that are pending - maps from executor ID for inquiry to a // list of requesters that should be responded to once we find out why the given executor // was lost. @@ -129,6 +137,7 @@ private[yarn] class YarnAllocator( private var numUnexpectedContainerRelease = 0L private val containerIdToExecutorId = new HashMap[ContainerId, String] + private val containerIdToResourceProfileId = new HashMap[ContainerId, Int] // Executor memory in MiB. protected val executorMemory = sparkConf.get(EXECUTOR_MEMORY).toInt @@ -143,7 +152,7 @@ private[yarn] class YarnAllocator( 0 } // Number of cores per executor. - protected val executorCores = sparkConf.get(EXECUTOR_CORES) + protected val defaultExecutorCores = sparkConf.get(EXECUTOR_CORES) private val executorResourceRequests = getYarnResourcesAndAmounts(sparkConf, config.YARN_EXECUTOR_RESOURCE_TYPES_PREFIX) ++ @@ -151,13 +160,22 @@ private[yarn] class YarnAllocator( // Resource capability requested for each executor private[yarn] val resource: Resource = { - val resource = Resource.newInstance( - executorMemory + executorOffHeapMemory + memoryOverhead + pysparkWorkerMemory, executorCores) + val resource: Resource = Resource.newInstance( + executorMemory + executorOffHeapMemory + memoryOverhead + pysparkWorkerMemory, + defaultExecutorCores) ResourceRequestHelper.setResourceRequests(executorResourceRequests, resource) logDebug(s"Created resource capability: $resource") resource } + private[yarn] val rpIdToYarnResource = new mutable.HashMap[Int, Resource] + rpIdToYarnResource(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) = resource + + // note currently we don't remove ResourceProfiles + private[yarn] val rpIdToResourceProfile = new mutable.HashMap[Int, ResourceProfile] + rpIdToResourceProfile(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) = + ResourceProfile.getOrCreateDefaultProfile(sparkConf) + private val launcherPool = ThreadUtils.newDaemonCachedThreadPool( "ContainerLauncher", sparkConf.get(CONTAINER_LAUNCH_MAX_THREADS)) @@ -166,17 +184,30 @@ private[yarn] class YarnAllocator( private val labelExpression = sparkConf.get(EXECUTOR_NODE_LABEL_EXPRESSION) - // A map to store preferred hostname and possible task numbers running on it. - private var hostToLocalTaskCounts: Map[String, Int] = Map.empty + // A map of ResourceProfile id to a map of preferred hostname and possible + // task numbers running on it. + private var hostToLocalTaskCountPerResourceProfileId: Map[Int, Map[String, Int]] = + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> Map.empty) - // Number of tasks that have locality preferences in active stages - private[yarn] var numLocalityAwareTasks: Int = 0 + // ResourceProfile Id to number of tasks that have locality preferences in active stages + private[yarn] var numLocalityAwareTasksPerResourceProfileId: Map[Int, Int] = + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 0) // A container placement strategy based on pending tasks' locality preference private[yarn] val containerPlacementStrategy = - new LocalityPreferredContainerPlacementStrategy(sparkConf, conf, resource, resolver) + new LocalityPreferredContainerPlacementStrategy(sparkConf, conf, resolver) - def getNumExecutorsRunning: Int = runningExecutors.size() + def getNumExecutorsRunning: Int = { + runningExecutorsPerResourceProfileId.values.asScala.map(_.size).sum + } + + def getNumLocalityAwareTasks: Int = { + numLocalityAwareTasksPerResourceProfileId.values.sum + } + + def getNumExecutorsStarting: Int = { + numExecutorsStartingPerResourceProfileId.values.map(_.get()).sum + } def getNumReleasedContainers: Int = releasedContainers.size() @@ -186,49 +217,145 @@ private[yarn] class YarnAllocator( /** * A sequence of pending container requests that have not yet been fulfilled. + * ResourceProfile id -> pendingAllocate container request */ - def getPendingAllocate: Seq[ContainerRequest] = getPendingAtLocation(ANY_HOST) + def getPendingAllocate: Map[Int, Seq[ContainerRequest]] = getPendingAtLocation(ANY_HOST) + + def getNumContainersPendingAllocate: Int = synchronized { + getPendingAllocate.values.flatten.size + } + + // YARN priorities are such that lower number is higher priority. + // We need to allocate a different priority for each ResourceProfile because YARN + // won't allow different container resource requirements within a Priority. + // We could allocate per Stage to make sure earlier stages get priority but Spark + // always finishes a stage before starting a later one and if we have 2 running in parallel + // I don't think the priority matters. + private def getContainerPriority(resourceProfileId: Int): Priority = { + Priority.newInstance(resourceProfileId) + } + + // The ResourceProfile id is the priority + private def getResourceProfileIdFromPriority(priority: Int): Int = { + priority + } - def numContainersPendingAllocate: Int = synchronized { - getPendingAllocate.size + private def getOrUpdateAllocatedHostToContainersMapForRPId( + rpId: Int): HashMap[String, collection.mutable.Set[ContainerId]] = { + allocatedHostToContainersMapPerRPId.getOrElseUpdate(rpId, + new HashMap[String, mutable.Set[ContainerId]]()) + } + + private def getOrUpdateRunningExecutorForRPId(rpId: Int): java.util.Set[String] = { + runningExecutorsPerResourceProfileId.putIfAbsent(rpId, + Collections.newSetFromMap[String](new ConcurrentHashMap[String, java.lang.Boolean]())) + } + + private def getOrUpdateNumExecutorsStartingForRPId(rpId: Int): AtomicInteger = { + numExecutorsStartingPerResourceProfileId.getOrElseUpdate(rpId, new AtomicInteger(0)) + } + + private def getTargetNumExecutorsForRPId(rpId: Int): Int = { + targetNumExecutorsPerResourceProfileId.getOrElseUpdate(rpId, + SchedulerBackendUtils.getInitialTargetExecutorNumber(sparkConf)) } /** - * A sequence of pending container requests at the given location that have not yet been - * fulfilled. + * A sequence of pending container requests at the given location for each ResourceProfile id + * that have not yet been fulfilled. */ - private def getPendingAtLocation(location: String): Seq[ContainerRequest] = - amClient.getMatchingRequests(RM_REQUEST_PRIORITY, location, resource).asScala - .flatMap(_.asScala) + private def getPendingAtLocation(location: String): Map[Int, Seq[ContainerRequest]] = { + val allContainerRequests = new mutable.HashMap[Int, Seq[ContainerRequest]] + rpIdToYarnResource.map { case (id, profResource) => + val result = amClient.getMatchingRequests(getContainerPriority(id), location, profResource) + .asScala.flatMap(_.asScala) + allContainerRequests(id) = result + } + allContainerRequests.toMap + } + + // if a ResourceProfile hasn't been seen yet, create the corresponding YARN Resource for it + private def createYarnResourceForResourceProfile( + resourceProfileToTotalExecs: Map[ResourceProfile, Int]): Unit = { + resourceProfileToTotalExecs.foreach { case (rp, num) => + if (!rpIdToYarnResource.contains(rp.id)) { + // Start with the application or default settings + var heapMem = executorMemory.toLong + // TODO - do we want to add to ExecutorResource? + var offHeapMem = executorOffHeapMemory.toLong + var overheadMem = memoryOverhead.toLong + var pysparkMem = pysparkWorkerMemory.toLong + var cores = defaultExecutorCores + val customResources = new mutable.HashMap[String, String] + // track the resource profile if not already there + getOrUpdateRunningExecutorForRPId(rp.id) + logInfo(s"Resource profile ${rp.id} doesn't exist, adding it") + val execResources = rp.executorResources + execResources.foreach { case (r, execReq) => + r match { + case ResourceProfile.MEMORY => + heapMem = execReq.amount + case ResourceProfile.OVERHEAD_MEM => + overheadMem = execReq.amount + case ResourceProfile.PYSPARK_MEM => + pysparkMem = execReq.amount + case ResourceProfile.CORES => + cores = execReq.amount.toInt + case "gpu" => + customResources(YARN_GPU_RESOURCE_CONFIG) = execReq.amount.toString + case "fpga" => + customResources(YARN_FPGA_RESOURCE_CONFIG) = execReq.amount.toString + case rName => + customResources(rName) = execReq.amount.toString + } + } + val totalMem = (heapMem + offHeapMem + overheadMem + pysparkMem).toInt + val resource = Resource.newInstance(totalMem, cores) + ResourceRequestHelper.setResourceRequests(customResources.toMap, resource) + logDebug(s"Created resource capability: $resource") + rpIdToYarnResource(rp.id) = resource + rpIdToResourceProfile(rp.id) = rp + } + } + } /** * Request as many executors from the ResourceManager as needed to reach the desired total. If * the requested total is smaller than the current number of running executors, no executors will * be killed. - * @param requestedTotal total number of containers requested - * @param localityAwareTasks number of locality aware tasks to be used as container placement hint - * @param hostToLocalTaskCount a map of preferred hostname to possible task counts to be used as - * container placement hint. + * @param resourceProfileToTotalExecs total number of containers requested for each + * ResourceProfile + * @param numLocalityAwareTasksPerResourceProfileId number of locality aware tasks for each + * ResourceProfile id to be used as container + * placement hint. + * @param hostToLocalTaskCount a map of preferred hostname to possible task counts for each + * ResourceProfile id to be used as container placement hint. * @param nodeBlacklist blacklisted nodes, which is passed in to avoid allocating new containers * on them. It will be used to update the application master's blacklist. * @return Whether the new requested total is different than the old value. */ def requestTotalExecutorsWithPreferredLocalities( - requestedTotal: Int, - localityAwareTasks: Int, - hostToLocalTaskCount: Map[String, Int], + resourceProfileToTotalExecs: Map[ResourceProfile, Int], + numLocalityAwareTasksPerResourceProfileId: Map[Int, Int], + hostToLocalTaskCountPerResourceProfileId: Map[Int, Map[String, Int]], nodeBlacklist: Set[String]): Boolean = synchronized { - this.numLocalityAwareTasks = localityAwareTasks - this.hostToLocalTaskCounts = hostToLocalTaskCount - - if (requestedTotal != targetNumExecutors) { - logInfo(s"Driver requested a total number of $requestedTotal executor(s).") - targetNumExecutors = requestedTotal - allocatorBlacklistTracker.setSchedulerBlacklistedNodes(nodeBlacklist) - true - } else { - false + this.numLocalityAwareTasksPerResourceProfileId = numLocalityAwareTasksPerResourceProfileId + this.hostToLocalTaskCountPerResourceProfileId = hostToLocalTaskCountPerResourceProfileId + + createYarnResourceForResourceProfile(resourceProfileToTotalExecs) + + val res = resourceProfileToTotalExecs.map { case (rp, numExecs) => + if (numExecs != getTargetNumExecutorsForRPId(rp.id)) { + logInfo(s"Driver requested a total number of $numExecs executor(s) " + + s"for resource profile id: ${rp.id}.") + targetNumExecutorsPerResourceProfileId(rp.id) = numExecs + allocatorBlacklistTracker.setSchedulerBlacklistedNodes(nodeBlacklist) + true + } else { + false + } } + res.exists(_ == true) } /** @@ -237,8 +364,9 @@ private[yarn] class YarnAllocator( def killExecutor(executorId: String): Unit = synchronized { executorIdToContainer.get(executorId) match { case Some(container) if !releasedContainers.contains(container.getId) => + val rpId = containerIdToResourceProfileId(container.getId) internalReleaseContainer(container) - runningExecutors.remove(executorId) + getOrUpdateRunningExecutorForRPId(rpId).remove(executorId) case _ => logWarning(s"Attempted to kill unknown executor $executorId!") } } @@ -267,8 +395,8 @@ private[yarn] class YarnAllocator( "Launching executor count: %d. Cluster resources: %s.") .format( allocatedContainers.size, - runningExecutors.size, - numExecutorsStarting.get, + getNumExecutorsRunning, + getNumExecutorsStarting, allocateResponse.getAvailableResources)) handleAllocatedContainers(allocatedContainers.asScala) @@ -278,8 +406,7 @@ private[yarn] class YarnAllocator( if (completedContainers.size > 0) { logDebug("Completed %d containers".format(completedContainers.size)) processCompletedContainers(completedContainers.asScala) - logDebug("Finished processing %d completed containers. Current running executor count: %d." - .format(completedContainers.size, runningExecutors.size)) + logDebug("Finished processing %d completed containers.".format(completedContainers.size)) } } @@ -290,97 +417,110 @@ private[yarn] class YarnAllocator( * Visible for testing. */ def updateResourceRequests(): Unit = { - val pendingAllocate = getPendingAllocate - val numPendingAllocate = pendingAllocate.size - val missing = targetNumExecutors - numPendingAllocate - - numExecutorsStarting.get - runningExecutors.size - logDebug(s"Updating resource requests, target: $targetNumExecutors, " + - s"pending: $numPendingAllocate, running: ${runningExecutors.size}, " + - s"executorsStarting: ${numExecutorsStarting.get}") - - // Split the pending container request into three groups: locality matched list, locality - // unmatched list and non-locality list. Take the locality matched container request into - // consideration of container placement, treat as allocated containers. - // For locality unmatched and locality free container requests, cancel these container - // requests, since required locality preference has been changed, recalculating using - // container placement strategy. - val (localRequests, staleRequests, anyHostRequests) = splitPendingAllocationsByLocality( - hostToLocalTaskCounts, pendingAllocate) - - if (missing > 0) { - if (log.isInfoEnabled()) { - var requestContainerMessage = s"Will request $missing executor container(s), each with " + + + val pendingAllocatePerResourceProfileId = getPendingAllocate + val missingPerProfile = targetNumExecutorsPerResourceProfileId.map { case (rpId, num) => + val starting = getOrUpdateNumExecutorsStartingForRPId(rpId).get + val pending = pendingAllocatePerResourceProfileId.getOrElse(rpId, Seq.empty).size + val running = getOrUpdateRunningExecutorForRPId(rpId).size() + logDebug(s"Updating resource requests for ResourceProfile id: $rpId, target: $num, " + + s"pending: $pending, running: $running, executorsStarting: $starting") + (rpId, num - pending - running - starting) + }.toMap + + missingPerProfile.foreach { case (rpId, missing) => + // Split the pending container request into three groups: locality matched list, locality + // unmatched list and non-locality list. Take the locality matched container request into + // consideration of container placement, treat as allocated containers. + // For locality unmatched and locality free container requests, cancel these container + // requests, since required locality preference has been changed, recalculating using + // container placement strategy. + val hostToLocalTaskCount = + hostToLocalTaskCountPerResourceProfileId.getOrElse(rpId, Map.empty) + val pendingAllocate = pendingAllocatePerResourceProfileId.getOrElse(rpId, Seq.empty) + val (localRequests, staleRequests, anyHostRequests) = splitPendingAllocationsByLocality( + hostToLocalTaskCount, pendingAllocate) + val numPendingAllocate = pendingAllocate.size + if (missing > 0) { + val resource = rpIdToYarnResource(rpId) + if (log.isInfoEnabled()) { + var requestContainerMessage = s"Will request $missing executor container(s) for " + + s" ResourceProfile Id: $rpId, each with " + s"${resource.getVirtualCores} core(s) and " + s"${resource.getMemory} MB memory (including $memoryOverhead MB of overhead)" - if (ResourceRequestHelper.isYarnResourceTypesAvailable() && - executorResourceRequests.nonEmpty) { - requestContainerMessage ++= s" with custom resources: " + resource.toString + if (ResourceRequestHelper.isYarnResourceTypesAvailable() && + ResourceRequestHelper.isYarnCustomResourcesNonEmpty(resource)) { + requestContainerMessage ++= s" with custom resources: " + resource.toString + } + logInfo(requestContainerMessage) } - logInfo(requestContainerMessage) - } - // cancel "stale" requests for locations that are no longer needed - staleRequests.foreach { stale => - amClient.removeContainerRequest(stale) - } - val cancelledContainers = staleRequests.size - if (cancelledContainers > 0) { - logInfo(s"Canceled $cancelledContainers container request(s) (locality no longer needed)") - } - - // consider the number of new containers and cancelled stale containers available - val availableContainers = missing + cancelledContainers + // cancel "stale" requests for locations that are no longer needed + staleRequests.foreach { stale => + amClient.removeContainerRequest(stale) + } + val cancelledContainers = staleRequests.size + if (cancelledContainers > 0) { + logInfo(s"Canceled $cancelledContainers container request(s) (locality no longer needed)") + } - // to maximize locality, include requests with no locality preference that can be cancelled - val potentialContainers = availableContainers + anyHostRequests.size + // consider the number of new containers and cancelled stale containers available + val availableContainers = missing + cancelledContainers - val containerLocalityPreferences = containerPlacementStrategy.localityOfRequestedContainers( - potentialContainers, numLocalityAwareTasks, hostToLocalTaskCounts, - allocatedHostToContainersMap, localRequests) + // to maximize locality, include requests with no locality preference that can be cancelled + val potentialContainers = availableContainers + anyHostRequests.size - val newLocalityRequests = new mutable.ArrayBuffer[ContainerRequest] - containerLocalityPreferences.foreach { - case ContainerLocalityPreferences(nodes, racks) if nodes != null => - newLocalityRequests += createContainerRequest(resource, nodes, racks) - case _ => - } + val allocatedHostToContainer = getOrUpdateAllocatedHostToContainersMapForRPId(rpId) + val numLocalityAwareTasks = numLocalityAwareTasksPerResourceProfileId.getOrElse(rpId, 0) + val containerLocalityPreferences = containerPlacementStrategy.localityOfRequestedContainers( + potentialContainers, numLocalityAwareTasks, hostToLocalTaskCount, + allocatedHostToContainer, localRequests, resource, rpIdToResourceProfile(rpId)) - if (availableContainers >= newLocalityRequests.size) { - // more containers are available than needed for locality, fill in requests for any host - for (i <- 0 until (availableContainers - newLocalityRequests.size)) { - newLocalityRequests += createContainerRequest(resource, null, null) - } - } else { - val numToCancel = newLocalityRequests.size - availableContainers - // cancel some requests without locality preferences to schedule more local containers - anyHostRequests.slice(0, numToCancel).foreach { nonLocal => - amClient.removeContainerRequest(nonLocal) - } - if (numToCancel > 0) { - logInfo(s"Canceled $numToCancel unlocalized container requests to resubmit with locality") + val newLocalityRequests = new mutable.ArrayBuffer[ContainerRequest] + containerLocalityPreferences.foreach { + case ContainerLocalityPreferences(nodes, racks) if nodes != null => + newLocalityRequests += createContainerRequest(resource, nodes, racks, rpId) + case _ => } - } - newLocalityRequests.foreach { request => - amClient.addContainerRequest(request) - } + if (availableContainers >= newLocalityRequests.size) { + // more containers are available than needed for locality, fill in requests for any host + for (i <- 0 until (availableContainers - newLocalityRequests.size)) { + newLocalityRequests += createContainerRequest(resource, null, null, rpId) + } + } else { + val numToCancel = newLocalityRequests.size - availableContainers + // cancel some requests without locality preferences to schedule more local containers + anyHostRequests.slice(0, numToCancel).foreach { nonLocal => + amClient.removeContainerRequest(nonLocal) + } + if (numToCancel > 0) { + logInfo(s"Canceled $numToCancel unlocalized container requests to " + + s"resubmit with locality") + } + } - if (log.isInfoEnabled()) { - val (localized, anyHost) = newLocalityRequests.partition(_.getNodes() != null) - if (anyHost.nonEmpty) { - logInfo(s"Submitted ${anyHost.size} unlocalized container requests.") + newLocalityRequests.foreach { request => + amClient.addContainerRequest(request) } - localized.foreach { request => - logInfo(s"Submitted container request for host ${hostStr(request)}.") + + if (log.isInfoEnabled()) { + val (localized, anyHost) = newLocalityRequests.partition(_.getNodes() != null) + if (anyHost.nonEmpty) { + logInfo(s"Submitted ${anyHost.size} unlocalized container requests.") + } + localized.foreach { request => + logInfo(s"Submitted container request for host ${hostStr(request)}.") + } } + } else if (numPendingAllocate > 0 && missing < 0) { + val numToCancel = math.min(numPendingAllocate, -missing) + logInfo(s"Canceling requests for $numToCancel executor container(s) to have a new " + + s"desired total ${getTargetNumExecutorsForRPId(rpId)} executors.") + // cancel pending allocate requests by taking locality preference into account + val cancelRequests = (staleRequests ++ anyHostRequests ++ localRequests).take(numToCancel) + cancelRequests.foreach(amClient.removeContainerRequest) } - } else if (numPendingAllocate > 0 && missing < 0) { - val numToCancel = math.min(numPendingAllocate, -missing) - logInfo(s"Canceling requests for $numToCancel executor container(s) to have a new desired " + - s"total $targetNumExecutors executors.") - // cancel pending allocate requests by taking locality preference into account - val cancelRequests = (staleRequests ++ anyHostRequests ++ localRequests).take(numToCancel) - cancelRequests.foreach(amClient.removeContainerRequest) } } @@ -405,8 +545,10 @@ private[yarn] class YarnAllocator( private def createContainerRequest( resource: Resource, nodes: Array[String], - racks: Array[String]): ContainerRequest = { - new ContainerRequest(resource, nodes, racks, RM_REQUEST_PRIORITY, true, labelExpression.orNull) + racks: Array[String], + resourceProfileId: Int): ContainerRequest = { + new ContainerRequest(resource, nodes, racks, getContainerPriority(resourceProfileId), + true, labelExpression.orNull) } /** @@ -499,20 +641,20 @@ private[yarn] class YarnAllocator( location: String, containersToUse: ArrayBuffer[Container], remaining: ArrayBuffer[Container]): Unit = { + val rpId = getResourceProfileIdFromPriority(allocatedContainer.getPriority.getPriority()) // SPARK-6050: certain Yarn configurations return a virtual core count that doesn't match the // request; for example, capacity scheduler + DefaultResourceCalculator. So match on requested // memory, but use the asked vcore count for matching, effectively disabling matching on vcore // count. - val matchingResource = Resource.newInstance(allocatedContainer.getResource.getMemory, - resource.getVirtualCores) - ResourceRequestHelper.setResourceRequests(executorResourceRequests, matchingResource) + // this should be exactly what we requested + val resourceForRP = rpIdToYarnResource(rpId) logDebug(s"Calling amClient.getMatchingRequests with parameters: " + s"priority: ${allocatedContainer.getPriority}, " + - s"location: $location, resource: $matchingResource") + s"location: $location, resource: $resourceForRP") val matchingRequests = amClient.getMatchingRequests(allocatedContainer.getPriority, location, - matchingResource) + resourceForRP) // Match the allocation to a request if (!matchingRequests.isEmpty) { @@ -530,28 +672,37 @@ private[yarn] class YarnAllocator( */ private def runAllocatedContainers(containersToUse: ArrayBuffer[Container]): Unit = { for (container <- containersToUse) { + val rpId = getResourceProfileIdFromPriority(container.getPriority.getPriority()) executorIdCounter += 1 val executorHostname = container.getNodeId.getHost val containerId = container.getId val executorId = executorIdCounter.toString - assert(container.getResource.getMemory >= resource.getMemory) + val yarnResourceForRpId = rpIdToYarnResource(rpId) + assert(container.getResource.getMemory >= yarnResourceForRpId.getMemory) logInfo(s"Launching container $containerId on host $executorHostname " + - s"for executor with ID $executorId") + s"for executor with ID $executorId for ResourceProfile Id $rpId") def updateInternalState(): Unit = synchronized { - runningExecutors.add(executorId) - numExecutorsStarting.decrementAndGet() + getOrUpdateRunningExecutorForRPId(rpId).add(executorId) + getOrUpdateNumExecutorsStartingForRPId(rpId).decrementAndGet() executorIdToContainer(executorId) = container containerIdToExecutorId(container.getId) = executorId + containerIdToResourceProfileId(container.getId) = rpId - val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, + val localallocatedHostToContainersMap = getOrUpdateAllocatedHostToContainersMapForRPId(rpId) + val containerSet = localallocatedHostToContainersMap.getOrElseUpdate(executorHostname, new HashSet[ContainerId]) containerSet += containerId allocatedContainerToHostMap.put(containerId, executorHostname) } - if (runningExecutors.size() < targetNumExecutors) { - numExecutorsStarting.incrementAndGet() + val rp = rpIdToResourceProfile(rpId) + val containerMem = rp.executorResources.get(ResourceProfile.MEMORY). + map(_.amount.toInt).getOrElse(executorMemory) + val containerCores = rp.getExecutorCores.getOrElse(defaultExecutorCores) + val rpRunningExecs = getOrUpdateRunningExecutorForRPId(rpId).size() + if (rpRunningExecs < getTargetNumExecutorsForRPId(rpId)) { + getOrUpdateNumExecutorsStartingForRPId(rpId).incrementAndGet() if (launchContainers) { launcherPool.execute(() => { try { @@ -562,17 +713,17 @@ private[yarn] class YarnAllocator( driverUrl, executorId, executorHostname, - executorMemory, - executorCores, + containerMem, + containerCores, appAttemptId.getApplicationId.toString, securityMgr, localResources, - ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID // use until fully supported + rp.id ).run() updateInternalState() } catch { case e: Throwable => - numExecutorsStarting.decrementAndGet() + getOrUpdateNumExecutorsStartingForRPId(rpId).decrementAndGet() if (NonFatal(e)) { logError(s"Failed to launch executor $executorId on container $containerId", e) // Assigned container should be released immediately @@ -589,8 +740,8 @@ private[yarn] class YarnAllocator( } } else { logInfo(("Skip launching executorRunnable as running executors count: %d " + - "reached target executors count: %d.").format( - runningExecutors.size, targetNumExecutors)) + "reached target executors count: %d.").format(rpRunningExecs, + getTargetNumExecutorsForRPId(rpId))) } } } @@ -599,6 +750,8 @@ private[yarn] class YarnAllocator( private[yarn] def processCompletedContainers(completedContainers: Seq[ContainerStatus]): Unit = { for (completedContainer <- completedContainers) { val containerId = completedContainer.getContainerId + val rpId = containerIdToResourceProfileId.getOrElse(containerId, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val alreadyReleased = releasedContainers.remove(containerId) val hostOpt = allocatedContainerToHostMap.get(containerId) val onHostStr = hostOpt.map(host => s" on host: $host").getOrElse("") @@ -606,7 +759,8 @@ private[yarn] class YarnAllocator( // Decrement the number of executors running. The next iteration of // the ApplicationMaster's reporting thread will take care of allocating. containerIdToExecutorId.get(containerId) match { - case Some(executorId) => runningExecutors.remove(executorId) + case Some(executorId) => + getOrUpdateRunningExecutorForRPId(rpId).remove(executorId) case None => logWarning(s"Cannot find executorId for container: ${containerId.toString}") } @@ -679,13 +833,13 @@ private[yarn] class YarnAllocator( for { host <- hostOpt - containerSet <- allocatedHostToContainersMap.get(host) + containerSet <- getOrUpdateAllocatedHostToContainersMapForRPId(rpId).get(host) } { containerSet.remove(containerId) if (containerSet.isEmpty) { - allocatedHostToContainersMap.remove(host) + getOrUpdateAllocatedHostToContainersMapForRPId(rpId).remove(host) } else { - allocatedHostToContainersMap.update(host, containerSet) + getOrUpdateAllocatedHostToContainersMapForRPId(rpId).update(host, containerSet) } allocatedContainerToHostMap.remove(containerId) @@ -706,6 +860,7 @@ private[yarn] class YarnAllocator( // container process. releasedExecutorLossReasons.put(eid, exitReason) } + containerIdToResourceProfileId.remove(containerId) if (!alreadyReleased) { // The executor could have gone away (like no route to host, node failure, etc) // Notify backend about the failure of the executor diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 9cefc4011c930..9f9f15d26837c 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -41,10 +41,6 @@ object YarnSparkHadoopUtil { val ANY_HOST = "*" - // All RM requests are issued with same priority : we do not (yet) have any distinction between - // request types (like map/reduce in hadoop for example) - val RM_REQUEST_PRIORITY = Priority.newInstance(1) - /** * Add a path variable to the given environment map. * If the map already contains this key, append the value to the existing value instead. diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala index 669e39fb7c1c7..e7e62bdb4fad3 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala @@ -72,9 +72,9 @@ private[spark] object YarnCoarseGrainedExecutorBackend extends Logging { def main(args: Array[String]): Unit = { val createFn: (RpcEnv, CoarseGrainedExecutorBackend.Arguments, SparkEnv, ResourceProfile) => CoarseGrainedExecutorBackend = { case (rpcEnv, arguments, env, resourceProfile) => - new YarnCoarseGrainedExecutorBackend(rpcEnv, arguments.driverUrl, arguments.executorId, - arguments.bindAddress, arguments.hostname, arguments.cores, arguments.userClassPath, env, - arguments.resourcesFileOpt, resourceProfile) + new YarnCoarseGrainedExecutorBackend(rpcEnv, arguments.driverUrl, arguments.executorId, + arguments.bindAddress, arguments.hostname, arguments.cores, arguments.userClassPath, env, + arguments.resourcesFileOpt, resourceProfile) } val backendArgs = CoarseGrainedExecutorBackend.parseArguments(args, this.getClass.getCanonicalName.stripSuffix("$")) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala index 64cd1bd088001..ad067e1a53f5d 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterManager.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler.cluster import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.resource.ResourceProfileManager import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} /** @@ -29,7 +30,9 @@ private[spark] class YarnClusterManager extends ExternalClusterManager { masterURL == "yarn" } - override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = { + override def createTaskScheduler( + sc: SparkContext, + masterURL: String): TaskScheduler = { sc.deployMode match { case "cluster" => new YarnClusterScheduler(sc) case "client" => new YarnScheduler(sc) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala index 1812a55c39589..b5cfa005b1a8c 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala @@ -24,7 +24,9 @@ import org.apache.spark.deploy.yarn.ApplicationMaster * This is a simple extension to YarnScheduler - to ensure that appropriate initialization of * ApplicationMaster, etc is done */ -private[spark] class YarnClusterScheduler(sc: SparkContext) extends YarnScheduler(sc) { +private[spark] class YarnClusterScheduler( + sc: SparkContext) + extends YarnScheduler(sc) { logInfo("Created YarnClusterScheduler") diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala index d466ed77a929e..e16caaa640ad8 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala @@ -24,7 +24,9 @@ import org.apache.spark.deploy.yarn.SparkRackResolver import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.util.Utils -private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(sc) { +private[spark] class YarnScheduler( + sc: SparkContext) + extends TaskSchedulerImpl(sc) { override val defaultRackValue: Option[String] = Some(NetworkTopology.DEFAULT_RACK) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index 471ee58d05cb8..273a6df351607 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -27,13 +27,13 @@ import scala.util.{Failure, Success} import scala.util.control.NonFatal import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId} -import org.eclipse.jetty.servlet.{FilterHolder, FilterMapping} import org.apache.spark.SparkContext import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.Logging import org.apache.spark.internal.config import org.apache.spark.internal.config.UI._ +import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -92,7 +92,7 @@ private[spark] abstract class YarnSchedulerBackend( try { // SPARK-12009: To prevent Yarn allocator from requesting backup for the executors which // was Stopped by SchedulerBackend. - requestTotalExecutors(0, 0, Map.empty) + requestTotalExecutors(Map.empty, Map.empty, Map.empty) super.stop() } finally { stopped.set(true) @@ -123,21 +123,24 @@ private[spark] abstract class YarnSchedulerBackend( } } - private[cluster] def prepareRequestExecutors(requestedTotal: Int): RequestExecutors = { + private[cluster] def prepareRequestExecutors( + resourceProfileToTotalExecs: Map[ResourceProfile, Int]): RequestExecutors = { val nodeBlacklist: Set[String] = scheduler.nodeBlacklist() // For locality preferences, ignore preferences for nodes that are blacklisted - val filteredHostToLocalTaskCount = - hostToLocalTaskCount.filter { case (k, v) => !nodeBlacklist.contains(k) } - RequestExecutors(requestedTotal, localityAwareTasks, filteredHostToLocalTaskCount, - nodeBlacklist) + val filteredRPHostToLocalTaskCount = rpHostToLocalTaskCount.map { case (rpid, v) => + (rpid, v.filter { case (host, count) => !nodeBlacklist.contains(host) }) + } + RequestExecutors(numLocalityAwareTasksPerResourceProfileId, + filteredRPHostToLocalTaskCount, nodeBlacklist, resourceProfileToTotalExecs) } /** * Request executors from the ApplicationMaster by specifying the total number desired. * This includes executors already pending or running. */ - override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { - yarnSchedulerEndpointRef.ask[Boolean](prepareRequestExecutors(requestedTotal)) + override def doRequestTotalExecutors( + resourceProfileToTotalExecs: Map[ResourceProfile, Int]): Future[Boolean] = { + yarnSchedulerEndpointRef.ask[Boolean](prepareRequestExecutors(resourceProfileToTotalExecs)) } /** diff --git a/resource-managers/yarn/src/test/resources/get1Gpu.sh b/resource-managers/yarn/src/test/resources/get1Gpu.sh new file mode 100755 index 0000000000000..f068c0db4e864 --- /dev/null +++ b/resource-managers/yarn/src/test/resources/get1Gpu.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# Fake script for testing that always returns 1 GPU address + +echo {\"name\": \"gpu\", \"addresses\":[\"0\"]} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala index 29f1c0512fbd5..530000e5d0847 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ContainerPlacementStrategySuite.scala @@ -17,10 +17,13 @@ package org.apache.spark.deploy.yarn +import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.scalatest.{BeforeAndAfterEach, Matchers} +import org.apache.spark.SparkConf import org.apache.spark.SparkFunSuite +import org.apache.spark.resource.ResourceProfile class ContainerPlacementStrategySuite extends SparkFunSuite with Matchers with BeforeAndAfterEach { @@ -28,7 +31,7 @@ class ContainerPlacementStrategySuite extends SparkFunSuite with Matchers with B import yarnAllocatorSuite._ def createContainerRequest(nodes: Array[String]): ContainerRequest = - new ContainerRequest(containerResource, nodes, null, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) + new ContainerRequest(containerResource, nodes, null, Priority.newInstance(1)) override def beforeEach(): Unit = { yarnAllocatorSuite.beforeEach() @@ -38,18 +41,24 @@ class ContainerPlacementStrategySuite extends SparkFunSuite with Matchers with B yarnAllocatorSuite.afterEach() } + val resource = yarnAllocatorSuite.containerResource + val defaultResourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID + test("allocate locality preferred containers with enough resource and no matched existed " + "containers") { // 1. All the locations of current containers cannot satisfy the new requirements // 2. Current requested container number can fully satisfy the pending tasks. - val handler = createAllocator(2) + val (handler, allocatorConf) = createAllocator(2) handler.updateResourceRequests() handler.handleAllocatedContainers(Array(createContainer("host1"), createContainer("host2"))) + ResourceProfile.clearDefaultProfile + val rp = ResourceProfile.getOrCreateDefaultProfile(allocatorConf) val localities = handler.containerPlacementStrategy.localityOfRequestedContainers( 3, 15, Map("host3" -> 15, "host4" -> 15, "host5" -> 10), - handler.allocatedHostToContainersMap, Seq.empty) + handler.allocatedHostToContainersMapPerRPId(defaultResourceProfileId), Seq.empty, + resource, rp) assert(localities.map(_.nodes) === Array( Array("host3", "host4", "host5"), @@ -62,7 +71,7 @@ class ContainerPlacementStrategySuite extends SparkFunSuite with Matchers with B // 1. Parts of current containers' locations can satisfy the new requirements // 2. Current requested container number can fully satisfy the pending tasks. - val handler = createAllocator(3) + val (handler, allocatorConf) = createAllocator(3) handler.updateResourceRequests() handler.handleAllocatedContainers(Array( createContainer("host1"), @@ -70,9 +79,13 @@ class ContainerPlacementStrategySuite extends SparkFunSuite with Matchers with B createContainer("host2") )) + ResourceProfile.clearDefaultProfile + val rp = ResourceProfile.getOrCreateDefaultProfile(allocatorConf) + val localities = handler.containerPlacementStrategy.localityOfRequestedContainers( 3, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10), - handler.allocatedHostToContainersMap, Seq.empty) + handler.allocatedHostToContainersMapPerRPId(defaultResourceProfileId), Seq.empty, + resource, rp) assert(localities.map(_.nodes) === Array(null, Array("host2", "host3"), Array("host2", "host3"))) @@ -83,7 +96,7 @@ class ContainerPlacementStrategySuite extends SparkFunSuite with Matchers with B // 1. Parts of current containers' locations can satisfy the new requirements // 2. Current requested container number cannot fully satisfy the pending tasks. - val handler = createAllocator(3) + val (handler, allocatorConf) = createAllocator(3) handler.updateResourceRequests() handler.handleAllocatedContainers(Array( createContainer("host1"), @@ -91,9 +104,12 @@ class ContainerPlacementStrategySuite extends SparkFunSuite with Matchers with B createContainer("host2") )) + ResourceProfile.clearDefaultProfile + val rp = ResourceProfile.getOrCreateDefaultProfile(allocatorConf) val localities = handler.containerPlacementStrategy.localityOfRequestedContainers( 1, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10), - handler.allocatedHostToContainersMap, Seq.empty) + handler.allocatedHostToContainersMapPerRPId(defaultResourceProfileId), Seq.empty, + resource, rp) assert(localities.map(_.nodes) === Array(Array("host2", "host3"))) } @@ -101,7 +117,7 @@ class ContainerPlacementStrategySuite extends SparkFunSuite with Matchers with B test("allocate locality preferred containers with fully matched containers") { // Current containers' locations can fully satisfy the new requirements - val handler = createAllocator(5) + val (handler, allocatorConf) = createAllocator(5) handler.updateResourceRequests() handler.handleAllocatedContainers(Array( createContainer("host1"), @@ -111,9 +127,12 @@ class ContainerPlacementStrategySuite extends SparkFunSuite with Matchers with B createContainer("host3") )) + ResourceProfile.clearDefaultProfile + val rp = ResourceProfile.getOrCreateDefaultProfile(allocatorConf) val localities = handler.containerPlacementStrategy.localityOfRequestedContainers( 3, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10), - handler.allocatedHostToContainersMap, Seq.empty) + handler.allocatedHostToContainersMapPerRPId(defaultResourceProfileId), Seq.empty, + resource, rp) assert(localities.map(_.nodes) === Array(null, null, null)) } @@ -121,18 +140,22 @@ class ContainerPlacementStrategySuite extends SparkFunSuite with Matchers with B test("allocate containers with no locality preference") { // Request new container without locality preference - val handler = createAllocator(2) + val (handler, allocatorConf) = createAllocator(2) handler.updateResourceRequests() handler.handleAllocatedContainers(Array(createContainer("host1"), createContainer("host2"))) + ResourceProfile.clearDefaultProfile + val rp = ResourceProfile.getOrCreateDefaultProfile(allocatorConf) val localities = handler.containerPlacementStrategy.localityOfRequestedContainers( - 1, 0, Map.empty, handler.allocatedHostToContainersMap, Seq.empty) + 1, 0, Map.empty, + handler.allocatedHostToContainersMapPerRPId(defaultResourceProfileId), Seq.empty, + resource, rp) assert(localities.map(_.nodes) === Array(null)) } test("allocate locality preferred containers by considering the localities of pending requests") { - val handler = createAllocator(3) + val (handler, allocatorConf) = createAllocator(3) handler.updateResourceRequests() handler.handleAllocatedContainers(Array( createContainer("host1"), @@ -144,9 +167,13 @@ class ContainerPlacementStrategySuite extends SparkFunSuite with Matchers with B createContainerRequest(Array("host2", "host3")), createContainerRequest(Array("host1", "host4"))) + ResourceProfile.clearDefaultProfile + val rp = ResourceProfile.getOrCreateDefaultProfile(allocatorConf) val localities = handler.containerPlacementStrategy.localityOfRequestedContainers( 1, 15, Map("host1" -> 15, "host2" -> 15, "host3" -> 10), - handler.allocatedHostToContainersMap, pendingAllocationRequests) + handler.allocatedHostToContainersMapPerRPId(defaultResourceProfileId), + pendingAllocationRequests, + resource, rp) assert(localities.map(_.nodes) === Array(Array("host3"))) } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala index b7f25656e49ac..cbba830fa1637 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.mockito.Mockito._ import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.resource.ResourceProfile class LocalityPlacementStrategySuite extends SparkFunSuite { @@ -58,7 +59,7 @@ class LocalityPlacementStrategySuite extends SparkFunSuite { val resource = Resource.newInstance(8 * 1024, 4) val strategy = new LocalityPreferredContainerPlacementStrategy(new SparkConf(), - yarnConf, resource, new MockResolver()) + yarnConf, new MockResolver()) val totalTasks = 32 * 1024 val totalContainers = totalTasks / 16 @@ -75,9 +76,10 @@ class LocalityPlacementStrategySuite extends SparkFunSuite { containers.drop(count * i).take(i).foreach { c => hostContainers += c } hostToContainerMap(host) = hostContainers } + val rp = ResourceProfile.getOrCreateDefaultProfile(new SparkConf) strategy.localityOfRequestedContainers(containers.size * 2, totalTasks, hosts, - hostToContainerMap, Nil) + hostToContainerMap, Nil, resource, rp) } } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index 6216d473882e6..8918015bde3b0 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.yarn import java.util.Collections import scala.collection.JavaConverters._ +import scala.collection.mutable import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.AMRMClient @@ -32,9 +33,9 @@ import org.scalatest.{BeforeAndAfterEach, Matchers} import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.yarn.ResourceRequestHelper._ -import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.config._ +import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfile, TaskResourceRequests} import org.apache.spark.resource.ResourceUtils.{AMOUNT, GPU} import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.rpc.RpcEndpointRef @@ -69,6 +70,11 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter var containerNum = 0 + // priority has to be 0 to match default profile id + val RM_REQUEST_PRIORITY = Priority.newInstance(0) + val defaultRPId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID + val defaultRP = ResourceProfile.getOrCreateDefaultProfile(sparkConf) + override def beforeEach(): Unit = { super.beforeEach() rmClient = AMRMClient.createAMRMClient() @@ -93,7 +99,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter def createAllocator( maxExecutors: Int = 5, rmClient: AMRMClient[ContainerRequest] = rmClient, - additionalConfigs: Map[String, String] = Map()): YarnAllocator = { + additionalConfigs: Map[String, String] = Map()): (YarnAllocator, SparkConf) = { val args = Array( "--jar", "somejar.jar", "--class", "SomeClass") @@ -107,7 +113,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter sparkConfClone.set(name, value) } - new YarnAllocator( + val allocator = new YarnAllocator( "not used", mock(classOf[RpcEndpointRef]), conf, @@ -118,16 +124,18 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter Map(), new MockResolver(), clock) + (allocator, sparkConfClone) } def createContainer( host: String, containerNumber: Int = containerNum, - resource: Resource = containerResource): Container = { + resource: Resource = containerResource, + priority: Priority = RM_REQUEST_PRIORITY): Container = { val containerId: ContainerId = ContainerId.newContainerId(appAttemptId, containerNum) containerNum += 1 val nodeId = NodeId.newInstance(host, 1000) - Container.newInstance(containerId, nodeId, "", resource, RM_REQUEST_PRIORITY, null) + Container.newInstance(containerId, nodeId, "", resource, priority, null) } def createContainers(hosts: Seq[String], containerIds: Seq[Int]): Seq[Container] = { @@ -145,20 +153,108 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter test("single container allocated") { // request a single container and receive it - val handler = createAllocator(1) + val (handler, _) = createAllocator(1) handler.updateResourceRequests() handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (1) + handler.getNumContainersPendingAllocate should be (1) val container = createContainer("host1") handler.handleAllocatedContainers(Array(container)) handler.getNumExecutorsRunning should be (1) handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1") - handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId) + val hostTocontainer = handler.allocatedHostToContainersMapPerRPId(defaultRPId) + hostTocontainer.get("host1").get should contain(container.getId) + + val size = rmClient.getMatchingRequests(container.getPriority, "host1", containerResource).size + size should be (0) + } + + test("single container allocated with ResourceProfile") { + assume(isYarnResourceTypesAvailable()) + val yarnResources = Seq(YARN_GPU_RESOURCE_CONFIG) + ResourceRequestTestHelper.initializeResourceTypes(yarnResources) + // create default profile so we get a different id to test below + val defaultRProf = ResourceProfile.getOrCreateDefaultProfile(sparkConf) + val execReq = new ExecutorResourceRequests().resource("gpu", 6) + val taskReq = new TaskResourceRequests().resource("gpu", 1) + val rprof = new ResourceProfile(execReq.requests, taskReq.requests) + // request a single container and receive it + val (handler, _) = createAllocator(0) + + val resourceProfileToTotalExecs = mutable.HashMap(defaultRProf -> 0, rprof -> 1) + val numLocalityAwareTasksPerResourceProfileId = mutable.HashMap(rprof.id -> 0) + handler.requestTotalExecutorsWithPreferredLocalities(resourceProfileToTotalExecs.toMap, + numLocalityAwareTasksPerResourceProfileId.toMap, Map.empty, Set.empty) + + handler.updateResourceRequests() + handler.getNumExecutorsRunning should be (0) + handler.getNumContainersPendingAllocate should be (1) + + val container = createContainer("host1", priority = Priority.newInstance(rprof.id)) + handler.handleAllocatedContainers(Array(container)) + + handler.getNumExecutorsRunning should be (1) + handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1") + val hostTocontainer = handler.allocatedHostToContainersMapPerRPId(rprof.id) + hostTocontainer.get("host1").get should contain(container.getId) val size = rmClient.getMatchingRequests(container.getPriority, "host1", containerResource).size size should be (0) + + ResourceProfile.reInitDefaultProfile(sparkConf) + } + + test("multiple containers allocated with ResourceProfiles") { + assume(isYarnResourceTypesAvailable()) + val yarnResources = Seq(YARN_GPU_RESOURCE_CONFIG, YARN_FPGA_RESOURCE_CONFIG) + ResourceRequestTestHelper.initializeResourceTypes(yarnResources) + // create default profile so we get a different id to test below + val defaultRProf = ResourceProfile.getOrCreateDefaultProfile(sparkConf) + val execReq = new ExecutorResourceRequests().resource("gpu", 6) + val taskReq = new TaskResourceRequests().resource("gpu", 1) + val rprof = new ResourceProfile(execReq.requests, taskReq.requests) + + val execReq2 = new ExecutorResourceRequests().memory("8g").resource("fpga", 2) + val taskReq2 = new TaskResourceRequests().resource("fpga", 1) + val rprof2 = new ResourceProfile(execReq2.requests, taskReq2.requests) + + + // request a single container and receive it + val (handler, _) = createAllocator(1) + val resourceProfileToTotalExecs = mutable.HashMap(defaultRProf -> 0, rprof -> 1, rprof2 -> 2) + val numLocalityAwareTasksPerResourceProfileId = mutable.HashMap(rprof.id -> 0, rprof2.id -> 0) + handler.requestTotalExecutorsWithPreferredLocalities(resourceProfileToTotalExecs.toMap, + numLocalityAwareTasksPerResourceProfileId.toMap, Map.empty, Set.empty) + + handler.updateResourceRequests() + handler.getNumExecutorsRunning should be (0) + handler.getNumContainersPendingAllocate should be (3) + + val containerResourcerp2 = Resource.newInstance(10240, 5) + + val container = createContainer("host1", priority = Priority.newInstance(rprof.id)) + val container2 = createContainer("host2", resource = containerResourcerp2, + priority = Priority.newInstance(rprof2.id)) + val container3 = createContainer("host3", resource = containerResourcerp2, + priority = Priority.newInstance(rprof2.id)) + handler.handleAllocatedContainers(Array(container, container2, container3)) + + handler.getNumExecutorsRunning should be (3) + handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1") + handler.allocatedContainerToHostMap.get(container2.getId).get should be ("host2") + handler.allocatedContainerToHostMap.get(container3.getId).get should be ("host3") + + val hostTocontainer = handler.allocatedHostToContainersMapPerRPId(rprof.id) + hostTocontainer.get("host1").get should contain(container.getId) + val hostTocontainer2 = handler.allocatedHostToContainersMapPerRPId(rprof2.id) + hostTocontainer2.get("host2").get should contain(container2.getId) + hostTocontainer2.get("host3").get should contain(container3.getId) + + val size = rmClient.getMatchingRequests(container.getPriority, "host1", containerResource).size + size should be (0) + + ResourceProfile.reInitDefaultProfile(sparkConf) } test("custom resource requested from yarn") { @@ -166,7 +262,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter ResourceRequestTestHelper.initializeResourceTypes(List("gpu")) val mockAmClient = mock(classOf[AMRMClient[ContainerRequest]]) - val handler = createAllocator(1, mockAmClient, + val (handler, _) = createAllocator(1, mockAmClient, Map(s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${GPU}.${AMOUNT}" -> "2G")) handler.updateResourceRequests() @@ -195,7 +291,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter Map(EXECUTOR_GPU_ID.amountConf -> "3", EXECUTOR_FPGA_ID.amountConf -> "2", madeupConfigName -> "5") - val handler = createAllocator(1, mockAmClient, sparkResources) + val (handler, _) = createAllocator(1, mockAmClient, sparkResources) handler.updateResourceRequests() val yarnRInfo = ResourceRequestTestHelper.getResources(handler.resource) @@ -210,17 +306,18 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter test("container should not be created if requested number if met") { // request a single container and receive it - val handler = createAllocator(1) + val (handler, _) = createAllocator(1) handler.updateResourceRequests() handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (1) + handler.getNumContainersPendingAllocate should be (1) val container = createContainer("host1") handler.handleAllocatedContainers(Array(container)) handler.getNumExecutorsRunning should be (1) handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1") - handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId) + val hostTocontainer = handler.allocatedHostToContainersMapPerRPId(defaultRPId) + hostTocontainer.get("host1").get should contain(container.getId) val container2 = createContainer("host2") handler.handleAllocatedContainers(Array(container2)) @@ -229,10 +326,10 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter test("some containers allocated") { // request a few containers and receive some of them - val handler = createAllocator(4) + val (handler, _) = createAllocator(4) handler.updateResourceRequests() handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (4) + handler.getNumContainersPendingAllocate should be (4) val container1 = createContainer("host1") val container2 = createContainer("host1") @@ -243,16 +340,17 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter handler.allocatedContainerToHostMap.get(container1.getId).get should be ("host1") handler.allocatedContainerToHostMap.get(container2.getId).get should be ("host1") handler.allocatedContainerToHostMap.get(container3.getId).get should be ("host2") - handler.allocatedHostToContainersMap.get("host1").get should contain (container1.getId) - handler.allocatedHostToContainersMap.get("host1").get should contain (container2.getId) - handler.allocatedHostToContainersMap.get("host2").get should contain (container3.getId) + val hostTocontainer = handler.allocatedHostToContainersMapPerRPId(defaultRPId) + hostTocontainer.get("host1").get should contain(container1.getId) + hostTocontainer.get("host1").get should contain (container2.getId) + hostTocontainer.get("host2").get should contain (container3.getId) } test("receive more containers than requested") { - val handler = createAllocator(2) + val (handler, _) = createAllocator(2) handler.updateResourceRequests() handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (2) + handler.getNumContainersPendingAllocate should be (2) val container1 = createContainer("host1") val container2 = createContainer("host2") @@ -263,42 +361,52 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter handler.allocatedContainerToHostMap.get(container1.getId).get should be ("host1") handler.allocatedContainerToHostMap.get(container2.getId).get should be ("host2") handler.allocatedContainerToHostMap.contains(container3.getId) should be (false) - handler.allocatedHostToContainersMap.get("host1").get should contain (container1.getId) - handler.allocatedHostToContainersMap.get("host2").get should contain (container2.getId) - handler.allocatedHostToContainersMap.contains("host4") should be (false) + val hostTocontainer = handler.allocatedHostToContainersMapPerRPId(defaultRPId) + hostTocontainer.get("host1").get should contain(container1.getId) + hostTocontainer.get("host2").get should contain (container2.getId) + hostTocontainer.contains("host4") should be (false) } test("decrease total requested executors") { - val handler = createAllocator(4) + val (handler, _) = createAllocator(4) handler.updateResourceRequests() handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (4) + handler.getNumContainersPendingAllocate should be (4) - handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map.empty, Set.empty) + val resourceProfileToTotalExecs = mutable.HashMap(defaultRP -> 3) + val numLocalityAwareTasksPerResourceProfileId = mutable.HashMap(defaultRPId -> 0) + handler.requestTotalExecutorsWithPreferredLocalities(resourceProfileToTotalExecs.toMap, + numLocalityAwareTasksPerResourceProfileId.toMap, Map.empty, Set.empty) handler.updateResourceRequests() - handler.getPendingAllocate.size should be (3) + handler.getNumContainersPendingAllocate should be (3) val container = createContainer("host1") handler.handleAllocatedContainers(Array(container)) handler.getNumExecutorsRunning should be (1) handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1") - handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId) + val hostTocontainer = handler.allocatedHostToContainersMapPerRPId(defaultRPId) + hostTocontainer.get("host1").get should contain(container.getId) - handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map.empty, Set.empty) + resourceProfileToTotalExecs(defaultRP) = 2 + handler.requestTotalExecutorsWithPreferredLocalities(resourceProfileToTotalExecs.toMap, + numLocalityAwareTasksPerResourceProfileId.toMap, Map.empty, Set.empty) handler.updateResourceRequests() - handler.getPendingAllocate.size should be (1) + handler.getNumContainersPendingAllocate should be (1) } test("decrease total requested executors to less than currently running") { - val handler = createAllocator(4) + val (handler, _) = createAllocator(4) handler.updateResourceRequests() handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (4) + handler.getNumContainersPendingAllocate should be (4) - handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map.empty, Set.empty) + val resourceProfileToTotalExecs = mutable.HashMap(defaultRP -> 3) + val numLocalityAwareTasksPerResourceProfileId = mutable.HashMap(defaultRPId -> 0) + handler.requestTotalExecutorsWithPreferredLocalities(resourceProfileToTotalExecs.toMap, + numLocalityAwareTasksPerResourceProfileId.toMap, Map.empty, Set.empty) handler.updateResourceRequests() - handler.getPendingAllocate.size should be (3) + handler.getNumContainersPendingAllocate should be (3) val container1 = createContainer("host1") val container2 = createContainer("host2") @@ -306,23 +414,28 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter handler.getNumExecutorsRunning should be (2) - handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map.empty, Set.empty) + resourceProfileToTotalExecs(defaultRP) = 1 + handler.requestTotalExecutorsWithPreferredLocalities(resourceProfileToTotalExecs.toMap, + numLocalityAwareTasksPerResourceProfileId.toMap, Map.empty, Set.empty) handler.updateResourceRequests() - handler.getPendingAllocate.size should be (0) + handler.getNumContainersPendingAllocate should be (0) handler.getNumExecutorsRunning should be (2) } test("kill executors") { - val handler = createAllocator(4) + val (handler, _) = createAllocator(4) handler.updateResourceRequests() handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (4) + handler.getNumContainersPendingAllocate should be (4) val container1 = createContainer("host1") val container2 = createContainer("host2") handler.handleAllocatedContainers(Array(container1, container2)) - handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map.empty, Set.empty) + val resourceProfileToTotalExecs = mutable.HashMap(defaultRP -> 1) + val numLocalityAwareTasksPerResourceProfileId = mutable.HashMap(defaultRPId -> 0) + handler.requestTotalExecutorsWithPreferredLocalities(resourceProfileToTotalExecs.toMap, + numLocalityAwareTasksPerResourceProfileId.toMap, Map.empty, Set.empty) handler.executorIdToContainer.keys.foreach { id => handler.killExecutor(id ) } val statuses = Seq(container1, container2).map { c => @@ -331,20 +444,20 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter handler.updateResourceRequests() handler.processCompletedContainers(statuses) handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (1) + handler.getNumContainersPendingAllocate should be (1) } test("kill same executor multiple times") { - val handler = createAllocator(2) + val (handler, _) = createAllocator(2) handler.updateResourceRequests() handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (2) + handler.getNumContainersPendingAllocate should be (2) val container1 = createContainer("host1") val container2 = createContainer("host2") handler.handleAllocatedContainers(Array(container1, container2)) handler.getNumExecutorsRunning should be (2) - handler.getPendingAllocate.size should be (0) + handler.getNumContainersPendingAllocate should be (0) val executorToKill = handler.executorIdToContainer.keys.head handler.killExecutor(executorToKill) @@ -353,22 +466,25 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter handler.killExecutor(executorToKill) handler.killExecutor(executorToKill) handler.getNumExecutorsRunning should be (1) - handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map.empty, Set.empty) + val resourceProfileToTotalExecs = mutable.HashMap(defaultRP -> 2) + val numLocalityAwareTasksPerResourceProfileId = mutable.HashMap(defaultRPId -> 0) + handler.requestTotalExecutorsWithPreferredLocalities(resourceProfileToTotalExecs.toMap, + numLocalityAwareTasksPerResourceProfileId.toMap, Map.empty, Set.empty) handler.updateResourceRequests() - handler.getPendingAllocate.size should be (1) + handler.getNumContainersPendingAllocate should be (1) } test("process same completed container multiple times") { - val handler = createAllocator(2) + val (handler, _) = createAllocator(2) handler.updateResourceRequests() handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (2) + handler.getNumContainersPendingAllocate should be (2) val container1 = createContainer("host1") val container2 = createContainer("host2") handler.handleAllocatedContainers(Array(container1, container2)) handler.getNumExecutorsRunning should be (2) - handler.getPendingAllocate.size should be (0) + handler.getNumContainersPendingAllocate should be (0) val statuses = Seq(container1, container1, container2).map { c => ContainerStatus.newInstance(c.getId(), ContainerState.COMPLETE, "Finished", 0) @@ -379,16 +495,19 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter } test("lost executor removed from backend") { - val handler = createAllocator(4) + val (handler, _) = createAllocator(4) handler.updateResourceRequests() handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (4) + handler.getNumContainersPendingAllocate should be (4) val container1 = createContainer("host1") val container2 = createContainer("host2") handler.handleAllocatedContainers(Array(container1, container2)) - handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map(), Set.empty) + val resourceProfileToTotalExecs = mutable.HashMap(defaultRP -> 2) + val numLocalityAwareTasksPerResourceProfileId = mutable.HashMap(defaultRPId -> 0) + handler.requestTotalExecutorsWithPreferredLocalities(resourceProfileToTotalExecs.toMap, + numLocalityAwareTasksPerResourceProfileId.toMap, Map(), Set.empty) val statuses = Seq(container1, container2).map { c => ContainerStatus.newInstance(c.getId(), ContainerState.COMPLETE, "Failed", -1) @@ -397,7 +516,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter handler.processCompletedContainers(statuses) handler.updateResourceRequests() handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (2) + handler.getNumContainersPendingAllocate should be (2) handler.getNumExecutorsFailed should be (2) handler.getNumUnexpectedContainerRelease should be (2) } @@ -406,28 +525,35 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter // Internally we track the set of blacklisted nodes, but yarn wants us to send *changes* // to the blacklist. This makes sure we are sending the right updates. val mockAmClient = mock(classOf[AMRMClient[ContainerRequest]]) - val handler = createAllocator(4, mockAmClient) - handler.requestTotalExecutorsWithPreferredLocalities(1, 0, Map(), Set("hostA")) + val (handler, _) = createAllocator(4, mockAmClient) + val resourceProfileToTotalExecs = mutable.HashMap(defaultRP -> 1) + val numLocalityAwareTasksPerResourceProfileId = mutable.HashMap(defaultRPId -> 0) + handler.requestTotalExecutorsWithPreferredLocalities(resourceProfileToTotalExecs.toMap, + numLocalityAwareTasksPerResourceProfileId.toMap, Map(), Set("hostA")) verify(mockAmClient).updateBlacklist(Seq("hostA").asJava, Seq[String]().asJava) val blacklistedNodes = Set( "hostA", "hostB" ) - handler.requestTotalExecutorsWithPreferredLocalities(2, 0, Map(), blacklistedNodes) - verify(mockAmClient).updateBlacklist(Seq("hostB").asJava, Seq[String]().asJava) - handler.requestTotalExecutorsWithPreferredLocalities(3, 0, Map(), Set.empty) + resourceProfileToTotalExecs(defaultRP) = 2 + handler.requestTotalExecutorsWithPreferredLocalities(resourceProfileToTotalExecs.toMap, + numLocalityAwareTasksPerResourceProfileId.toMap, Map(), blacklistedNodes) + verify(mockAmClient).updateBlacklist(Seq("hostB").asJava, Seq[String]().asJava) + resourceProfileToTotalExecs(defaultRP) = 3 + handler.requestTotalExecutorsWithPreferredLocalities(resourceProfileToTotalExecs.toMap, + numLocalityAwareTasksPerResourceProfileId.toMap, Map(), Set.empty) verify(mockAmClient).updateBlacklist(Seq[String]().asJava, Seq("hostA", "hostB").asJava) } test("window based failure executor counting") { sparkConf.set(EXECUTOR_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS, 100 * 1000L) - val handler = createAllocator(4) + val (handler, _) = createAllocator(4) handler.updateResourceRequests() handler.getNumExecutorsRunning should be (0) - handler.getPendingAllocate.size should be (4) + handler.getNumContainersPendingAllocate should be (4) val containers = Seq( createContainer("host1"), @@ -468,7 +594,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter val rmClientSpy = spy(rmClient) val maxExecutors = 11 - val handler = createAllocator( + val (handler, _) = createAllocator( maxExecutors, rmClientSpy, Map( @@ -525,9 +651,9 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter try { sparkConf.set(MEMORY_OFFHEAP_ENABLED, true) sparkConf.set(MEMORY_OFFHEAP_SIZE, offHeapMemoryInByte) - val allocator = createAllocator(maxExecutors = 1, + val (handler, _) = createAllocator(maxExecutors = 1, additionalConfigs = Map(EXECUTOR_MEMORY.key -> executorMemory.toString)) - val memory = allocator.resource.getMemory + val memory = handler.resource.getMemory assert(memory == executorMemory + offHeapMemoryInMB + YarnSparkHadoopUtil.MEMORY_OVERHEAD_MIN) } finally { diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala index 8c62069a8dd67..1db325b49360d 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala @@ -32,6 +32,8 @@ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Network._ import org.apache.spark.network.shuffle.ShuffleTestAccessor import org.apache.spark.network.yarn.{YarnShuffleService, YarnTestAccessor} +import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfileBuilder, TaskResourceRequests} +import org.apache.spark.resource.ResourceUtils.GPU import org.apache.spark.tags.ExtendedYarnTest /** @@ -160,3 +162,4 @@ private object YarnExternalShuffleDriver extends Logging with Matchers { } } + diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala index a87820b1528ad..9003c2f630975 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala @@ -24,6 +24,7 @@ import org.mockito.Mockito.when import org.scalatestplus.mockito.MockitoSugar import org.apache.spark._ +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.serializer.JavaSerializer import org.apache.spark.ui.TestFilter @@ -50,8 +51,8 @@ class YarnSchedulerBackendSuite extends SparkFunSuite with MockitoSugar with Loc private class TestYarnSchedulerBackend(scheduler: TaskSchedulerImpl, sc: SparkContext) extends YarnSchedulerBackend(scheduler, sc) { - def setHostToLocalTaskCount(hostToLocalTaskCount: Map[String, Int]): Unit = { - this.hostToLocalTaskCount = hostToLocalTaskCount + def setHostToLocalTaskCount(hostToLocalTaskCount: Map[Int, Map[String, Int]]): Unit = { + this.rpHostToLocalTaskCount = hostToLocalTaskCount } } @@ -62,20 +63,24 @@ class YarnSchedulerBackendSuite extends SparkFunSuite with MockitoSugar with Loc val yarnSchedulerBackendExtended = new TestYarnSchedulerBackend(sched, sc) yarnSchedulerBackend = yarnSchedulerBackendExtended val ser = new JavaSerializer(sc.conf).newInstance() + val defaultResourceProf = ResourceProfile.getOrCreateDefaultProfile(sc.getConf) for { blacklist <- IndexedSeq(Set[String](), Set("a", "b", "c")) numRequested <- 0 until 10 hostToLocalCount <- IndexedSeq( - Map[String, Int](), - Map("a" -> 1, "b" -> 2) + Map(defaultResourceProf.id -> Map.empty[String, Int]), + Map(defaultResourceProf.id -> Map("a" -> 1, "b" -> 2)) ) } { yarnSchedulerBackendExtended.setHostToLocalTaskCount(hostToLocalCount) sched.setNodeBlacklist(blacklist) - val req = yarnSchedulerBackendExtended.prepareRequestExecutors(numRequested) - assert(req.requestedTotal === numRequested) + val request = Map(defaultResourceProf -> numRequested) + val req = yarnSchedulerBackendExtended.prepareRequestExecutors(request) + assert(req.resourceProfileToTotalExecs(defaultResourceProf) === numRequested) assert(req.nodeBlacklist === blacklist) - assert(req.hostToLocalTaskCount.keySet.intersect(blacklist).isEmpty) + val hosts = + req.hostToLocalTaskCount(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID).keySet + assert(hosts.intersect(blacklist).isEmpty) // Serialize to make sure serialization doesn't throw an error ser.serialize(req) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala index a88abc8209a88..c09ff51ecaff2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala @@ -27,6 +27,7 @@ import org.apache.spark.{SparkConf, TaskState} import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config.Status._ +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler._ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetricInfo @@ -89,7 +90,8 @@ object MetricsAggregationBenchmark extends BenchmarkBase { val taskEventsTime = (0 until numStages).map { _ => val stageInfo = new StageInfo(idgen.incrementAndGet(), 0, getClass().getName(), - numTasks, Nil, Nil, getClass().getName()) + numTasks, Nil, Nil, getClass().getName(), + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val jobId = idgen.incrementAndGet() val jobStart = SparkListenerJobStart( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala index 55b551d0af078..fdfd392a224cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala @@ -30,6 +30,7 @@ import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config import org.apache.spark.internal.config.Status._ import org.apache.spark.rdd.RDD +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler._ import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.catalyst.InternalRow @@ -86,7 +87,8 @@ class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTestUtils name = "", rddInfos = Nil, parentIds = Nil, - details = "") + details = "", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) } private def createTaskInfo( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala index e85a3b9009c32..edc79b67b15f9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala @@ -23,6 +23,7 @@ import scala.util.Random import org.apache.spark.{ExecutorAllocationClient, SparkConf} import org.apache.spark.internal.Logging import org.apache.spark.internal.config.Streaming._ +import org.apache.spark.resource.ResourceProfile import org.apache.spark.streaming.util.RecurringTimer import org.apache.spark.util.{Clock, Utils} @@ -111,7 +112,10 @@ private[streaming] class ExecutorAllocationManager( logDebug(s"Executors (${allExecIds.size}) = ${allExecIds}") val targetTotalExecutors = math.max(math.min(maxNumExecutors, allExecIds.size + numNewExecutors), minNumExecutors) - client.requestTotalExecutors(targetTotalExecutors, 0, Map.empty) + // Just map the targetTotalExecutors to the default ResourceProfile + client.requestTotalExecutors(Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 0), + Map.empty, + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> targetTotalExecutors)) logInfo(s"Requested total $targetTotalExecutors executors") } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala index 9121da4b9b673..188f14445f07f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala @@ -27,6 +27,7 @@ import org.scalatestplus.mockito.MockitoSugar import org.apache.spark.{ExecutorAllocationClient, SparkConf} import org.apache.spark.internal.config.{DYN_ALLOCATION_ENABLED, DYN_ALLOCATION_TESTING} import org.apache.spark.internal.config.Streaming._ +import org.apache.spark.resource.ResourceProfile import org.apache.spark.streaming.{DummyInputDStream, Seconds, StreamingContext, TestSuiteBase} import org.apache.spark.util.{ManualClock, Utils} @@ -71,10 +72,13 @@ class ExecutorAllocationManagerSuite extends TestSuiteBase if (expectedRequestedTotalExecs.nonEmpty) { require(expectedRequestedTotalExecs.get > 0) verify(allocationClient, times(1)).requestTotalExecutors( - meq(expectedRequestedTotalExecs.get), meq(0), meq(Map.empty)) + meq(Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 0)), meq(Map.empty), + meq(Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> + expectedRequestedTotalExecs.get))) } else { - verify(allocationClient, never).requestTotalExecutors(0, 0, Map.empty) - } + verify(allocationClient, never).requestTotalExecutors( + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 0), Map.empty, + Map(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID -> 0))} } /** Verify that a particular executor was killed */ @@ -139,8 +143,10 @@ class ExecutorAllocationManagerSuite extends TestSuiteBase reset(allocationClient) when(allocationClient.getExecutorIds()).thenReturn((1 to numExecs).map(_.toString)) requestExecutors(allocationManager, numNewExecs) + val defaultProfId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID verify(allocationClient, times(1)).requestTotalExecutors( - meq(expectedRequestedTotalExecs), meq(0), meq(Map.empty)) + meq(Map(defaultProfId -> 0)), meq(Map.empty), + meq(Map(defaultProfId -> expectedRequestedTotalExecs))) } withAllocationManager(numReceivers = 1) { case (_, allocationManager) =>