Skip to content

Commit

Permalink
Merge branch 'master' into stratified
Browse files Browse the repository at this point in the history
  • Loading branch information
dorx committed Aug 9, 2014
2 parents 555a3f9 + 28dbae8 commit e990325
Show file tree
Hide file tree
Showing 9 changed files with 54 additions and 44 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,7 @@ private[spark] class Worker(
logInfo("Spark home: " + sparkHome)
createWorkDir()
context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
webUi = new WorkerWebUI(this, workDir, Some(webUiPort))
webUi = new WorkerWebUI(this, workDir, webUiPort)
webUi.bind()
registerWithMaster()

Expand Down Expand Up @@ -373,7 +373,8 @@ private[spark] class Worker(
private[spark] object Worker extends Logging {
def main(argStrings: Array[String]) {
SignalLogger.register(log)
val args = new WorkerArguments(argStrings)
val conf = new SparkConf
val args = new WorkerArguments(argStrings, conf)
val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores,
args.memory, args.masters, args.workDir)
actorSystem.awaitTermination()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,12 @@ package org.apache.spark.deploy.worker
import java.lang.management.ManagementFactory

import org.apache.spark.util.{IntParam, MemoryParam, Utils}
import org.apache.spark.SparkConf

/**
* Command-line parser for the worker.
*/
private[spark] class WorkerArguments(args: Array[String]) {
private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) {
var host = Utils.localHostName()
var port = 0
var webUiPort = 8081
Expand All @@ -46,6 +47,9 @@ private[spark] class WorkerArguments(args: Array[String]) {
if (System.getenv("SPARK_WORKER_WEBUI_PORT") != null) {
webUiPort = System.getenv("SPARK_WORKER_WEBUI_PORT").toInt
}
if (conf.contains("spark.worker.ui.port")) {
webUiPort = conf.get("spark.worker.ui.port").toInt
}
if (System.getenv("SPARK_WORKER_DIR") != null) {
workDir = System.getenv("SPARK_WORKER_DIR")
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,8 +34,8 @@ private[spark]
class WorkerWebUI(
val worker: Worker,
val workDir: File,
port: Option[Int] = None)
extends WebUI(worker.securityMgr, getUIPort(port, worker.conf), worker.conf, name = "WorkerUI")
requestedPort: Int)
extends WebUI(worker.securityMgr, requestedPort, worker.conf, name = "WorkerUI")
with Logging {

val timeout = AkkaUtils.askTimeout(worker.conf)
Expand All @@ -55,10 +55,5 @@ class WorkerWebUI(
}

private[spark] object WorkerWebUI {
val DEFAULT_PORT = 8081
val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR

def getUIPort(requestedPort: Option[Int], conf: SparkConf): Int = {
requestedPort.getOrElse(conf.getInt("spark.worker.ui.port", WorkerWebUI.DEFAULT_PORT))
}
}
3 changes: 1 addition & 2 deletions core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -83,8 +83,7 @@ class UnionRDD[T: ClassTag](

override def compute(s: Partition, context: TaskContext): Iterator[T] = {
val part = s.asInstanceOf[UnionPartition[T]]
val parentRdd = dependencies(part.parentRddIndex).rdd.asInstanceOf[RDD[T]]
parentRdd.iterator(part.parentPartition, context)
parent[T](part.parentRddIndex).iterator(part.parentPartition, context)
}

override def getPreferredLocations(s: Partition): Seq[String] =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,19 +47,19 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
{
// Use an atomic variable to track total number of cores in the cluster for simplicity and speed
var totalCoreCount = new AtomicInteger(0)
var totalExpectedExecutors = new AtomicInteger(0)
var totalRegisteredExecutors = new AtomicInteger(0)
val conf = scheduler.sc.conf
private val timeout = AkkaUtils.askTimeout(conf)
private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf)
// Submit tasks only after (registered executors / total expected executors)
// Submit tasks only after (registered resources / total expected resources)
// is equal to at least this value, that is double between 0 and 1.
var minRegisteredRatio = conf.getDouble("spark.scheduler.minRegisteredExecutorsRatio", 0)
if (minRegisteredRatio > 1) minRegisteredRatio = 1
// Whatever minRegisteredExecutorsRatio is arrived, submit tasks after the time(milliseconds).
var minRegisteredRatio =
math.min(1, conf.getDouble("spark.scheduler.minRegisteredResourcesRatio", 0))
// Submit tasks after maxRegisteredWaitingTime milliseconds
// if minRegisteredRatio has not yet been reached
val maxRegisteredWaitingTime =
conf.getInt("spark.scheduler.maxRegisteredExecutorsWaitingTime", 30000)
conf.getInt("spark.scheduler.maxRegisteredResourcesWaitingTime", 30000)
val createTime = System.currentTimeMillis()
var ready = if (minRegisteredRatio <= 0) true else false

class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor {
private val executorActor = new HashMap[String, ActorRef]
Expand Down Expand Up @@ -94,12 +94,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
executorAddress(executorId) = sender.path.address
addressToExecutorId(sender.path.address) = executorId
totalCoreCount.addAndGet(cores)
if (executorActor.size >= totalExpectedExecutors.get() * minRegisteredRatio && !ready) {
ready = true
logInfo("SchedulerBackend is ready for scheduling beginning, registered executors: " +
executorActor.size + ", total expected executors: " + totalExpectedExecutors.get() +
", minRegisteredExecutorsRatio: " + minRegisteredRatio)
}
totalRegisteredExecutors.addAndGet(1)
makeOffers()
}

Expand Down Expand Up @@ -268,14 +263,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
}
}

def sufficientResourcesRegistered(): Boolean = true

override def isReady(): Boolean = {
if (ready) {
if (sufficientResourcesRegistered) {
logInfo("SchedulerBackend is ready for scheduling beginning after " +
s"reached minRegisteredResourcesRatio: $minRegisteredRatio")
return true
}
if ((System.currentTimeMillis() - createTime) >= maxRegisteredWaitingTime) {
ready = true
logInfo("SchedulerBackend is ready for scheduling beginning after waiting " +
"maxRegisteredExecutorsWaitingTime: " + maxRegisteredWaitingTime)
s"maxRegisteredResourcesWaitingTime: $maxRegisteredWaitingTime(ms)")
return true
}
false
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ private[spark] class SparkDeploySchedulerBackend(
var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _

val maxCores = conf.getOption("spark.cores.max").map(_.toInt)
val totalExpectedCores = maxCores.getOrElse(0)

override def start() {
super.start()
Expand Down Expand Up @@ -97,7 +98,6 @@ private[spark] class SparkDeploySchedulerBackend(

override def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int,
memory: Int) {
totalExpectedExecutors.addAndGet(1)
logInfo("Granted executor ID %s on hostPort %s with %d cores, %s RAM".format(
fullId, hostPort, cores, Utils.megabytesToString(memory)))
}
Expand All @@ -110,4 +110,8 @@ private[spark] class SparkDeploySchedulerBackend(
logInfo("Executor %s removed: %s".format(fullId, message))
removeExecutor(fullId.split("/")(1), reason.toString)
}

override def sufficientResourcesRegistered(): Boolean = {
totalCoreCount.get() >= totalExpectedCores * minRegisteredRatio
}
}
13 changes: 7 additions & 6 deletions docs/configuration.md
Original file line number Diff line number Diff line change
Expand Up @@ -825,21 +825,22 @@ Apart from these, the following properties are also available, and may be useful
</td>
</tr>
</tr>
<td><code>spark.scheduler.minRegisteredExecutorsRatio</code></td>
<td><code>spark.scheduler.minRegisteredResourcesRatio</code></td>
<td>0</td>
<td>
The minimum ratio of registered executors (registered executors / total expected executors)
The minimum ratio of registered resources (registered resources / total expected resources)
(resources are executors in yarn mode, CPU cores in standalone mode)
to wait for before scheduling begins. Specified as a double between 0 and 1.
Regardless of whether the minimum ratio of executors has been reached,
Regardless of whether the minimum ratio of resources has been reached,
the maximum amount of time it will wait before scheduling begins is controlled by config
<code>spark.scheduler.maxRegisteredExecutorsWaitingTime</code>
<code>spark.scheduler.maxRegisteredResourcesWaitingTime</code>
</td>
</tr>
<tr>
<td><code>spark.scheduler.maxRegisteredExecutorsWaitingTime</code></td>
<td><code>spark.scheduler.maxRegisteredResourcesWaitingTime</code></td>
<td>30000</td>
<td>
Maximum amount of time to wait for executors to register before scheduling begins
Maximum amount of time to wait for resources to register before scheduling begins
(in milliseconds).
</td>
</tr>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,15 +30,15 @@ private[spark] class YarnClientSchedulerBackend(
extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
with Logging {

if (conf.getOption("spark.scheduler.minRegisteredExecutorsRatio").isEmpty) {
if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
minRegisteredRatio = 0.8
ready = false
}

var client: Client = null
var appId: ApplicationId = null
var checkerThread: Thread = null
var stopping: Boolean = false
var totalExpectedExecutors = 0

private[spark] def addArg(optionName: String, envVar: String, sysProp: String,
arrayBuf: ArrayBuffer[String]) {
Expand Down Expand Up @@ -84,7 +84,7 @@ private[spark] class YarnClientSchedulerBackend(

logDebug("ClientArguments called with: " + argsArrayBuf)
val args = new ClientArguments(argsArrayBuf.toArray, conf)
totalExpectedExecutors.set(args.numExecutors)
totalExpectedExecutors = args.numExecutors
client = new Client(args, conf)
appId = client.runApp()
waitForApp()
Expand Down Expand Up @@ -150,4 +150,7 @@ private[spark] class YarnClientSchedulerBackend(
logInfo("Stopped")
}

override def sufficientResourcesRegistered(): Boolean = {
totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,19 +27,24 @@ private[spark] class YarnClusterSchedulerBackend(
sc: SparkContext)
extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) {

if (conf.getOption("spark.scheduler.minRegisteredExecutorsRatio").isEmpty) {
var totalExpectedExecutors = 0

if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
minRegisteredRatio = 0.8
ready = false
}

override def start() {
super.start()
var numExecutors = ApplicationMasterArguments.DEFAULT_NUMBER_EXECUTORS
totalExpectedExecutors = ApplicationMasterArguments.DEFAULT_NUMBER_EXECUTORS
if (System.getenv("SPARK_EXECUTOR_INSTANCES") != null) {
numExecutors = IntParam.unapply(System.getenv("SPARK_EXECUTOR_INSTANCES")).getOrElse(numExecutors)
totalExpectedExecutors = IntParam.unapply(System.getenv("SPARK_EXECUTOR_INSTANCES"))
.getOrElse(totalExpectedExecutors)
}
// System property can override environment variable.
numExecutors = sc.getConf.getInt("spark.executor.instances", numExecutors)
totalExpectedExecutors.set(numExecutors)
totalExpectedExecutors = sc.getConf.getInt("spark.executor.instances", totalExpectedExecutors)
}

override def sufficientResourcesRegistered(): Boolean = {
totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio
}
}

0 comments on commit e990325

Please sign in to comment.