Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -369,7 +369,8 @@ private[spark] class MesosClusterScheduler(
}

private def getDriverFrameworkID(desc: MesosDriverDescription): String = {
s"${frameworkId}-${desc.submissionId}"
val retries = desc.retryState.map { d => s"-retry-${d.retries.toString}" }.getOrElse("")
s"${frameworkId}-${desc.submissionId}${retries}"
}

private def adjust[A, B](m: collection.Map[A, B], k: A, default: B)(f: B => B) = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.spark.scheduler.cluster.mesos

import java.io.File
import java.util.{Collections, List => JList}
import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong}
import java.util.concurrent.locks.ReentrantLock

import scala.collection.JavaConverters._
Expand Down Expand Up @@ -170,6 +171,15 @@ private[spark] class MesosCoarseGrainedSchedulerBackend(

override def start() {
super.start()

val startedBefore = IdHelper.startedBefore.getAndSet(true)

val suffix = if (startedBefore) {
f"-${IdHelper.nextSCNumber.incrementAndGet()}%04d"
} else {
""
}

val driver = createSchedulerDriver(
master,
MesosCoarseGrainedSchedulerBackend.this,
Expand All @@ -179,10 +189,9 @@ private[spark] class MesosCoarseGrainedSchedulerBackend(
sc.conf.getOption("spark.mesos.driver.webui.url").orElse(sc.ui.map(_.webUrl)),
None,
Some(sc.conf.get(DRIVER_FAILOVER_TIMEOUT)),
sc.conf.getOption("spark.mesos.driver.frameworkId")
sc.conf.getOption("spark.mesos.driver.frameworkId").map(_ + suffix)
)

unsetFrameworkID(sc)
startScheduler(driver)
}

Expand Down Expand Up @@ -271,6 +280,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend(
driver: org.apache.mesos.SchedulerDriver,
frameworkId: FrameworkID,
masterInfo: MasterInfo) {

this.appId = frameworkId.getValue
this.mesosExternalShuffleClient.foreach(_.init(appId))
this.schedulerDriver = driver
Expand Down Expand Up @@ -672,3 +682,9 @@ private class Slave(val hostname: String) {
var taskFailures = 0
var shuffleRegistered = false
}

object IdHelper {
// Use atomic values since Spark contexts can be initialized in parallel
private[mesos] val nextSCNumber = new AtomicLong(0)
private[mesos] val startedBefore = new AtomicBoolean(false)
}