Github user andrewor14 commented on a diff in the pull request:
https://github.com/apache/spark/pull/5144#discussion_r28026705
--- Diff:
core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala
---
@@ -0,0 +1,546 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util.concurrent.atomic.AtomicLong
+import java.util.concurrent.locks.ReentrantLock
+import java.util.{Collections, Date, List => JList}
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+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.{SparkConf, SparkException, SecurityManager,
TaskState}
+
+
+/**
+ * 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
+ * @param retryState Retry state for this task (only applicable to
supervised drivers)
+ */
+private[spark] class MesosClusterTaskState(
+ val submission: MesosDriverDescription,
+ val taskId: TaskID,
+ val slaveId: SlaveID,
+ var taskState: Option[TaskStatus],
+ var startDate: Date,
+ val retryState: Option[RetryState] = None)
+ extends Serializable {
+
+ def copy(): MesosClusterTaskState = {
+ new MesosClusterTaskState(
+ submission, taskId, slaveId, taskState, startDate, retryState)
+ }
+}
+
+/**
+ * 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[RetryState])
+
+/**
+ * Mesos cluster scheduler for running, killing and requesting
+ * status of Spark drivers that will be launched in a Mesos cluster.
+ * This interface is mainly for
[[org.apache.spark.deploy.rest.mesos.MesosRestServer]] and
+ * [[org.apache.spark.deploy.mesos.ui.MesosClusterPage]] to interact with
+ * [[MesosClusterSchedulerDriver]] and hide all the Mesos specific methods
that it doesn't
+ * need to care about.
+ */
+private[spark] trait MesosClusterScheduler {
+ def submitDriver(desc: MesosDriverDescription): CreateSubmissionResponse
+ def killDriver(submissionId: String): KillSubmissionResponse
+ def getStatus(submissionId: String): SubmissionStatusResponse
+ def getState(): MesosClusterSchedulerState
+}
+
+/**
+ * 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 MesosClusterSchedulerDriver(
+ engineFactory: MesosClusterPersistenceEngineFactory,
+ conf: SparkConf)
+ extends Scheduler with MesosSchedulerUtils with MesosClusterScheduler {
+
+ 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 state = engineFactory.createEngine("scheduler")
+ private val stateLock = new ReentrantLock()
+ private val finishedDrivers = new
mutable.ArrayBuffer[MesosClusterTaskState](retainedDrivers)
+ private val nextDriverNumber: AtomicLong = new AtomicLong(0)
+ private var frameworkId: String = null
+
+ // Stores all the launched and running drivers' states.
+ var launchedDrivers: LaunchedDrivers = _
+
+ // A queue that stores all the submitted drivers that hasn't been
launched.
+ var queue: DriverQueue = _
+
+ // All supervised drivers that are waiting to retry after termination.
+ var superviseRetryList: SuperviseRetryList = _
+
+ private var masterInfo: Option[MasterInfo] = None
+
+ private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss")
// For application IDs
+ private def newDriverId(submitDate: Date): String = {
+ "driver-%s-%04d".format(
+ createDateFormat.format(submitDate),
nextDriverNumber.incrementAndGet())
+ }
+
+ def submitDriver(desc: MesosDriverDescription): CreateSubmissionResponse
= {
+ val c = new CreateSubmissionResponse
+ val submitDate = new Date()
+ desc.submissionId = Some(newDriverId(submitDate))
+ desc.submissionDate = Some(submitDate)
+
+ stateLock.synchronized {
+ if (queue.isFull) {
+ c.success = false
+ c.message = "Already reached maximum submission size"
+ }
+ c.submissionId = desc.submissionId.get
+ queue.offer(desc)
+ c.success = true
+ }
+ c
+ }
+
+ def killDriver(submissionId: String): KillSubmissionResponse = {
+ val k = new KillSubmissionResponse
+ 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.get(submissionId)
+ driver.killTask(task.taskId)
+ k.success = true
+ k.message = "Killing running driver"
+ } else if (queue.remove(submissionId)) {
+ k.success = true
+ k.message = "Removed driver while it's still pending"
+ } else if (superviseRetryList.remove(submissionId)) {
+ k.success = true
+ k.message = "Removed driver while it's retrying"
+ } else if (finishedDrivers.exists(s =>
s.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.
+ */
+ def recoverState: Unit = {
--- End diff --
This is not a getter without side-effects. The convention is to use `()` to
denote such methods.
---
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]