Github user tnachen commented on a diff in the pull request:
https://github.com/apache/spark/pull/5144#discussion_r28570148
--- Diff:
core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala
---
@@ -0,0 +1,614 @@
+/*
+ * 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.scheduler.cluster.mesos
+
+import java.io.File
+import java.util.concurrent.locks.ReentrantLock
+import java.util.{Collections, Date, List => JList}
+
+import org.apache.mesos.Protos.Environment.Variable
+import org.apache.mesos.Protos.TaskStatus.Reason
+import org.apache.mesos.Protos.{TaskState => MesosTaskState, _}
+import org.apache.mesos.{Scheduler, SchedulerDriver}
+import org.apache.spark.deploy.mesos.MesosDriverDescription
+import org.apache.spark.deploy.rest.{CreateSubmissionResponse,
KillSubmissionResponse, SubmissionStatusResponse}
+import org.apache.spark.metrics.MetricsSystem
+import org.apache.spark.util.Utils
+import org.apache.spark.{SecurityManager, SparkConf, SparkException,
TaskState}
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+
+/**
+ * Tracks the current state of a Mesos Task that runs a Spark driver.
+ * @param submission Submitted driver description from
+ * [[org.apache.spark.deploy.rest.mesos.MesosRestServer]]
+ * @param taskId Mesos TaskID generated for the task
+ * @param slaveId Slave ID that the task is assigned to
+ * @param taskState The last known task status update.
+ * @param startDate The date the task was launched
+ */
+private[spark] class MesosClusterTaskState(
+ val submission: MesosDriverDescription,
+ val taskId: TaskID,
+ val slaveId: SlaveID,
+ var taskState: Option[TaskStatus],
+ var startDate: Date)
+ extends Serializable {
+
+ def copy(): MesosClusterTaskState = {
+ new MesosClusterTaskState(
+ submission, taskId, slaveId, taskState, startDate)
+ }
+}
+
+/**
+ * Tracks the retry state of a driver, which includes the next time it
should be scheduled
+ * and necessary information to do exponential backoff.
+ * This class is not thread-safe, and we expect the caller to handle
synchronizing state.
+ * @param lastFailureStatus Last Task status when it failed.
+ * @param retries Number of times it has retried.
+ * @param nextRetry Next retry time to be scheduled.
+ * @param waitTime The amount of time driver is scheduled to wait until
next retry.
+ */
+private[spark] class RetryState(
+ val lastFailureStatus: TaskStatus,
+ val retries: Int,
+ val nextRetry: Date,
+ val waitTime: Int) extends Serializable {
+ def copy(): RetryState =
+ new RetryState(lastFailureStatus, retries, nextRetry, waitTime)
+}
+
+/**
+ * The full state of the cluster scheduler, currently being used for
displaying
+ * information on the UI.
+ * @param frameworkId Mesos Framework id for the cluster scheduler.
+ * @param masterUrl The Mesos master url
+ * @param queuedDrivers All drivers queued to be launched
+ * @param launchedDrivers All launched or running drivers
+ * @param finishedDrivers All terminated drivers
+ * @param retryList All drivers pending to be retried
+ */
+private[spark] class MesosClusterSchedulerState(
+ val frameworkId: String,
+ val masterUrl: Option[String],
+ val queuedDrivers: Iterable[MesosDriverDescription],
+ val launchedDrivers: Iterable[MesosClusterTaskState],
+ val finishedDrivers: Iterable[MesosClusterTaskState],
+ val retryList: Iterable[MesosDriverDescription])
+
+/**
+ * A Mesos scheduler that is responsible for launching submitted Spark
drivers in cluster mode
+ * as Mesos tasks in a Mesos cluster.
+ * All drivers are launched asynchronously by the framework, which will
eventually be launched
+ * by one of the slaves in the cluster. The results of the driver will be
stored in slave's task
+ * sandbox which is accessible by visiting the Mesos UI.
+ * This scheduler supports recovery by persisting all its state and
performs task reconciliation
+ * on recover, which gets all the latest state for all the drivers from
Mesos master.
+ */
+private[spark] class MesosClusterScheduler(
+ engineFactory: MesosClusterPersistenceEngineFactory,
+ conf: SparkConf)
+ extends Scheduler with MesosSchedulerUtils {
+ var frameworkUrl: String = _
+
+ private val metricsSystem =
+ MetricsSystem.createMetricsSystem("mesos_cluster", conf, new
SecurityManager(conf))
+ private val master = conf.get("spark.master")
+ private val appName = conf.get("spark.app.name")
+ private val queuedCapacity =
conf.getInt("spark.deploy.mesos.queuedDrivers", 200)
+ private val retainedDrivers =
conf.getInt("spark.deploy.retainedDrivers", 200)
+ private val maxRetryWaitTime =
conf.getInt("spark.mesos.cluster.retry.wait.max", 60) // 1 minute
+ private val schedulerState = engineFactory.createEngine("scheduler")
+ private val stateLock = new ReentrantLock()
+ private val finishedDrivers = new
mutable.ArrayBuffer[MesosClusterTaskState](retainedDrivers)
+ private var frameworkId: String = null
+ val launchedDrivers = new mutable.HashMap[String,
MesosClusterTaskState]()
+ // Holds the list of tasks that needs to reconcile with Mesos master.
+ // All states that are loaded after failover are added here.
+ private val pendingRecover = new mutable.HashMap[String, SlaveID]()
+ // A queue that stores all the submitted drivers that hasn't been
launched.
+ val queuedDrivers = new ArrayBuffer[MesosDriverDescription]()
+ // All supervised drivers that are waiting to retry after termination.
+ val pendingRetryDrivers = new ArrayBuffer[MesosDriverDescription]()
+ private val queuedDriversState =
engineFactory.createEngine("driverQueue")
+ private val launchedDriversState =
engineFactory.createEngine("launchedDrivers")
+ private val pendingRetryDriversState =
engineFactory.createEngine("retryList")
+ // Flag to mark if the scheduler is ready to be called, which is until
the scheduler
+ // is registered with Mesos master.
+ protected var ready = false
+ private var masterInfo: Option[MasterInfo] = None
+
+ private def isQueueFull(): Boolean = launchedDrivers.size >=
queuedCapacity
+
+ def submitDriver(desc: MesosDriverDescription): CreateSubmissionResponse
= {
+ val c = new CreateSubmissionResponse
+ if (!ready) {
+ c.success = false
+ c.message = "Scheduler is not ready to take requests"
+ return c
+ }
+
+ stateLock.synchronized {
+ if (isQueueFull()) {
+ c.success = false
+ c.message = "Already reached maximum submission size"
+ }
+ c.submissionId = desc.submissionId
+ queuedDrivers += desc
+ c.success = true
+ }
+ c
+ }
+
+ private def removeFromQueuedDrivers(id: String): Boolean = {
+ val index = queuedDrivers.indexWhere(_.submissionId.equals(id))
+ if (index != -1) {
+ queuedDrivers.remove(index)
+ queuedDriversState.expunge(id)
+ true
+ } else {
+ false
+ }
+ }
+
+ private def removeFromLaunchedDrivers(id: String): Boolean = {
+ if (launchedDrivers.remove(id).isDefined) {
+ launchedDriversState.expunge(id)
+ true
+ } else {
+ false
+ }
+ }
+
+ private def removeFromPendingRetryDrivers(id: String): Boolean = {
+ val index = pendingRetryDrivers.indexWhere(_.submissionId.equals(id))
+ if (index != -1) {
+ pendingRetryDrivers.remove(index)
+ pendingRetryDriversState.expunge(id)
+ true
+ } else {
+ false
+ }
+ }
+
+ def killDriver(submissionId: String): KillSubmissionResponse = {
+ val k = new KillSubmissionResponse
+ if (!ready) {
+ k.success = false
+ k.message = "Scheduler is not ready to take requests"
+ return k
+ }
+ k.submissionId = submissionId
+ stateLock.synchronized {
+ // We look for the requested driver in the following places:
+ // 1. Check if submission is running or launched.
+ // 2. Check if it's still queued.
+ // 3. Check if it's in the retry list.
+ // 4. Check if it has already completed.
+ if (launchedDrivers.contains(submissionId)) {
+ val task = launchedDrivers(submissionId)
+ driver.killTask(task.taskId)
+ k.success = true
+ k.message = "Killing running driver"
+ } else if (removeFromQueuedDrivers(submissionId)) {
+ k.success = true
+ k.message = "Removed driver while it's still pending"
+ } else if (removeFromPendingRetryDrivers(submissionId)) {
+ k.success = true
+ k.message = "Removed driver while it's retrying"
+ } else if
(finishedDrivers.exists(_.submission.submissionId.equals(submissionId))) {
+ k.success = false
+ k.message = "Driver already terminated"
+ } else {
+ k.success = false
+ k.message = "Cannot find driver"
+ }
+ }
+ k
+ }
+
+ /**
+ * Recover scheduler state that is persisted.
+ * We still need to do task reconciliation to be up to date of the
latest task states
+ * as it might have changed while the scheduler is failing over.
+ */
+ private def recoverState(): Unit = {
+ stateLock.synchronized {
+ launchedDriversState.fetchAll[MesosClusterTaskState]().foreach {
state =>
+ launchedDrivers(state.taskId.getValue) = state
+ pendingRecover(state.taskId.getValue) = state.slaveId
+ }
+ queuedDriversState.fetchAll[MesosDriverDescription]().foreach(d =>
queuedDrivers += d)
+ // There is potential timing issue where a queued driver might have
been launched
+ // but the scheduler shuts down before the queued driver was able to
be removed
+ // from the queue. We try to mitigate this issue by walking through
all queued drivers
+ // and remove if they're already launched.
+ queuedDrivers
+ .filter(d => launchedDrivers.contains(d.submissionId))
+ .toSeq
--- End diff --
I'm later removing from the queuedDrivers again so want to make sure I
materialize the filter before I modify the list.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]