From d9a3e720f6f6bce93e6b5f88b7c45a37866cf63f Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 30 Jun 2015 16:52:15 +0800 Subject: [PATCH 01/27] Add a new Receiver scheduling mechanism --- .../scala/org/apache/spark/SparkContext.scala | 23 +++ .../receiver/ReceiverSupervisor.scala | 17 +- .../receiver/ReceiverSupervisorImpl.scala | 8 +- .../scheduler/ReceiverScheduler.scala | 84 ++++++++ .../streaming/scheduler/ReceiverTracker.scala | 191 +++++++++++++++--- 5 files changed, 286 insertions(+), 37 deletions(-) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d2547eeff2b4..b9d30b5d8e23 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1859,6 +1859,29 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli new SimpleFutureAction(waiter, resultFunc) } + /** + * Submit a job for execution and return a FutureJob holding the result. + */ + private[spark] def submitAsyncJob[T, U, R]( + rdd: RDD[T], + processPartition: (TaskContext, Iterator[T]) => U, + resultHandler: (Int, U) => Unit, + resultFunc: => R): SimpleFutureAction[R] = + { + assertNotStopped() + val cleanF = clean(processPartition) + val callSite = getCallSite + val waiter = dagScheduler.submitJob( + rdd, + (context: TaskContext, iter: Iterator[T]) => cleanF(context, iter), + 0 until rdd.partitions.size, + callSite, + allowLocal = false, + resultHandler, + localProperties.get) + new SimpleFutureAction(waiter, resultFunc) + } + /** * Cancel active jobs for the specified group. See [[org.apache.spark.SparkContext.setJobGroup]] * for more information. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index 33be067ebdaf..835b4fadb0a6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -25,7 +25,7 @@ import scala.concurrent._ import org.apache.spark.{Logging, SparkConf} import org.apache.spark.storage.StreamBlockId -import org.apache.spark.util.ThreadUtils +import org.apache.spark.util.{Utils, ThreadUtils} /** * Abstract class that is responsible for supervising a Receiver in the worker. @@ -64,6 +64,8 @@ private[streaming] abstract class ReceiverSupervisor( /** State of the receiver */ @volatile private[streaming] var receiverState = Initialized + val host = Utils.localHostName() + /** Push a single data item to backend data store. */ def pushSingle(data: Any) @@ -161,12 +163,19 @@ private[streaming] abstract class ReceiverSupervisor( stopReceiver("Restarting receiver with delay " + delay + "ms: " + message, error) logDebug("Sleeping for " + delay) Thread.sleep(delay) - logInfo("Starting receiver again") - startReceiver() - logInfo("Receiver started again") + if (rescheduleReceiver().contains(host)) { + logInfo("Starting receiver again") + startReceiver() + logInfo("Receiver started again") + } else { + stop("Receiver is scheduled to another executor", None) + } }(futureExecutionContext) } + /** Reschedule this receiver and return a candidate executor list */ + def rescheduleReceiver(): Seq[String] + /** Check if receiver has been marked for stopping */ def isReceiverStarted(): Boolean = { logDebug("state = " + receiverState) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 6078cdf8f879..c44b408b03c1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -30,7 +30,7 @@ import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.Time import org.apache.spark.streaming.scheduler._ import org.apache.spark.streaming.util.WriteAheadLogUtils -import org.apache.spark.util.{RpcUtils, Utils} +import org.apache.spark.util.RpcUtils import org.apache.spark.{Logging, SparkEnv, SparkException} /** @@ -164,7 +164,7 @@ private[streaming] class ReceiverSupervisorImpl( override protected def onReceiverStart() { val msg = RegisterReceiver( - streamId, receiver.getClass.getSimpleName, Utils.localHostName(), endpoint) + streamId, receiver.getClass.getSimpleName, host, endpoint) trackerEndpoint.askWithRetry[Boolean](msg) } @@ -182,4 +182,8 @@ private[streaming] class ReceiverSupervisorImpl( logDebug(s"Cleaning up blocks older then $cleanupThreshTime") receivedBlockHandler.cleanupOldBlocks(cleanupThreshTime.milliseconds) } + + override def rescheduleReceiver(): Seq[String] = { + trackerEndpoint.askWithRetry[Seq[String]](ScheduleReceiver(streamId)) + } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala new file mode 100644 index 000000000000..65f571a6218e --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala @@ -0,0 +1,84 @@ +/* + * 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.streaming.scheduler + +import scala.collection.mutable +import scala.util.Random + +import org.apache.spark.streaming.scheduler.ReceiverState._ + +private[streaming] case class ReceiverTrackingInfo( + receiverId: Int, + state: ReceiverState, + preferredLocation: Option[String], + scheduledLocations: Option[Seq[String]], + runningLocation: Option[String]) + +private[streaming] trait ReceiverScheduler { + + def scheduleReceiver( + receiverId: Int, + preferredLocation: Option[String], + receiverTrackingInfoMap: Map[Int, ReceiverTrackingInfo], + executors: Seq[String]): Seq[String] +} + +/** + * A ReceiverScheduler trying to balance executors' load. + */ +private[streaming] class LoadBalanceReceiverSchedulerImpl extends ReceiverScheduler { + + def scheduleReceiver( + receiverId: Int, + preferredLocation: Option[String], + receiverTrackingInfoMap: Map[Int, ReceiverTrackingInfo], + executors: Seq[String]): Seq[String] = { + if (executors.isEmpty) { + return Seq.empty + } + + // Always try to schedule to the preferred locations + val locations = mutable.Set[String]() + locations ++= preferredLocation + + val executorWeights = receiverTrackingInfoMap.filter { case (id, _) => + // Ignore the receiver to be scheduled. It may be still running. + id != receiverId + }.values.flatMap { receiverTrackingInfo => + receiverTrackingInfo.state match { + case ReceiverState.INACTIVE => Nil + case ReceiverState.SCHEDULED => + val scheduledLocations = receiverTrackingInfo.scheduledLocations.get + // The probability that a scheduled receiver will run in an executor is + // 1.0 / scheduledLocations.size + scheduledLocations.map(location => (location, 1.0 / scheduledLocations.size)) + case ReceiverState.ACTIVE => Seq(receiverTrackingInfo.runningLocation.get -> 1.0) + } + }.groupBy(_._1).mapValues(_.map(_._2).sum) // Sum weights for each executor + + val idleExecutors = (executors.toSet -- executorWeights.keys).toSeq + if (idleExecutors.nonEmpty) { + // If there are idle executors, randomly select one + locations += idleExecutors(Random.nextInt(idleExecutors.size)) + } else { + // Use the executor that runs the least receivers + locations += executorWeights.minBy(_._2)._1 + } + locations.toSeq + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 644e581cd827..d708167ee2d4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -17,18 +17,29 @@ package org.apache.spark.streaming.scheduler +import java.util.concurrent.CountDownLatch + import scala.collection.mutable.{ArrayBuffer, HashMap, SynchronizedMap} +import scala.concurrent.ExecutionContext import scala.language.existentials import scala.math.max -import org.apache.spark.rdd._ +import scala.util.{Failure, Success} import org.apache.spark.streaming.util.WriteAheadLogUtils -import org.apache.spark.{Logging, SparkEnv, SparkException} +import org.apache.spark.{TaskContext, Logging, SparkEnv, SparkException} +import org.apache.spark.rdd.RDD import org.apache.spark.rpc._ import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.receiver.{CleanupOldBlocks, Receiver, ReceiverSupervisorImpl, StopReceiver} -import org.apache.spark.util.SerializableConfiguration +import org.apache.spark.util.{ThreadUtils, SerializableConfiguration} + + +/** Enumeration to identify current state of a Receiver */ +private[streaming] object ReceiverState extends Enumeration { + type ReceiverState = Value + val INACTIVE, SCHEDULED, ACTIVE = Value +} /** * Messages used by the NetworkReceiver and the ReceiverTracker to communicate @@ -47,6 +58,9 @@ private[streaming] case class ReportError(streamId: Int, message: String, error: private[streaming] case class DeregisterReceiver(streamId: Int, msg: String, error: String) extends ReceiverTrackerMessage +/** It's used to ask ReceiverTracker to return a candidate executor list to run the receiver */ +private[streaming] case class ScheduleReceiver(streamId: Int) extends ReceiverTrackerMessage + /** * This class manages the execution of the receivers of ReceiverInputDStreams. Instance of * this class must be created after all input streams have been added and StreamingContext.start() @@ -75,6 +89,19 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false // This not being null means the tracker has been started and not stopped private var endpoint: RpcEndpointRef = null + private val scheduler: ReceiverScheduler = new LoadBalanceReceiverSchedulerImpl() + + @volatile private var stopping = false + + // Track receivers' status for scheduling + private val receiverTrackingInfos = new HashMap[Int, ReceiverTrackingInfo] + + // Store all preferred locations for all receivers. We need this information to schedule receivers + private val receiverPreferredLocations = new HashMap[Int, Option[String]] + + // Use a separate lock to avoid dead-lock + private val receiverTrackingInfosLock = new AnyRef + /** Start the endpoint and receiver execution thread. */ def start(): Unit = synchronized { if (endpoint != null) { @@ -151,6 +178,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } receiverInfo(streamId) = ReceiverInfo( streamId, s"${typ}-${streamId}", receiverEndpoint, true, host) + updateReceiverRunningLocation(streamId, host) listenerBus.post(StreamingListenerReceiverStarted(receiverInfo(streamId))) logInfo("Registered receiver for stream " + streamId + " from " + senderAddress) } @@ -227,6 +255,8 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false case DeregisterReceiver(streamId, message, error) => deregisterReceiver(streamId, message, error) context.reply(true) + case ScheduleReceiver(streamId) => + context.reply(scheduleReceiver(streamId)) } } @@ -250,6 +280,8 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } def stop(graceful: Boolean) { + stopping = true + // Send the stop signal to all the receivers stopReceivers() @@ -274,15 +306,6 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } } - /** - * Get the list of executors excluding driver - */ - private def getExecutors(ssc: StreamingContext): List[String] = { - val executors = ssc.sparkContext.getExecutorMemoryStatus.map(_._1.split(":")(0)).toList - val driver = ssc.sparkContext.getConf.get("spark.driver.host") - executors.diff(List(driver)) - } - /** Set host location(s) for each receiver so as to distribute them over * executors in a round-robin fashion taking into account preferredLocation if set */ @@ -309,6 +332,9 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false locations } + private val submitJobThread = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("streaming-submit-job")) + /** * Get the receivers from the ReceiverInputDStreams, distributes them to the * worker nodes as a parallel collection, and runs them. @@ -320,21 +346,30 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false rcvr }) + initReceiverTrackingInfos(receivers) + + // Tracking the active receiver number. When a receiver exits, countDown will be called. + val receiverExitLatch = new CountDownLatch(receivers.size) + val checkpointDirOption = Option(ssc.checkpointDir) val serializableHadoopConf = new SerializableConfiguration(ssc.sparkContext.hadoopConfiguration) // Function to start the receiver on the worker node - val startReceiver = (iterator: Iterator[Receiver[_]]) => { + val startReceiver = (context: TaskContext, iterator: Iterator[Receiver[_]]) => { if (!iterator.hasNext) { throw new SparkException( "Could not start receiver as object not found.") } - val receiver = iterator.next() - val supervisor = new ReceiverSupervisorImpl( - receiver, SparkEnv.get, serializableHadoopConf.value, checkpointDirOption) - supervisor.start() - supervisor.awaitTermination() + if (context.attemptNumber() == 0) { + val receiver = iterator.next() + val supervisor = new ReceiverSupervisorImpl( + receiver, SparkEnv.get, serializableHadoopConf.value, checkpointDirOption) + supervisor.start() + supervisor.awaitTermination() + } else { + // It's restarted by TaskScheduler, but we want to reschedule it again. So exit it. + } } // Run the dummy Spark job to ensure that all slaves have registered. @@ -343,24 +378,60 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect() } - // Get the list of executors and schedule receivers - val executors = getExecutors(ssc) - val tempRDD = - if (!executors.isEmpty) { - val locations = scheduleReceivers(receivers, executors) - val roundRobinReceivers = (0 until receivers.length).map(i => - (receivers(i), locations(i))) - ssc.sc.makeRDD[Receiver[_]](roundRobinReceivers) - } else { - ssc.sc.makeRDD(receivers, receivers.size) - } - // Distribute the receivers and start them logInfo("Starting " + receivers.length + " receivers") running = true - ssc.sparkContext.runJob(tempRDD, ssc.sparkContext.clean(startReceiver)) + for (receiver <- receivers) { + submitJobThread.execute(new Runnable { + override def run(): Unit = { + if (stopping) { + receiverExitLatch.countDown() + return + } + + val self = this + val scheduledLocations = scheduler.scheduleReceiver( + receiver.streamId, + receiver.preferredLocation, + getReceiverTrackingInfoMap(), + getExecutors(ssc)) + updateReceiverScheduledLocations(receiver.streamId, scheduledLocations) + val receiverRDD: RDD[Receiver[_]] = + if (scheduledLocations.isEmpty) { + ssc.sc.makeRDD(Seq(receiver), 1) + } else { + ssc.sc.makeRDD(Seq(receiver -> scheduledLocations)) + } + val future = ssc.sparkContext.submitAsyncJob[Receiver[_], Unit, Unit]( + receiverRDD, startReceiver, (_, _) => Unit, ()) + // TODO Refactor JobWaiter to avoid creating a new Thread here. Otherwise, it's not + // scalable because we need a thread for each Receiver. + future.onComplete { + case Success(_) => + if (stopping) { + receiverExitLatch.countDown() + } else { + logInfo(s"Restarting Receiver ${receiver.streamId}") + submitJobThread.execute(self) + } + case Failure(e) => + if (stopping) { + receiverExitLatch.countDown() + } else { + logError("Receiver has been stopped. Try to restart it.", e) + logInfo(s"Restarting Receiver ${receiver.streamId}") + submitJobThread.execute(self) + } + }(submitJobThread) + logInfo(s"Receiver ${receiver.streamId} started") + } + }) + } + // Wait until all receivers exit + receiverExitLatch.await() running = false logInfo("All of the receivers have been terminated") + submitJobThread.shutdownNow() } /** Stops the receivers. */ @@ -371,4 +442,62 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false logInfo("Sent stop signal to all " + receiverInfo.size + " receivers") } } + + def initReceiverTrackingInfos(receivers: Seq[Receiver[_]]): Unit = synchronized { + for (receiver <- receivers) { + receiverPreferredLocations(receiver.streamId) = receiver.preferredLocation + receiverTrackingInfos.put(receiver.streamId, ReceiverTrackingInfo( + receiver.streamId, + ReceiverState.INACTIVE, + receiver.preferredLocation, + None, + None)) + } + } + + private def getReceiverTrackingInfoMap(): Map[Int, ReceiverTrackingInfo] = + receiverTrackingInfosLock.synchronized { + // Copy to an immutable Map so that we don't need to use `synchronized` when using it + receiverTrackingInfos.toMap + } + + private def updateReceiverScheduledLocations( + receiverId: Int, scheduledLocations: Seq[String]): Unit = + receiverTrackingInfosLock.synchronized { + receiverTrackingInfos.put(receiverId, ReceiverTrackingInfo( + receiverId, + ReceiverState.SCHEDULED, + receiverPreferredLocations(receiverId), + Some(scheduledLocations), + None)) + } + + private def updateReceiverRunningLocation(receiverId: Int, runningLocation: String): Unit = + receiverTrackingInfosLock.synchronized { + receiverTrackingInfos.put(receiverId, ReceiverTrackingInfo( + receiverId, + ReceiverState.ACTIVE, + receiverPreferredLocations(receiverId), + None, + Some(runningLocation))) + } + + private def scheduleReceiver(receiverId: Int): Seq[String] = { + val preferredLocation = receiverTrackingInfosLock.synchronized { + receiverPreferredLocations(receiverId) + } + val scheduledLocations = scheduler.scheduleReceiver( + receiverId, preferredLocation, getReceiverTrackingInfoMap(), getExecutors(ssc)) + updateReceiverScheduledLocations(receiverId, scheduledLocations) + scheduledLocations + } + + /** + * Get the list of executors excluding driver + */ + private def getExecutors(ssc: StreamingContext): List[String] = { + val executors = ssc.sparkContext.getExecutorMemoryStatus.map(_._1.split(":")(0)).toList + val driver = ssc.sparkContext.getConf.get("spark.driver.host") + executors.diff(List(driver)) + } } From cc7614207b8260e43b6792fa60934d19ada00af6 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 8 Jul 2015 20:16:36 +0800 Subject: [PATCH 02/27] Add JobWaiter.toFuture to avoid blocking threads --- .../main/scala/org/apache/spark/SparkContext.scala | 11 ++++++----- .../scala/org/apache/spark/scheduler/JobWaiter.scala | 12 ++++++++++++ .../streaming/receiver/ReceiverSupervisor.scala | 5 +++-- .../streaming/scheduler/ReceiverScheduler.scala | 5 +++++ .../spark/streaming/scheduler/ReceiverTracker.scala | 11 ++++++----- 5 files changed, 32 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index b9d30b5d8e23..e236b3d078b8 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -17,8 +17,6 @@ package org.apache.spark -import scala.language.implicitConversions - import java.io._ import java.lang.reflect.Constructor import java.net.URI @@ -30,6 +28,8 @@ import scala.collection.{Map, Set} import scala.collection.JavaConversions._ import scala.collection.generic.Growable import scala.collection.mutable.HashMap +import scala.concurrent.Future +import scala.language.implicitConversions import scala.reflect.{ClassTag, classTag} import scala.util.control.NonFatal @@ -1860,13 +1860,14 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } /** - * Submit a job for execution and return a FutureJob holding the result. + * Submit a job for execution and return a FutureJob holding the result. Return a Future for + * monitoring the job success or failure event. */ private[spark] def submitAsyncJob[T, U, R]( rdd: RDD[T], processPartition: (TaskContext, Iterator[T]) => U, resultHandler: (Int, U) => Unit, - resultFunc: => R): SimpleFutureAction[R] = + resultFunc: => R): Future[Unit] = { assertNotStopped() val cleanF = clean(processPartition) @@ -1879,7 +1880,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli allowLocal = false, resultHandler, localProperties.get) - new SimpleFutureAction(waiter, resultFunc) + waiter.toFuture } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala index 382b09422a4a..4b15488260eb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import scala.concurrent.{Future, Promise} + /** * An object that waits for a DAGScheduler job to complete. As tasks finish, it passes their * results to the given handler function. @@ -28,6 +30,8 @@ private[spark] class JobWaiter[T]( resultHandler: (Int, T) => Unit) extends JobListener { + private val promise = Promise[Unit] + private var finishedTasks = 0 // Is the job as a whole finished (succeeded or failed)? @@ -58,6 +62,7 @@ private[spark] class JobWaiter[T]( if (finishedTasks == totalTasks) { _jobFinished = true jobResult = JobSucceeded + promise.success() this.notifyAll() } } @@ -65,6 +70,7 @@ private[spark] class JobWaiter[T]( override def jobFailed(exception: Exception): Unit = synchronized { _jobFinished = true jobResult = JobFailed(exception) + promise.failure(exception) this.notifyAll() } @@ -74,4 +80,10 @@ private[spark] class JobWaiter[T]( } return jobResult } + + /** + * Return a Future to monitoring the job success or failure event. You can use this method to + * avoid blocking your thread. + */ + def toFuture: Future[Unit] = promise.future } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index 835b4fadb0a6..9c418ef46a07 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -163,7 +163,8 @@ private[streaming] abstract class ReceiverSupervisor( stopReceiver("Restarting receiver with delay " + delay + "ms: " + message, error) logDebug("Sleeping for " + delay) Thread.sleep(delay) - if (rescheduleReceiver().contains(host)) { + val scheduledLocations = rescheduleReceiver() + if (scheduledLocations.isEmpty || scheduledLocations.contains(host)) { logInfo("Starting receiver again") startReceiver() logInfo("Receiver started again") @@ -174,7 +175,7 @@ private[streaming] abstract class ReceiverSupervisor( } /** Reschedule this receiver and return a candidate executor list */ - def rescheduleReceiver(): Seq[String] + def rescheduleReceiver(): Seq[String] = Seq.empty /** Check if receiver has been marked for stopping */ def isReceiverStarted(): Boolean = { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala index 65f571a6218e..e60524558cc6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala @@ -20,6 +20,7 @@ package org.apache.spark.streaming.scheduler import scala.collection.mutable import scala.util.Random +import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.scheduler.ReceiverState._ private[streaming] case class ReceiverTrackingInfo( @@ -31,6 +32,10 @@ private[streaming] case class ReceiverTrackingInfo( private[streaming] trait ReceiverScheduler { + /** + * Return a candidate executor list to run the receiver. If the list is empty, the caller can run + * this receiver in arbitrary executor. + */ def scheduleReceiver( receiverId: Int, preferredLocation: Option[String], diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index d708167ee2d4..90a7bab1e20d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -333,7 +333,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } private val submitJobThread = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("streaming-submit-job")) + ThreadUtils.newDaemonSingleThreadExecutor("streaming-submit-job")) /** * Get the receivers from the ReceiverInputDStreams, distributes them to the @@ -390,8 +390,9 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } val self = this + val receiverId = receiver.streamId val scheduledLocations = scheduler.scheduleReceiver( - receiver.streamId, + receiverId, receiver.preferredLocation, getReceiverTrackingInfoMap(), getExecutors(ssc)) @@ -411,7 +412,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false if (stopping) { receiverExitLatch.countDown() } else { - logInfo(s"Restarting Receiver ${receiver.streamId}") + logInfo(s"Restarting Receiver $receiverId") submitJobThread.execute(self) } case Failure(e) => @@ -419,10 +420,10 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false receiverExitLatch.countDown() } else { logError("Receiver has been stopped. Try to restart it.", e) - logInfo(s"Restarting Receiver ${receiver.streamId}") + logInfo(s"Restarting Receiver $receiverId") submitJobThread.execute(self) } - }(submitJobThread) + }(ThreadUtils.sameThread) logInfo(s"Receiver ${receiver.streamId} started") } }) From 27acd45d345ef8e364682f03dbd8c3e9eef60feb Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 8 Jul 2015 21:58:17 +0800 Subject: [PATCH 03/27] Add unit tests for LoadBalanceReceiverSchedulerImplSuite --- .../scheduler/ReceiverScheduler.scala | 2 - .../streaming/scheduler/ReceiverTracker.scala | 3 - ...oadBalanceReceiverSchedulerImplSuite.scala | 79 +++++++++++++++++++ 3 files changed, 79 insertions(+), 5 deletions(-) create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/scheduler/LoadBalanceReceiverSchedulerImplSuite.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala index e60524558cc6..e7d5f6b7133f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala @@ -20,13 +20,11 @@ package org.apache.spark.streaming.scheduler import scala.collection.mutable import scala.util.Random -import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.scheduler.ReceiverState._ private[streaming] case class ReceiverTrackingInfo( receiverId: Int, state: ReceiverState, - preferredLocation: Option[String], scheduledLocations: Option[Seq[String]], runningLocation: Option[String]) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 90a7bab1e20d..387dad535b03 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -450,7 +450,6 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false receiverTrackingInfos.put(receiver.streamId, ReceiverTrackingInfo( receiver.streamId, ReceiverState.INACTIVE, - receiver.preferredLocation, None, None)) } @@ -468,7 +467,6 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false receiverTrackingInfos.put(receiverId, ReceiverTrackingInfo( receiverId, ReceiverState.SCHEDULED, - receiverPreferredLocations(receiverId), Some(scheduledLocations), None)) } @@ -478,7 +476,6 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false receiverTrackingInfos.put(receiverId, ReceiverTrackingInfo( receiverId, ReceiverState.ACTIVE, - receiverPreferredLocations(receiverId), None, Some(runningLocation))) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/LoadBalanceReceiverSchedulerImplSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/LoadBalanceReceiverSchedulerImplSuite.scala new file mode 100644 index 000000000000..ccd26fdc6ca2 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/LoadBalanceReceiverSchedulerImplSuite.scala @@ -0,0 +1,79 @@ +/* + * 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.streaming.scheduler + +import org.apache.spark.SparkFunSuite + +class LoadBalanceReceiverSchedulerImplSuite extends SparkFunSuite { + + val receiverScheduler = new LoadBalanceReceiverSchedulerImpl + + test("empty executors") { + val scheduledLocations = + receiverScheduler.scheduleReceiver(0, None, Map.empty, executors = Seq.empty) + assert(scheduledLocations === Seq.empty) + } + + test("receiver preferredLocation") { + val receiverTrackingInfoMap = Map( + 0 -> ReceiverTrackingInfo(0, ReceiverState.INACTIVE, None, None)) + val scheduledLocations = receiverScheduler.scheduleReceiver( + 0, Some("host1"), receiverTrackingInfoMap, executors = Seq("host2")) + assert(scheduledLocations.toSet === Set("host1", "host2")) + } + + test("choose the idle executor") { + val executors = Seq("host1", "host2", "host3") + // host3 is idle + val receiverTrackingInfoMap = Map( + 0 -> ReceiverTrackingInfo(0, ReceiverState.ACTIVE, None, Some("host1")), + 1 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host2")), None)) + val scheduledLocations = receiverScheduler.scheduleReceiver( + 2, None, receiverTrackingInfoMap, executors) + assert(scheduledLocations.toSet === Set("host3")) + } + + test("all executors are busy") { + val executors = Seq("host1", "host2", "host3") + // Weights: host1 = 1.5, host2 = 0.5, host3 = 1.0 + val receiverTrackingInfoMap = Map( + 0 -> ReceiverTrackingInfo(0, ReceiverState.ACTIVE, None, Some("host1")), + 1 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host2", "host3")), None), + 2 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host1", "host3")), None)) + val scheduledLocations = receiverScheduler.scheduleReceiver( + 3, None, receiverTrackingInfoMap, executors) + assert(scheduledLocations.toSet === Set("host2")) + } + + test("ignore the receiver's info") { + val executors = Seq("host1", "host2", "host3") + // Weights: host1 = 1.0, host2 = 1.5, host3 = 1.5 + // But since we are scheduling the receiver 1, we should ignore + // receiver 1's ReceiverTrackingInfo + // So the new weights are host1 = 1.0, host2 = 0.5, host3 = 1.5 + // Then the scheduled location should be "host2" + val receiverTrackingInfoMap = Map( + 0 -> ReceiverTrackingInfo(0, ReceiverState.ACTIVE, None, Some("host1")), + 1 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host2")), None), + 2 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host3")), None), + 3 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host2", "host3")), None)) + val scheduledLocations = receiverScheduler.scheduleReceiver( + 1, None, receiverTrackingInfoMap, executors) + assert(scheduledLocations.toSet === Set("host2")) + } +} From ca6fe359fcbd752eaec8b1e713f1c6cffb90b751 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 8 Jul 2015 22:41:46 +0800 Subject: [PATCH 04/27] Add a test for Receiver.restart --- .../org/apache/spark/streaming/ReceiverSuite.scala | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index 5d7127627eea..38652e7880c6 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -129,6 +129,20 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { } } + test("restart receiver should consider the scheduled locations") { + val receiver = new FakeReceiver + val executor = new FakeReceiverSupervisor(receiver) { + override def rescheduleReceiver: Seq[String] = Seq("unknown-host") + } + executor.start() + receiver.restart("force the receiver restart") + eventually(timeout(30000 millis), interval(10 millis)) { + // Since the scheduled location is not the current host, the receiver should exit + assert(receiver.onStopCalled) + assert(receiver.isStopped) + } + } + test("block generator") { val blockGeneratorListener = new FakeBlockGeneratorListener val blockIntervalMs = 200 From 2c86a9e0ff4dd11ea0680cda45bb4f64c4e01aff Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 9 Jul 2015 17:07:47 +0800 Subject: [PATCH 05/27] Use tryFailure to support calling jobFailed multiple times --- .../src/main/scala/org/apache/spark/scheduler/JobWaiter.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala index 4b15488260eb..64b160c4814b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala @@ -62,7 +62,7 @@ private[spark] class JobWaiter[T]( if (finishedTasks == totalTasks) { _jobFinished = true jobResult = JobSucceeded - promise.success() + promise.trySuccess() this.notifyAll() } } @@ -70,7 +70,7 @@ private[spark] class JobWaiter[T]( override def jobFailed(exception: Exception): Unit = synchronized { _jobFinished = true jobResult = JobFailed(exception) - promise.failure(exception) + promise.tryFailure(exception) this.notifyAll() } From 28d1bee9c1afbc23be650d21ef53c946c7c49298 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sat, 11 Jul 2015 23:13:40 +0800 Subject: [PATCH 06/27] Make 'host' protected; rescheduleReceiver -> getAllowedLocations --- .../spark/streaming/receiver/ReceiverSupervisor.scala | 6 +++--- .../spark/streaming/receiver/ReceiverSupervisorImpl.scala | 2 +- .../scala/org/apache/spark/streaming/ReceiverSuite.scala | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index 9c418ef46a07..b8d2fa300947 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -64,7 +64,7 @@ private[streaming] abstract class ReceiverSupervisor( /** State of the receiver */ @volatile private[streaming] var receiverState = Initialized - val host = Utils.localHostName() + protected val host = Utils.localHostName() /** Push a single data item to backend data store. */ def pushSingle(data: Any) @@ -163,7 +163,7 @@ private[streaming] abstract class ReceiverSupervisor( stopReceiver("Restarting receiver with delay " + delay + "ms: " + message, error) logDebug("Sleeping for " + delay) Thread.sleep(delay) - val scheduledLocations = rescheduleReceiver() + val scheduledLocations = getAllowedLocations() if (scheduledLocations.isEmpty || scheduledLocations.contains(host)) { logInfo("Starting receiver again") startReceiver() @@ -175,7 +175,7 @@ private[streaming] abstract class ReceiverSupervisor( } /** Reschedule this receiver and return a candidate executor list */ - def rescheduleReceiver(): Seq[String] = Seq.empty + def getAllowedLocations(): Seq[String] = Seq.empty /** Check if receiver has been marked for stopping */ def isReceiverStarted(): Boolean = { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index c44b408b03c1..2f7314e06a19 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -183,7 +183,7 @@ private[streaming] class ReceiverSupervisorImpl( receivedBlockHandler.cleanupOldBlocks(cleanupThreshTime.milliseconds) } - override def rescheduleReceiver(): Seq[String] = { + override def getAllowedLocations(): Seq[String] = { trackerEndpoint.askWithRetry[Seq[String]](ScheduleReceiver(streamId)) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index 38652e7880c6..8219f64d33ae 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -132,7 +132,7 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { test("restart receiver should consider the scheduled locations") { val receiver = new FakeReceiver val executor = new FakeReceiverSupervisor(receiver) { - override def rescheduleReceiver: Seq[String] = Seq("unknown-host") + override def getAllowedLocations: Seq[String] = Seq("unknown-host") } executor.start() receiver.restart("force the receiver restart") From 9ecc08e412921b38ffd7e20032a0cafd467d2424 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 14 Jul 2015 10:32:53 +0800 Subject: [PATCH 07/27] Fix comments and code style --- .../main/scala/org/apache/spark/SparkContext.scala | 7 +------ .../streaming/receiver/ReceiverSupervisor.scala | 2 +- .../streaming/scheduler/ReceiverScheduler.scala | 6 +++--- .../spark/streaming/scheduler/ReceiverTracker.scala | 12 +++++++----- 4 files changed, 12 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e236b3d078b8..3e9403974c7c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1859,16 +1859,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli new SimpleFutureAction(waiter, resultFunc) } - /** - * Submit a job for execution and return a FutureJob holding the result. Return a Future for - * monitoring the job success or failure event. - */ private[spark] def submitAsyncJob[T, U, R]( rdd: RDD[T], processPartition: (TaskContext, Iterator[T]) => U, resultHandler: (Int, U) => Unit, - resultFunc: => R): Future[Unit] = - { + resultFunc: => R): Future[Unit] = { assertNotStopped() val cleanF = clean(processPartition) val callSite = getCallSite diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index b8d2fa300947..5b1d3608c945 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -174,7 +174,7 @@ private[streaming] abstract class ReceiverSupervisor( }(futureExecutionContext) } - /** Reschedule this receiver and return a candidate executor list */ + /** Return a list of candidate executors to run the receiver */ def getAllowedLocations(): Seq[String] = Seq.empty /** Check if receiver has been marked for stopping */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala index e7d5f6b7133f..900fb0107195 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala @@ -31,8 +31,8 @@ private[streaming] case class ReceiverTrackingInfo( private[streaming] trait ReceiverScheduler { /** - * Return a candidate executor list to run the receiver. If the list is empty, the caller can run - * this receiver in arbitrary executor. + * Return a list of candidate executors to run the receiver. If the list is empty, the caller can + * run this receiver in arbitrary executor. */ def scheduleReceiver( receiverId: Int, @@ -69,7 +69,7 @@ private[streaming] class LoadBalanceReceiverSchedulerImpl extends ReceiverSchedu val scheduledLocations = receiverTrackingInfo.scheduledLocations.get // The probability that a scheduled receiver will run in an executor is // 1.0 / scheduledLocations.size - scheduledLocations.map(location => (location, 1.0 / scheduledLocations.size)) + scheduledLocations.map(location => location -> 1.0 / scheduledLocations.size) case ReceiverState.ACTIVE => Seq(receiverTrackingInfo.runningLocation.get -> 1.0) } }.groupBy(_._1).mapValues(_.map(_._2).sum) // Sum weights for each executor diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 387dad535b03..6cb13d8fdc88 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -93,13 +93,17 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false @volatile private var stopping = false - // Track receivers' status for scheduling + /** + * Track receivers' status for scheduling + */ private val receiverTrackingInfos = new HashMap[Int, ReceiverTrackingInfo] - // Store all preferred locations for all receivers. We need this information to schedule receivers + /** + * Store all preferred locations for all receivers. We need this information to schedule receivers + */ private val receiverPreferredLocations = new HashMap[Int, Option[String]] - // Use a separate lock to avoid dead-lock + /** Use a separate lock to avoid dead-lock */ private val receiverTrackingInfosLock = new AnyRef /** Start the endpoint and receiver execution thread. */ @@ -405,8 +409,6 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } val future = ssc.sparkContext.submitAsyncJob[Receiver[_], Unit, Unit]( receiverRDD, startReceiver, (_, _) => Unit, ()) - // TODO Refactor JobWaiter to avoid creating a new Thread here. Otherwise, it's not - // scalable because we need a thread for each Receiver. future.onComplete { case Success(_) => if (stopping) { From f549595a77639c1f5f6e98feb3fb1aeb4be59890 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 14 Jul 2015 11:36:57 +0800 Subject: [PATCH 08/27] Add comments for the scheduling approach --- .../scheduler/ReceiverScheduler.scala | 23 ++++++++++++++++++- 1 file changed, 22 insertions(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala index 900fb0107195..2a2da7fa3943 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala @@ -42,7 +42,28 @@ private[streaming] trait ReceiverScheduler { } /** - * A ReceiverScheduler trying to balance executors' load. + * A ReceiverScheduler trying to balance executors' load. Here is the approach to schedule executors + * for a receiver. + *
    + *
  1. + * If preferredLocation is set, preferredLocation should be one of the candidate executors. + *
  2. + *
  3. + * Every executor will be assigned to a weight according to the receivers running or scheduling + * on it. + *
      + *
    • + * If a receiver is running on an executor, it contributes 1.0 to the executor's weight. + *
    • + *
    • + * If a receiver is scheduled to an executor but has not yet run, it contributes + * `1.0 / #candidate_executors_of_this_receiver` to the executor's weight.
    • + *
    + * At last, we will randomly select one of the executors that have the least weight and add it + * to the candidate list. + *
  4. + *
+ * */ private[streaming] class LoadBalanceReceiverSchedulerImpl extends ReceiverScheduler { From a86850c5071666a4aa6ccb41ccc286a8ead47169 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 14 Jul 2015 15:29:09 +0800 Subject: [PATCH 09/27] Remove submitAsyncJob and revert JobWaiter --- .../scala/org/apache/spark/SparkContext.scala | 19 ------------------- .../apache/spark/scheduler/JobWaiter.scala | 12 ------------ .../streaming/scheduler/ReceiverTracker.scala | 6 +++--- 3 files changed, 3 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3e9403974c7c..d09802ade52d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1859,25 +1859,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli new SimpleFutureAction(waiter, resultFunc) } - private[spark] def submitAsyncJob[T, U, R]( - rdd: RDD[T], - processPartition: (TaskContext, Iterator[T]) => U, - resultHandler: (Int, U) => Unit, - resultFunc: => R): Future[Unit] = { - assertNotStopped() - val cleanF = clean(processPartition) - val callSite = getCallSite - val waiter = dagScheduler.submitJob( - rdd, - (context: TaskContext, iter: Iterator[T]) => cleanF(context, iter), - 0 until rdd.partitions.size, - callSite, - allowLocal = false, - resultHandler, - localProperties.get) - waiter.toFuture - } - /** * Cancel active jobs for the specified group. See [[org.apache.spark.SparkContext.setJobGroup]] * for more information. diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala index 64b160c4814b..382b09422a4a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala @@ -17,8 +17,6 @@ package org.apache.spark.scheduler -import scala.concurrent.{Future, Promise} - /** * An object that waits for a DAGScheduler job to complete. As tasks finish, it passes their * results to the given handler function. @@ -30,8 +28,6 @@ private[spark] class JobWaiter[T]( resultHandler: (Int, T) => Unit) extends JobListener { - private val promise = Promise[Unit] - private var finishedTasks = 0 // Is the job as a whole finished (succeeded or failed)? @@ -62,7 +58,6 @@ private[spark] class JobWaiter[T]( if (finishedTasks == totalTasks) { _jobFinished = true jobResult = JobSucceeded - promise.trySuccess() this.notifyAll() } } @@ -70,7 +65,6 @@ private[spark] class JobWaiter[T]( override def jobFailed(exception: Exception): Unit = synchronized { _jobFinished = true jobResult = JobFailed(exception) - promise.tryFailure(exception) this.notifyAll() } @@ -80,10 +74,4 @@ private[spark] class JobWaiter[T]( } return jobResult } - - /** - * Return a Future to monitoring the job success or failure event. You can use this method to - * avoid blocking your thread. - */ - def toFuture: Future[Unit] = promise.future } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 6cb13d8fdc88..e631b1f41333 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -360,12 +360,12 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false new SerializableConfiguration(ssc.sparkContext.hadoopConfiguration) // Function to start the receiver on the worker node - val startReceiver = (context: TaskContext, iterator: Iterator[Receiver[_]]) => { + val startReceiver = (iterator: Iterator[Receiver[_]]) => { if (!iterator.hasNext) { throw new SparkException( "Could not start receiver as object not found.") } - if (context.attemptNumber() == 0) { + if (TaskContext.get().attemptNumber() == 0) { val receiver = iterator.next() val supervisor = new ReceiverSupervisorImpl( receiver, SparkEnv.get, serializableHadoopConf.value, checkpointDirOption) @@ -407,7 +407,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } else { ssc.sc.makeRDD(Seq(receiver -> scheduledLocations)) } - val future = ssc.sparkContext.submitAsyncJob[Receiver[_], Unit, Unit]( + val future = ssc.sparkContext.submitJob[Receiver[_], Unit, Unit]( receiverRDD, startReceiver, (_, _) => Unit, ()) future.onComplete { case Success(_) => From 3b87e4ad87dbcfd498adf93883d48caab601c0ed Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 15 Jul 2015 01:23:48 +0800 Subject: [PATCH 10/27] Revert SparkContext.scala --- core/src/main/scala/org/apache/spark/SparkContext.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d09802ade52d..d2547eeff2b4 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -17,6 +17,8 @@ package org.apache.spark +import scala.language.implicitConversions + import java.io._ import java.lang.reflect.Constructor import java.net.URI @@ -28,8 +30,6 @@ import scala.collection.{Map, Set} import scala.collection.JavaConversions._ import scala.collection.generic.Growable import scala.collection.mutable.HashMap -import scala.concurrent.Future -import scala.language.implicitConversions import scala.reflect.{ClassTag, classTag} import scala.util.control.NonFatal From e530bcc271df0b649a55182a4acd67072d35ad8d Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 25 May 2015 16:59:20 +0800 Subject: [PATCH 11/27] [SPARK-5681][Streaming] Use a lock to eliminate the race condition when stopping receivers and registering receivers happen at the same time #6294 --- .../receiver/ReceiverSupervisor.scala | 42 +++++++---- .../receiver/ReceiverSupervisorImpl.scala | 2 +- .../streaming/scheduler/ReceiverTracker.scala | 75 ++++++++++++++++--- .../streaming/StreamingContextSuite.scala | 13 ++++ 4 files changed, 104 insertions(+), 28 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index 33be067ebdaf..ea1f6ca79974 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -22,6 +22,7 @@ import java.util.concurrent.CountDownLatch import scala.collection.mutable.ArrayBuffer import scala.concurrent._ +import scala.util.control.NonFatal import org.apache.spark.{Logging, SparkConf} import org.apache.spark.storage.StreamBlockId @@ -36,7 +37,7 @@ private[streaming] abstract class ReceiverSupervisor( conf: SparkConf ) extends Logging { - /** Enumeration to identify current state of the StreamingContext */ + /** Enumeration to identify current state of the Receiver */ object ReceiverState extends Enumeration { type CheckpointState = Value val Initialized, Started, Stopped = Value @@ -97,8 +98,8 @@ private[streaming] abstract class ReceiverSupervisor( /** Called when supervisor is stopped */ protected def onStop(message: String, error: Option[Throwable]) { } - /** Called when receiver is started */ - protected def onReceiverStart() { } + /** Called when receiver is started. Return if the driver accepts us */ + protected def onReceiverStart(): Boolean = true /** Called when receiver is stopped */ protected def onReceiverStop(message: String, error: Option[Throwable]) { } @@ -121,13 +122,17 @@ private[streaming] abstract class ReceiverSupervisor( /** Start receiver */ def startReceiver(): Unit = synchronized { try { - logInfo("Starting receiver") - receiver.onStart() - logInfo("Called receiver onStart") - onReceiverStart() - receiverState = Started + if (onReceiverStart()) { + logInfo("Starting receiver") + receiverState = Started + receiver.onStart() + logInfo("Called receiver onStart") + } else { + // The driver refused us + stop("Registered unsuccessfully because the driver refused" + streamId, None) + } } catch { - case t: Throwable => + case NonFatal(t) => stop("Error starting receiver " + streamId, Some(t)) } } @@ -136,12 +141,19 @@ private[streaming] abstract class ReceiverSupervisor( def stopReceiver(message: String, error: Option[Throwable]): Unit = synchronized { try { logInfo("Stopping receiver with message: " + message + ": " + error.getOrElse("")) - receiverState = Stopped - receiver.onStop() - logInfo("Called receiver onStop") - onReceiverStop(message, error) + receiverState match { + case Initialized => + logWarning("Skip stopping receiver because it has not yet stared") + case Started => + receiverState = Stopped + receiver.onStop() + logInfo("Called receiver onStop") + onReceiverStop(message, error) + case Stopped => + logWarning("Receiver has been stopped") + } } catch { - case t: Throwable => + case NonFatal(t) => logError("Error stopping receiver " + streamId + t.getStackTraceString) } } @@ -167,7 +179,7 @@ private[streaming] abstract class ReceiverSupervisor( }(futureExecutionContext) } - /** Check if receiver has been marked for stopping */ + /** Check if receiver has been marked for starting */ def isReceiverStarted(): Boolean = { logDebug("state = " + receiverState) receiverState == Started diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 92938379b9c1..7b8f05ffeac0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -167,7 +167,7 @@ private[streaming] class ReceiverSupervisorImpl( env.rpcEnv.stop(endpoint) } - override protected def onReceiverStart() { + override protected def onReceiverStart(): Boolean = { val msg = RegisterReceiver( streamId, receiver.getClass.getSimpleName, Utils.localHostName(), endpoint) trackerEndpoint.askWithRetry[Boolean](msg) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index f73f7e705ee0..d459c2d3e971 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -67,13 +67,41 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false ) private val listenerBus = ssc.scheduler.listenerBus + /** Enumeration to identify current state of the ReceiverTracker */ + object TrackerState extends Enumeration { + type CheckpointState = Value + val Initialized, Started, Stopping, Stopped = Value + } + import TrackerState._ + + /** State of the tracker. Protected by "trackerStateLock" */ + private var trackerState = Initialized + + /** "trackerStateLock" is used to protect reading/writing "trackerState" */ + private val trackerStateLock = new AnyRef + // endpoint is created when generator starts. // This not being null means the tracker has been started and not stopped private var endpoint: RpcEndpointRef = null + /** Check if tracker has been marked for starting */ + private def isTrackerStarted(): Boolean = trackerStateLock.synchronized { + trackerState == Started + } + + /** Check if tracker has been marked for stopping */ + private def isTrackerStopping(): Boolean = trackerStateLock.synchronized { + trackerState == Stopping + } + + /** Check if tracker has been marked for stopped */ + private def isTrackerStopped(): Boolean = trackerStateLock.synchronized { + trackerState == Stopped + } + /** Start the endpoint and receiver execution thread. */ def start(): Unit = synchronized { - if (endpoint != null) { + if (isTrackerStarted) { throw new SparkException("ReceiverTracker already started") } @@ -82,13 +110,19 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false "ReceiverTracker", new ReceiverTrackerEndpoint(ssc.env.rpcEnv)) if (!skipReceiverLaunch) receiverExecutor.start() logInfo("ReceiverTracker started") + trackerStateLock.synchronized { + trackerState = Started + } } } /** Stop the receiver execution thread. */ def stop(graceful: Boolean): Unit = synchronized { - if (!receiverInputStreams.isEmpty && endpoint != null) { + if (isTrackerStarted) { // First, stop the receivers + trackerStateLock.synchronized { + trackerState = Stopping + } if (!skipReceiverLaunch) receiverExecutor.stop(graceful) // Finally, stop the endpoint @@ -96,6 +130,9 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false endpoint = null receivedBlockTracker.stop() logInfo("ReceiverTracker stopped") + trackerStateLock.synchronized { + trackerState = Stopped + } } } @@ -141,14 +178,24 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false host: String, receiverEndpoint: RpcEndpointRef, senderAddress: RpcAddress - ) { + ): Boolean = { if (!receiverInputStreamIds.contains(streamId)) { throw new SparkException("Register received for unexpected id " + streamId) } - receiverInfo(streamId) = ReceiverInfo( - streamId, s"${typ}-${streamId}", receiverEndpoint, true, host) - listenerBus.post(StreamingListenerReceiverStarted(receiverInfo(streamId))) - logInfo("Registered receiver for stream " + streamId + " from " + senderAddress) + + trackerStateLock.synchronized { + if (isTrackerStopping || isTrackerStopped) { + false + } else { + // When updating "receiverInfo", we should make sure "trackerState" won't be changed at the + // same time. Therefore the following line should be in "trackerStateLock.synchronized". + receiverInfo(streamId) = ReceiverInfo( + streamId, s"${typ}-${streamId}", receiverEndpoint, true, host) + listenerBus.post(StreamingListenerReceiverStarted(receiverInfo(streamId))) + logInfo("Registered receiver for stream " + streamId + " from " + senderAddress) + true + } + } } /** Deregister a receiver */ @@ -216,8 +263,9 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case RegisterReceiver(streamId, typ, host, receiverEndpoint) => - registerReceiver(streamId, typ, host, receiverEndpoint, context.sender.address) - context.reply(true) + val successful = + registerReceiver(streamId, typ, host, receiverEndpoint, context.sender.address) + context.reply(successful) case AddBlock(receivedBlockInfo) => context.reply(addBlock(receivedBlockInfo)) case DeregisterReceiver(streamId, message, error) => @@ -317,9 +365,12 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false // Distribute the receivers and start them logInfo("Starting " + receivers.length + " receivers") running = true - ssc.sparkContext.runJob(tempRDD, ssc.sparkContext.clean(startReceiver)) - running = false - logInfo("All of the receivers have been terminated") + try { + ssc.sparkContext.runJob(tempRDD, ssc.sparkContext.clean(startReceiver)) + logInfo("All of the receivers have been terminated") + } finally { + running = false + } } /** Stops the receivers. */ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index f8e8030791df..a2ca282795fc 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -255,6 +255,19 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w } } + test("stop gracefully even if a receiver misses StopReceiver") { + val conf = new SparkConf().setMaster(master).setAppName(appName) + sc = new SparkContext(conf) + ssc = new StreamingContext(sc, Milliseconds(100)) + val input = ssc.receiverStream(new TestReceiver) + input.foreachRDD(_ => {}) + ssc.start() + // Call `ssc.stop` at once so that it's possible that the receiver will miss "StopReceiver" + failAfter(30000 millis) { + ssc.stop(stopSparkContext = true, stopGracefully = true) + } + } + test("stop slow receiver gracefully") { val conf = new SparkConf().setMaster(master).setAppName(appName) conf.set("spark.streaming.gracefulStopTimeout", "20000s") From 881edb99b55f64c5fd2e6861a7a2cf384fd96190 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 16 Jul 2015 00:35:09 +0800 Subject: [PATCH 12/27] ReceiverScheduler -> ReceiverSchedulingPolicy --- ...eduler.scala => ReceiverSchedulingPolicy.scala} | 4 ++-- .../streaming/scheduler/ReceiverTracker.scala | 7 ++++--- ...BalanceReceiverSchedulingPolicyImplSuite.scala} | 14 +++++++------- 3 files changed, 13 insertions(+), 12 deletions(-) rename streaming/src/main/scala/org/apache/spark/streaming/scheduler/{ReceiverScheduler.scala => ReceiverSchedulingPolicy.scala} (96%) rename streaming/src/test/scala/org/apache/spark/streaming/scheduler/{LoadBalanceReceiverSchedulerImplSuite.scala => LoadBalanceReceiverSchedulingPolicyImplSuite.scala} (85%) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala similarity index 96% rename from streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala rename to streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala index 2a2da7fa3943..f878ce919685 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala @@ -28,7 +28,7 @@ private[streaming] case class ReceiverTrackingInfo( scheduledLocations: Option[Seq[String]], runningLocation: Option[String]) -private[streaming] trait ReceiverScheduler { +private[streaming] trait ReceiverSchedulingPolicy { /** * Return a list of candidate executors to run the receiver. If the list is empty, the caller can @@ -65,7 +65,7 @@ private[streaming] trait ReceiverScheduler { * * */ -private[streaming] class LoadBalanceReceiverSchedulerImpl extends ReceiverScheduler { +private[streaming] class LoadBalanceReceiverSchedulingPolicyImpl extends ReceiverSchedulingPolicy { def scheduleReceiver( receiverId: Int, diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index e631b1f41333..f0a6f2427508 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -89,7 +89,8 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false // This not being null means the tracker has been started and not stopped private var endpoint: RpcEndpointRef = null - private val scheduler: ReceiverScheduler = new LoadBalanceReceiverSchedulerImpl() + private val schedulingPolicy: ReceiverSchedulingPolicy = + new LoadBalanceReceiverSchedulingPolicyImpl() @volatile private var stopping = false @@ -395,7 +396,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false val self = this val receiverId = receiver.streamId - val scheduledLocations = scheduler.scheduleReceiver( + val scheduledLocations = schedulingPolicy.scheduleReceiver( receiverId, receiver.preferredLocation, getReceiverTrackingInfoMap(), @@ -486,7 +487,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false val preferredLocation = receiverTrackingInfosLock.synchronized { receiverPreferredLocations(receiverId) } - val scheduledLocations = scheduler.scheduleReceiver( + val scheduledLocations = schedulingPolicy.scheduleReceiver( receiverId, preferredLocation, getReceiverTrackingInfoMap(), getExecutors(ssc)) updateReceiverScheduledLocations(receiverId, scheduledLocations) scheduledLocations diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/LoadBalanceReceiverSchedulerImplSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/LoadBalanceReceiverSchedulingPolicyImplSuite.scala similarity index 85% rename from streaming/src/test/scala/org/apache/spark/streaming/scheduler/LoadBalanceReceiverSchedulerImplSuite.scala rename to streaming/src/test/scala/org/apache/spark/streaming/scheduler/LoadBalanceReceiverSchedulingPolicyImplSuite.scala index ccd26fdc6ca2..e6550f0fbe5e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/LoadBalanceReceiverSchedulerImplSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/LoadBalanceReceiverSchedulingPolicyImplSuite.scala @@ -19,20 +19,20 @@ package org.apache.spark.streaming.scheduler import org.apache.spark.SparkFunSuite -class LoadBalanceReceiverSchedulerImplSuite extends SparkFunSuite { +class LoadBalanceReceiverSchedulingPolicyImplSuite extends SparkFunSuite { - val receiverScheduler = new LoadBalanceReceiverSchedulerImpl + val receiverSchedulingPolicy = new LoadBalanceReceiverSchedulingPolicyImpl test("empty executors") { val scheduledLocations = - receiverScheduler.scheduleReceiver(0, None, Map.empty, executors = Seq.empty) + receiverSchedulingPolicy.scheduleReceiver(0, None, Map.empty, executors = Seq.empty) assert(scheduledLocations === Seq.empty) } test("receiver preferredLocation") { val receiverTrackingInfoMap = Map( 0 -> ReceiverTrackingInfo(0, ReceiverState.INACTIVE, None, None)) - val scheduledLocations = receiverScheduler.scheduleReceiver( + val scheduledLocations = receiverSchedulingPolicy.scheduleReceiver( 0, Some("host1"), receiverTrackingInfoMap, executors = Seq("host2")) assert(scheduledLocations.toSet === Set("host1", "host2")) } @@ -43,7 +43,7 @@ class LoadBalanceReceiverSchedulerImplSuite extends SparkFunSuite { val receiverTrackingInfoMap = Map( 0 -> ReceiverTrackingInfo(0, ReceiverState.ACTIVE, None, Some("host1")), 1 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host2")), None)) - val scheduledLocations = receiverScheduler.scheduleReceiver( + val scheduledLocations = receiverSchedulingPolicy.scheduleReceiver( 2, None, receiverTrackingInfoMap, executors) assert(scheduledLocations.toSet === Set("host3")) } @@ -55,7 +55,7 @@ class LoadBalanceReceiverSchedulerImplSuite extends SparkFunSuite { 0 -> ReceiverTrackingInfo(0, ReceiverState.ACTIVE, None, Some("host1")), 1 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host2", "host3")), None), 2 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host1", "host3")), None)) - val scheduledLocations = receiverScheduler.scheduleReceiver( + val scheduledLocations = receiverSchedulingPolicy.scheduleReceiver( 3, None, receiverTrackingInfoMap, executors) assert(scheduledLocations.toSet === Set("host2")) } @@ -72,7 +72,7 @@ class LoadBalanceReceiverSchedulerImplSuite extends SparkFunSuite { 1 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host2")), None), 2 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host3")), None), 3 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host2", "host3")), None)) - val scheduledLocations = receiverScheduler.scheduleReceiver( + val scheduledLocations = receiverSchedulingPolicy.scheduleReceiver( 1, None, receiverTrackingInfoMap, executors) assert(scheduledLocations.toSet === Set("host2")) } From 9e242c88ed483fa506182b4c72c0cd53a00df0c7 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 16 Jul 2015 01:07:20 +0800 Subject: [PATCH 13/27] Remove the ScheduleReceiver message because we can refuse it when receiving RegisterReceiver --- .../streaming/receiver/ReceiverSupervisor.scala | 14 +++----------- .../receiver/ReceiverSupervisorImpl.scala | 3 --- .../streaming/scheduler/ReceiverTracker.scala | 12 +++++------- .../org/apache/spark/streaming/ReceiverSuite.scala | 4 +--- 4 files changed, 9 insertions(+), 24 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index 35e33a65522d..066431a0d43d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -175,20 +175,12 @@ private[streaming] abstract class ReceiverSupervisor( stopReceiver("Restarting receiver with delay " + delay + "ms: " + message, error) logDebug("Sleeping for " + delay) Thread.sleep(delay) - val scheduledLocations = getAllowedLocations() - if (scheduledLocations.isEmpty || scheduledLocations.contains(host)) { - logInfo("Starting receiver again") - startReceiver() - logInfo("Receiver started again") - } else { - stop("Receiver is scheduled to another executor", None) - } + logInfo("Starting receiver again") + startReceiver() + logInfo("Receiver started again") }(futureExecutionContext) } - /** Return a list of candidate executors to run the receiver */ - def getAllowedLocations(): Seq[String] = Seq.empty - /** Check if receiver has been marked for starting */ def isReceiverStarted(): Boolean = { logDebug("state = " + receiverState) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index f669b21ba3ad..697637525651 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -183,7 +183,4 @@ private[streaming] class ReceiverSupervisorImpl( receivedBlockHandler.cleanupOldBlocks(cleanupThreshTime.milliseconds) } - override def getAllowedLocations(): Seq[String] = { - trackerEndpoint.askWithRetry[Seq[String]](ScheduleReceiver(streamId)) - } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 4f05d4256c12..78fcc36a24b4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -58,9 +58,6 @@ private[streaming] case class ReportError(streamId: Int, message: String, error: private[streaming] case class DeregisterReceiver(streamId: Int, msg: String, error: String) extends ReceiverTrackerMessage -/** It's used to ask ReceiverTracker to return a candidate executor list to run the receiver */ -private[streaming] case class ScheduleReceiver(streamId: Int) extends ReceiverTrackerMessage - /** * This class manages the execution of the receivers of ReceiverInputDStreams. Instance of * this class must be created after all input streams have been added and StreamingContext.start() @@ -122,12 +119,12 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false private def isTrackerStarted(): Boolean = trackerStateLock.synchronized { trackerState == Started } - + /** Check if tracker has been marked for stopping */ private def isTrackerStopping(): Boolean = trackerStateLock.synchronized { trackerState == Stopping } - + /** Check if tracker has been marked for stopped */ private def isTrackerStopped(): Boolean = trackerStateLock.synchronized { trackerState == Stopped @@ -219,6 +216,9 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false trackerStateLock.synchronized { if (isTrackerStopping || isTrackerStopped) { false + } else if (!ssc.sparkContext.isLocal && // We don't need to schedule it in the local mode + !scheduleReceiver(streamId).contains(host)) { + false } else { // When updating "receiverInfo", we should make sure "trackerState" won't be changed at the // same time. Therefore the following line should be in "trackerStateLock.synchronized". @@ -305,8 +305,6 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false case DeregisterReceiver(streamId, message, error) => deregisterReceiver(streamId, message, error) context.reply(true) - case ScheduleReceiver(streamId) => - context.reply(scheduleReceiver(streamId)) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index 8219f64d33ae..8d0b7a3b4a73 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -131,9 +131,7 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { test("restart receiver should consider the scheduled locations") { val receiver = new FakeReceiver - val executor = new FakeReceiverSupervisor(receiver) { - override def getAllowedLocations: Seq[String] = Seq("unknown-host") - } + val executor = new FakeReceiverSupervisor(receiver) executor.start() receiver.restart("force the receiver restart") eventually(timeout(30000 millis), interval(10 millis)) { From 5fee1324be4289e73d847b6a8c1a0ef0cb28555b Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 16 Jul 2015 01:18:59 +0800 Subject: [PATCH 14/27] Update tha scheduling algorithm to avoid to keep restarting Receiver --- .../scheduler/ReceiverSchedulingPolicy.scala | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala index f878ce919685..831a40dea9c7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala @@ -59,8 +59,8 @@ private[streaming] trait ReceiverSchedulingPolicy { * If a receiver is scheduled to an executor but has not yet run, it contributes * `1.0 / #candidate_executors_of_this_receiver` to the executor's weight. * - * At last, we will randomly select one of the executors that have the least weight and add it - * to the candidate list. + * At last, if there are more than 3 idle executors (weight = 0), returns all idle executors. + * Otherwise, we only return 3 best options according to the weights. * * * @@ -96,12 +96,14 @@ private[streaming] class LoadBalanceReceiverSchedulingPolicyImpl extends Receive }.groupBy(_._1).mapValues(_.map(_._2).sum) // Sum weights for each executor val idleExecutors = (executors.toSet -- executorWeights.keys).toSeq - if (idleExecutors.nonEmpty) { - // If there are idle executors, randomly select one - locations += idleExecutors(Random.nextInt(idleExecutors.size)) + if (idleExecutors.size >= 3) { + // If there are more than 3 idle executors, return all of them + locations ++= idleExecutors } else { - // Use the executor that runs the least receivers - locations += executorWeights.minBy(_._2)._1 + // If there are less than 3 idle executors, return 3 best options + locations ++= idleExecutors + val sortedExecutors = executorWeights.toSeq.sortBy(_._2).map(_._1) + locations ++= (idleExecutors ++ sortedExecutors).take(3) } locations.toSeq } From f60d0217151cdc72c05671437ef3433bd1c18bba Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 22 Jul 2015 00:29:26 +0800 Subject: [PATCH 15/27] Reorganize ReceiverTracker to use an event loop for lock free 1. Add fields into ReceiverTrackingInfo to contain all necessary information for ReceiverInfo. When sending ReceiverInfo to StreamingListener, create ReceiverInfo from ReceiverTrackingInfo directly. 2. Move methods that use ReceiverTrackingInfo into ReceiverTrackerEndpoint so that we can maintain all ReceiverTrackingInfos in an event loop. 3. Move ReceiverTrackingInfo to a separate file ReceiverTrackingInfo.scala. 4. Address some code style issues --- .../receiver/ReceiverSupervisorImpl.scala | 1 - .../streaming/scheduler/ReceiverInfo.scala | 1 - .../scheduler/ReceiverSchedulingPolicy.scala | 36 +- .../streaming/scheduler/ReceiverTracker.scala | 549 +++++++++--------- .../scheduler/ReceiverTrackingInfo.scala | 55 ++ ...la => ReceiverSchedulingPolicySuite.scala} | 39 +- .../scheduler/ReceiverTrackerSuite.scala | 90 --- .../StreamingJobProgressListenerSuite.scala | 6 +- 8 files changed, 362 insertions(+), 415 deletions(-) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTrackingInfo.scala rename streaming/src/test/scala/org/apache/spark/streaming/scheduler/{LoadBalanceReceiverSchedulingPolicyImplSuite.scala => ReceiverSchedulingPolicySuite.scala} (59%) delete mode 100644 streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 697637525651..3ee648ad9ad3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -182,5 +182,4 @@ private[streaming] class ReceiverSupervisorImpl( logDebug(s"Cleaning up blocks older then $cleanupThreshTime") receivedBlockHandler.cleanupOldBlocks(cleanupThreshTime.milliseconds) } - } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala index de85f24dd988..59df892397fe 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala @@ -28,7 +28,6 @@ import org.apache.spark.rpc.RpcEndpointRef case class ReceiverInfo( streamId: Int, name: String, - private[streaming] val endpoint: RpcEndpointRef, active: Boolean, location: String, lastErrorMessage: String = "", diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala index 831a40dea9c7..de6373c9e91f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala @@ -17,29 +17,8 @@ package org.apache.spark.streaming.scheduler +import scala.collection.Map import scala.collection.mutable -import scala.util.Random - -import org.apache.spark.streaming.scheduler.ReceiverState._ - -private[streaming] case class ReceiverTrackingInfo( - receiverId: Int, - state: ReceiverState, - scheduledLocations: Option[Seq[String]], - runningLocation: Option[String]) - -private[streaming] trait ReceiverSchedulingPolicy { - - /** - * Return a list of candidate executors to run the receiver. If the list is empty, the caller can - * run this receiver in arbitrary executor. - */ - def scheduleReceiver( - receiverId: Int, - preferredLocation: Option[String], - receiverTrackingInfoMap: Map[Int, ReceiverTrackingInfo], - executors: Seq[String]): Seq[String] -} /** * A ReceiverScheduler trying to balance executors' load. Here is the approach to schedule executors @@ -65,8 +44,12 @@ private[streaming] trait ReceiverSchedulingPolicy { * * */ -private[streaming] class LoadBalanceReceiverSchedulingPolicyImpl extends ReceiverSchedulingPolicy { +private[streaming] class ReceiverSchedulingPolicy { + /** + * Return a list of candidate executors to run the receiver. If the list is empty, the caller can + * run this receiver in arbitrary executor. + */ def scheduleReceiver( receiverId: Int, preferredLocation: Option[String], @@ -80,17 +63,14 @@ private[streaming] class LoadBalanceReceiverSchedulingPolicyImpl extends Receive val locations = mutable.Set[String]() locations ++= preferredLocation - val executorWeights = receiverTrackingInfoMap.filter { case (id, _) => - // Ignore the receiver to be scheduled. It may be still running. - id != receiverId - }.values.flatMap { receiverTrackingInfo => + val executorWeights = receiverTrackingInfoMap.values.flatMap { receiverTrackingInfo => receiverTrackingInfo.state match { case ReceiverState.INACTIVE => Nil case ReceiverState.SCHEDULED => val scheduledLocations = receiverTrackingInfo.scheduledLocations.get // The probability that a scheduled receiver will run in an executor is // 1.0 / scheduledLocations.size - scheduledLocations.map(location => location -> 1.0 / scheduledLocations.size) + scheduledLocations.map(location => location -> (1.0 / scheduledLocations.size)) case ReceiverState.ACTIVE => Seq(receiverTrackingInfo.runningLocation.get -> 1.0) } }.groupBy(_._1).mapValues(_.map(_._2).sum) // Sum weights for each executor diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 15773cdda773..c9645c15c936 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -19,10 +19,9 @@ package org.apache.spark.streaming.scheduler import java.util.concurrent.CountDownLatch -import scala.collection.mutable.{ArrayBuffer, HashMap, SynchronizedMap} +import scala.collection.mutable.HashMap import scala.concurrent.ExecutionContext import scala.language.existentials -import scala.math.max import scala.util.{Failure, Success} import org.apache.spark.streaming.util.WriteAheadLogUtils @@ -30,8 +29,7 @@ import org.apache.spark.{TaskContext, Logging, SparkEnv, SparkException} import org.apache.spark.rdd.RDD import org.apache.spark.rpc._ import org.apache.spark.streaming.{StreamingContext, Time} -import org.apache.spark.streaming.receiver.{CleanupOldBlocks, Receiver, ReceiverSupervisorImpl, - StopReceiver} +import org.apache.spark.streaming.receiver._ import org.apache.spark.util.{ThreadUtils, SerializableConfiguration} @@ -58,7 +56,28 @@ private[streaming] case class ReportError(streamId: Int, message: String, error: private[streaming] case class DeregisterReceiver(streamId: Int, msg: String, error: String) extends ReceiverTrackerMessage -private[streaming] case object StopAllReceivers extends ReceiverTrackerMessage +/** + * Messages used by the driver and ReceiverTrackerEndpoint to communicate locally. + */ +private[streaming] sealed trait ReceiverTrackerLocalMessage + +/** + * This message will trigger ReceiverTrackerEndpoint to start a Spark job for the receiver. + */ +private[streaming] case class StartReceiver(receiver: Receiver[_]) + extends ReceiverTrackerLocalMessage + +/** + * This message will trigger ReceiverTrackerEndpoint to send stop signals to all registered + * receivers. + */ +private[streaming] case object StopAllReceivers extends ReceiverTrackerLocalMessage + +/** + * A message used by ReceiverTracker to ask all receiver's ids still stored in + * ReceiverTrackerEndpoint. + */ +private[streaming] case object AllReceiverIds extends ReceiverTrackerLocalMessage /** * This class manages the execution of the receivers of ReceiverInputDStreams. Instance of @@ -73,7 +92,6 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false private val receiverInputStreams = ssc.graph.getReceiverInputStreams() private val receiverInputStreamIds = receiverInputStreams.map { _.id } private val receiverExecutor = new ReceiverLauncher() - private val receiverInfo = new HashMap[Int, ReceiverInfo] with SynchronizedMap[Int, ReceiverInfo] private val receivedBlockTracker = new ReceivedBlockTracker( ssc.sparkContext.conf, ssc.sparkContext.hadoopConfiguration, @@ -98,21 +116,11 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false // This not being null means the tracker has been started and not stopped private var endpoint: RpcEndpointRef = null - private val schedulingPolicy: ReceiverSchedulingPolicy = - new LoadBalanceReceiverSchedulingPolicyImpl() - - /** - * Track receivers' status for scheduling - */ - private val receiverTrackingInfos = new HashMap[Int, ReceiverTrackingInfo] - - /** - * Store all preferred locations for all receivers. We need this information to schedule receivers - */ - private val receiverPreferredLocations = new HashMap[Int, Option[String]] + private val schedulingPolicy: ReceiverSchedulingPolicy = new ReceiverSchedulingPolicy() - /** Use a separate lock to avoid dead-lock */ - private val receiverTrackingInfosLock = new AnyRef + // Track the active receiver job number. When a receiver job exits ultimately, countDown will + // be called. + private val receiverJobExitLatch = new CountDownLatch(receiverInputStreams.size) /** Start the endpoint and receiver execution thread. */ def start(): Unit = synchronized { @@ -145,15 +153,16 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false if (graceful) { val pollTime = 100 logInfo("Waiting for receiver job to terminate gracefully") - while (receiverInfo.nonEmpty || receiverExecutor.running) { + while (receiverExecutor.running) { Thread.sleep(pollTime) } logInfo("Waited for receiver job to terminate gracefully") } // Check if all the receivers have been deregistered or not - if (receiverInfo.nonEmpty) { - logWarning("Not all of the receivers have deregistered, " + receiverInfo) + val receivers = endpoint.askWithRetry[Seq[Int]](AllReceiverIds) + if (receivers.nonEmpty) { + logWarning("Not all of the receivers have deregistered, " + receivers) } else { logInfo("All of the receivers have deregistered successfully") } @@ -198,102 +207,64 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false // Signal the receivers to delete old block data if (WriteAheadLogUtils.enableReceiverLog(ssc.conf)) { logInfo(s"Cleanup old received batch data: $cleanupThreshTime") - receiverInfo.values.flatMap { info => Option(info.endpoint) } - .foreach { _.send(CleanupOldBlocks(cleanupThreshTime)) } + endpoint.send(CleanupOldBlocks(cleanupThreshTime)) } } - /** Register a receiver */ - private def registerReceiver( - streamId: Int, - typ: String, - host: String, - receiverEndpoint: RpcEndpointRef, - senderAddress: RpcAddress - ): Boolean = { - if (!receiverInputStreamIds.contains(streamId)) { - throw new SparkException("Register received for unexpected id " + streamId) - } - - if (isTrackerStopping || isTrackerStopped) { - false - } else if (!ssc.sparkContext.isLocal && // We don't need to schedule it in the local mode - !scheduleReceiver(streamId).contains(host)) { - false - } else { - receiverInfo(streamId) = ReceiverInfo( - streamId, s"${typ}-${streamId}", receiverEndpoint, true, host) - updateReceiverRunningLocation(streamId, host) - listenerBus.post(StreamingListenerReceiverStarted(receiverInfo(streamId))) - logInfo("Registered receiver for stream " + streamId + " from " + senderAddress) - true - } + /** Check if any blocks are left to be processed */ + def hasUnallocatedBlocks: Boolean = { + receivedBlockTracker.hasUnallocatedReceivedBlocks } - /** Deregister a receiver */ - private def deregisterReceiver(streamId: Int, message: String, error: String) { - val newReceiverInfo = receiverInfo.get(streamId) match { - case Some(oldInfo) => - val lastErrorTime = - if (error == null || error == "") -1 else ssc.scheduler.clock.getTimeMillis() - oldInfo.copy(endpoint = null, active = false, lastErrorMessage = message, - lastError = error, lastErrorTime = lastErrorTime) - case None => - logWarning("No prior receiver info") - val lastErrorTime = - if (error == null || error == "") -1 else ssc.scheduler.clock.getTimeMillis() - ReceiverInfo(streamId, "", null, false, "", lastErrorMessage = message, - lastError = error, lastErrorTime = lastErrorTime) - } - receiverInfo -= streamId - listenerBus.post(StreamingListenerReceiverStopped(newReceiverInfo)) - val messageWithError = if (error != null && !error.isEmpty) { - s"$message - $error" - } else { - s"$message" - } - logError(s"Deregistered receiver for stream $streamId: $messageWithError") + /** + * Get the list of executors excluding driver + */ + private def getExecutors(ssc: StreamingContext): List[String] = { + val executors = ssc.sparkContext.getExecutorMemoryStatus.map(_._1.split(":")(0)).toList + val driver = ssc.sparkContext.getConf.get("spark.driver.host") + executors.diff(List(driver)) } - /** Add new blocks for the given stream */ - private def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = { - receivedBlockTracker.addBlock(receivedBlockInfo) - } + /** Check if tracker has been marked for starting */ + private def isTrackerStarted(): Boolean = trackerState == Started - /** Report error sent by a receiver */ - private def reportError(streamId: Int, message: String, error: String) { - val newReceiverInfo = receiverInfo.get(streamId) match { - case Some(oldInfo) => - oldInfo.copy(lastErrorMessage = message, lastError = error) - case None => - logWarning("No prior receiver info") - ReceiverInfo(streamId, "", null, false, "", lastErrorMessage = message, - lastError = error, lastErrorTime = ssc.scheduler.clock.getTimeMillis()) - } - receiverInfo(streamId) = newReceiverInfo - listenerBus.post(StreamingListenerReceiverError(receiverInfo(streamId))) - val messageWithError = if (error != null && !error.isEmpty) { - s"$message - $error" - } else { - s"$message" - } - logWarning(s"Error reported by receiver for stream $streamId: $messageWithError") - } + /** Check if tracker has been marked for stopping */ + private def isTrackerStopping(): Boolean = trackerState == Stopping - /** Check if any blocks are left to be processed */ - def hasUnallocatedBlocks: Boolean = { - receivedBlockTracker.hasUnallocatedReceivedBlocks - } + /** Check if tracker has been marked for stopped */ + private def isTrackerStopped(): Boolean = trackerState == Stopped /** RpcEndpoint to receive messages from the receivers. */ private class ReceiverTrackerEndpoint(override val rpcEnv: RpcEnv) extends ThreadSafeRpcEndpoint { + /** + * Track all receivers' information. The key is the receiver id, the value is the receiver info. + */ + private val receiverTrackingInfos = new HashMap[Int, ReceiverTrackingInfo] + + /** + * Store all preferred locations for all receivers. We need this information to schedule + * receivers + */ + private val receiverPreferredLocations = new HashMap[Int, Option[String]] + + // TODO Remove this thread pool after https://github.com/apache/spark/issues/7385 is merged + private val submitJobThreadPool = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("submit-job-thead-pool")) + override def receive: PartialFunction[Any, Unit] = { + // Local messages + case StartReceiver(receiver) => + startReceiver(receiver) + case c @ CleanupOldBlocks(cleanupThreshTime) => + receiverTrackingInfos.values.flatMap(_.endpoint).foreach(_.send(c)) + // Remote messages case ReportError(streamId, message, error) => reportError(streamId, message, error) } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + // Remote messages case RegisterReceiver(streamId, typ, host, receiverEndpoint) => val successful = registerReceiver(streamId, typ, host, receiverEndpoint, context.sender.address) @@ -303,18 +274,202 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false case DeregisterReceiver(streamId, message, error) => deregisterReceiver(streamId, message, error) context.reply(true) + // Local messages + case AllReceiverIds => + context.reply(receiverTrackingInfos.keys.toSeq) case StopAllReceivers => assert(isTrackerStopping || isTrackerStopped) stopReceivers() context.reply(true) } + private def startReceiver(receiver: Receiver[_]): Unit = { + val checkpointDirOption = Option(ssc.checkpointDir) + val serializableHadoopConf = + new SerializableConfiguration(ssc.sparkContext.hadoopConfiguration) + + // Function to start the receiver on the worker node + val startReceiverFunc = new StartReceiverFunc(checkpointDirOption, serializableHadoopConf) + + receiverPreferredLocations(receiver.streamId) = receiver.preferredLocation + val receiverId = receiver.streamId + + if (isTrackerStopping() || isTrackerStopped()) { + onReceiverJobFinish(receiverId) + return + } + + val scheduledLocations = schedulingPolicy.scheduleReceiver( + receiverId, + receiver.preferredLocation, + receiverTrackingInfos, + getExecutors(ssc)) + updateReceiverScheduledLocations(receiver.streamId, scheduledLocations) + + // Create the RDD using the scheduledLocations to run the receiver in a Spark job + val receiverRDD: RDD[Receiver[_]] = + if (scheduledLocations.isEmpty) { + ssc.sc.makeRDD(Seq(receiver), 1) + } else { + ssc.sc.makeRDD(Seq(receiver -> scheduledLocations)) + } + val future = ssc.sparkContext.submitJob[Receiver[_], Unit, Unit]( + receiverRDD, startReceiverFunc, Seq(0), (_, _) => Unit, ()) + // We will keep restarting the receiver job until ReceiverTracker is stopped + future.onComplete { + case Success(_) => + if (isTrackerStopping() || isTrackerStopped()) { + onReceiverJobFinish(receiverId) + } else { + logInfo(s"Restarting Receiver $receiverId") + self.send(StartReceiver(receiver)) + } + case Failure(e) => + if (isTrackerStopping() || isTrackerStopped()) { + onReceiverJobFinish(receiverId) + } else { + logError("Receiver has been stopped. Try to restart it.", e) + logInfo(s"Restarting Receiver $receiverId") + self.send(StartReceiver(receiver)) + } + }(submitJobThreadPool) + logInfo(s"Receiver ${receiver.streamId} started") + } + + override def onStop(): Unit = { + submitJobThreadPool.shutdownNow() + } + + /** + * Call when a receiver is terminated. It means we won't restart its Spark job. + */ + private def onReceiverJobFinish(receiverId: Int): Unit = { + receiverJobExitLatch.countDown() + receiverTrackingInfos.remove(receiverId).foreach { receiverTrackingInfo => + if (receiverTrackingInfo.state == ReceiverState.ACTIVE) { + logWarning(s"Receiver $receiverId exited but didn't deregister") + } + } + } + + /** Register a receiver */ + private def registerReceiver( + streamId: Int, + typ: String, + host: String, + receiverEndpoint: RpcEndpointRef, + senderAddress: RpcAddress + ): Boolean = { + if (!receiverInputStreamIds.contains(streamId)) { + throw new SparkException("Register received for unexpected id " + streamId) + } + + if (isTrackerStopping || isTrackerStopped) { + false + } else if (!ssc.sparkContext.isLocal && // We don't need to schedule it in the local mode + !scheduleReceiver(streamId).contains(host)) { + // Refuse it since it's scheduled to a wrong executor + false + } else { + val name = s"${typ}-${streamId}" + val receiverInfo = ReceiverInfo(streamId, name, true, host) + receiverTrackingInfos.put(streamId, + ReceiverTrackingInfo( + streamId, + ReceiverState.ACTIVE, + scheduledLocations = None, + runningLocation = Some(host), + name = Some(name), + endpoint = Some(receiverEndpoint))) + listenerBus.post(StreamingListenerReceiverStarted(receiverInfo)) + logInfo("Registered receiver for stream " + streamId + " from " + senderAddress) + true + } + } + + /** Deregister a receiver */ + private def deregisterReceiver(streamId: Int, message: String, error: String) { + val lastErrorTime = + if (error == null || error == "") -1 else ssc.scheduler.clock.getTimeMillis() + val errorInfo = ReceiverErrorInfo( + lastErrorMessage = message, lastError = error, lastErrorTime = lastErrorTime) + val newReceiverTrackingInfo = receiverTrackingInfos.get(streamId) match { + case Some(oldInfo) => + oldInfo.copy(errorInfo = Some(errorInfo)) + case None => + logWarning("No prior receiver info") + ReceiverTrackingInfo( + streamId, ReceiverState.INACTIVE, None, None, None, None, Some(errorInfo)) + } + receiverTrackingInfos -= streamId + listenerBus.post(StreamingListenerReceiverStopped(newReceiverTrackingInfo.toReceiverInfo)) + val messageWithError = if (error != null && !error.isEmpty) { + s"$message - $error" + } else { + s"$message" + } + logError(s"Deregistered receiver for stream $streamId: $messageWithError") + } + + /** Add new blocks for the given stream */ + private def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = { + receivedBlockTracker.addBlock(receivedBlockInfo) + } + + /** Report error sent by a receiver */ + private def reportError(streamId: Int, message: String, error: String) { + val newReceiverTrackingInfo = receiverTrackingInfos.get(streamId) match { + case Some(oldInfo) => + val errorInfo = ReceiverErrorInfo(lastErrorMessage = message, lastError = error, + lastErrorTime = oldInfo.errorInfo.map(_.lastErrorTime).getOrElse(-1L)) + oldInfo.copy(errorInfo = Some(errorInfo)) + case None => + logWarning("No prior receiver info") + val errorInfo = ReceiverErrorInfo(lastErrorMessage = message, lastError = error, + lastErrorTime = ssc.scheduler.clock.getTimeMillis()) + ReceiverTrackingInfo( + streamId, ReceiverState.INACTIVE, None, None, None, None, Some(errorInfo)) + } + + receiverTrackingInfos(streamId) = newReceiverTrackingInfo + listenerBus.post(StreamingListenerReceiverError(newReceiverTrackingInfo.toReceiverInfo)) + val messageWithError = if (error != null && !error.isEmpty) { + s"$message - $error" + } else { + s"$message" + } + logWarning(s"Error reported by receiver for stream $streamId: $messageWithError") + } + + private def updateReceiverScheduledLocations( + receiverId: Int, scheduledLocations: Seq[String]): Unit = { + val newReceiverTrackingInfo = receiverTrackingInfos.get(receiverId) match { + case Some(oldInfo) => + oldInfo.copy(state = ReceiverState.SCHEDULED, + scheduledLocations = Some(scheduledLocations)) + case None => + ReceiverTrackingInfo( + receiverId, + ReceiverState.SCHEDULED, + Some(scheduledLocations), + None) + } + receiverTrackingInfos.put(receiverId, newReceiverTrackingInfo) + } + + private def scheduleReceiver(receiverId: Int): Seq[String] = { + val preferredLocation = receiverPreferredLocations.getOrElse(receiverId, None) + val scheduledLocations = schedulingPolicy.scheduleReceiver( + receiverId, preferredLocation, receiverTrackingInfos, getExecutors(ssc)) + updateReceiverScheduledLocations(receiverId, scheduledLocations) + scheduledLocations + } + /** Send stop signal to the receivers. */ private def stopReceivers() { // Signal the receivers to stop - receiverInfo.values.flatMap { info => Option(info.endpoint)} - .foreach { _.send(StopReceiver) } - logInfo("Sent stop signal to all " + receiverInfo.size + " receivers") + receiverTrackingInfos.values.flatMap(_.endpoint).foreach { _.send(StopReceiver) } + logInfo("Sent stop signal to all " + receiverTrackingInfos.size + " receivers") } } @@ -337,35 +492,6 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false thread.start() } - /** Set host location(s) for each receiver so as to distribute them over - * executors in a round-robin fashion taking into account preferredLocation if set - */ - private[streaming] def scheduleReceivers(receivers: Seq[Receiver[_]], - executors: List[String]): Array[ArrayBuffer[String]] = { - val locations = new Array[ArrayBuffer[String]](receivers.length) - var i = 0 - for (i <- 0 until receivers.length) { - locations(i) = new ArrayBuffer[String]() - if (receivers(i).preferredLocation.isDefined) { - locations(i) += receivers(i).preferredLocation.get - } - } - var count = 0 - for (i <- 0 until max(receivers.length, executors.length)) { - if (!receivers(i % receivers.length).preferredLocation.isDefined) { - locations(i % receivers.length) += executors(count) - count += 1 - if (count == executors.length) { - count = 0 - } - } - } - locations - } - - private val submitJobThread = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonSingleThreadExecutor("streaming-submit-job")) - /** * Get the receivers from the ReceiverInputDStreams, distributes them to the * worker nodes as a parallel collection, and runs them. @@ -377,32 +503,6 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false rcvr }) - initReceiverTrackingInfos(receivers) - - // Tracking the active receiver number. When a receiver exits, countDown will be called. - val receiverExitLatch = new CountDownLatch(receivers.size) - - val checkpointDirOption = Option(ssc.checkpointDir) - val serializableHadoopConf = - new SerializableConfiguration(ssc.sparkContext.hadoopConfiguration) - - // Function to start the receiver on the worker node - val startReceiver = (iterator: Iterator[Receiver[_]]) => { - if (!iterator.hasNext) { - throw new SparkException( - "Could not start receiver as object not found.") - } - if (TaskContext.get().attemptNumber() == 0) { - val receiver = iterator.next() - val supervisor = new ReceiverSupervisorImpl( - receiver, SparkEnv.get, serializableHadoopConf.value, checkpointDirOption) - supervisor.start() - supervisor.awaitTermination() - } else { - // It's restarted by TaskScheduler, but we want to reschedule it again. So exit it. - } - } - // Run the dummy Spark job to ensure that all slaves have registered. // This avoids all the receivers to be scheduled on the same node. if (!ssc.sparkContext.isLocal) { @@ -413,58 +513,15 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false logInfo("Starting " + receivers.length + " receivers") running = true - for (receiver <- receivers) { - submitJobThread.execute(new Runnable { - override def run(): Unit = { - if (isTrackerStopping()) { - receiverExitLatch.countDown() - return - } - - val self = this - val receiverId = receiver.streamId - val scheduledLocations = schedulingPolicy.scheduleReceiver( - receiverId, - receiver.preferredLocation, - getReceiverTrackingInfoMap(), - getExecutors(ssc)) - updateReceiverScheduledLocations(receiver.streamId, scheduledLocations) - val receiverRDD: RDD[Receiver[_]] = - if (scheduledLocations.isEmpty) { - ssc.sc.makeRDD(Seq(receiver), 1) - } else { - ssc.sc.makeRDD(Seq(receiver -> scheduledLocations)) - } - val future = ssc.sparkContext.submitJob[Receiver[_], Unit, Unit]( - receiverRDD, startReceiver, Seq(0), (_, _) => Unit, ()) - future.onComplete { - case Success(_) => - if (isTrackerStopping()) { - receiverExitLatch.countDown() - } else { - logInfo(s"Restarting Receiver $receiverId") - submitJobThread.execute(self) - } - case Failure(e) => - if (isTrackerStopping()) { - receiverExitLatch.countDown() - } else { - logError("Receiver has been stopped. Try to restart it.", e) - logInfo(s"Restarting Receiver $receiverId") - submitJobThread.execute(self) - } - }(ThreadUtils.sameThread) - logInfo(s"Receiver ${receiver.streamId} started") - } - }) - } try { + for (receiver <- receivers) { + endpoint.send(StartReceiver(receiver)) + } // Wait until all receivers exit - receiverExitLatch.await() + receiverJobExitLatch.await() logInfo("All of the receivers have been terminated") } finally { running = false - submitJobThread.shutdownNow() } } @@ -477,68 +534,32 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } } - def initReceiverTrackingInfos(receivers: Seq[Receiver[_]]): Unit = synchronized { - for (receiver <- receivers) { - receiverPreferredLocations(receiver.streamId) = receiver.preferredLocation - receiverTrackingInfos.put(receiver.streamId, ReceiverTrackingInfo( - receiver.streamId, - ReceiverState.INACTIVE, - None, - None)) - } - } - - private def getReceiverTrackingInfoMap(): Map[Int, ReceiverTrackingInfo] = - receiverTrackingInfosLock.synchronized { - // Copy to an immutable Map so that we don't need to use `synchronized` when using it - receiverTrackingInfos.toMap - } - - private def updateReceiverScheduledLocations( - receiverId: Int, scheduledLocations: Seq[String]): Unit = - receiverTrackingInfosLock.synchronized { - receiverTrackingInfos.put(receiverId, ReceiverTrackingInfo( - receiverId, - ReceiverState.SCHEDULED, - Some(scheduledLocations), - None)) - } +} - private def updateReceiverRunningLocation(receiverId: Int, runningLocation: String): Unit = - receiverTrackingInfosLock.synchronized { - receiverTrackingInfos.put(receiverId, ReceiverTrackingInfo( - receiverId, - ReceiverState.ACTIVE, - None, - Some(runningLocation))) +/** + * Function to start the receiver on the worker node. Use a class instead of closure to avoid + * the serialization issue. + */ +private class StartReceiverFunc( + checkpointDirOption: Option[String], + serializableHadoopConf: SerializableConfiguration) + extends (Iterator[Receiver[_]] => Unit) with Serializable { + + override def apply(iterator: Iterator[Receiver[_]]): Unit = { + if (!iterator.hasNext) { + throw new SparkException( + "Could not start receiver as object not found.") } - - private def scheduleReceiver(receiverId: Int): Seq[String] = { - val preferredLocation = receiverTrackingInfosLock.synchronized { - receiverPreferredLocations(receiverId) + if (TaskContext.get().attemptNumber() == 0) { + val receiver = iterator.next() + assert(iterator.hasNext == false) + val supervisor = new ReceiverSupervisorImpl( + receiver, SparkEnv.get, serializableHadoopConf.value, checkpointDirOption) + supervisor.start() + supervisor.awaitTermination() + } else { + // It's restarted by TaskScheduler, but we want to reschedule it again. So exit it. } - val scheduledLocations = schedulingPolicy.scheduleReceiver( - receiverId, preferredLocation, getReceiverTrackingInfoMap(), getExecutors(ssc)) - updateReceiverScheduledLocations(receiverId, scheduledLocations) - scheduledLocations - } - - /** - * Get the list of executors excluding driver - */ - private def getExecutors(ssc: StreamingContext): List[String] = { - val executors = ssc.sparkContext.getExecutorMemoryStatus.map(_._1.split(":")(0)).toList - val driver = ssc.sparkContext.getConf.get("spark.driver.host") - executors.diff(List(driver)) } - /** Check if tracker has been marked for starting */ - private def isTrackerStarted(): Boolean = trackerState == Started - - /** Check if tracker has been marked for stopping */ - private def isTrackerStopping(): Boolean = trackerState == Stopping - - /** Check if tracker has been marked for stopped */ - private def isTrackerStopped(): Boolean = trackerState == Stopped - } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTrackingInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTrackingInfo.scala new file mode 100644 index 000000000000..75afc1ad42b5 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTrackingInfo.scala @@ -0,0 +1,55 @@ +/* + * 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.streaming.scheduler + +import org.apache.spark.rpc.RpcEndpointRef +import org.apache.spark.streaming.scheduler.ReceiverState._ + +private[streaming] case class ReceiverErrorInfo( + lastErrorMessage: String = "", lastError: String = "", lastErrorTime: Long = -1L) + +/** + * Class having information about a receiver. + * + * @param receiverId the unique receiver id + * @param state the current Receiver state + * @param scheduledLocations the scheduled locations provided by ReceiverSchedulingPolicy + * @param runningLocation the running location if the receiver is active + * @param name the receiver name + * @param endpoint the receiver endpoint. It can be used to send messages to the receiver + * @param errorInfo the receiver error information if it fails + */ +private[streaming] case class ReceiverTrackingInfo( + receiverId: Int, + state: ReceiverState, + scheduledLocations: Option[Seq[String]], + runningLocation: Option[String], + name: Option[String] = None, + endpoint: Option[RpcEndpointRef] = None, + errorInfo: Option[ReceiverErrorInfo] = None) { + + def toReceiverInfo: ReceiverInfo = ReceiverInfo( + receiverId, + name.getOrElse(""), + state == ReceiverState.ACTIVE, + location = runningLocation.getOrElse(""), + lastErrorMessage = errorInfo.map(_.lastErrorMessage).getOrElse(""), + lastError = errorInfo.map(_.lastError).getOrElse(""), + lastErrorTime = errorInfo.map(_.lastErrorTime).getOrElse(-1L) + ) +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/LoadBalanceReceiverSchedulingPolicyImplSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala similarity index 59% rename from streaming/src/test/scala/org/apache/spark/streaming/scheduler/LoadBalanceReceiverSchedulingPolicyImplSuite.scala rename to streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala index e6550f0fbe5e..a56267eb762c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/LoadBalanceReceiverSchedulingPolicyImplSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.streaming.scheduler import org.apache.spark.SparkFunSuite -class LoadBalanceReceiverSchedulingPolicyImplSuite extends SparkFunSuite { +class ReceiverSchedulingPolicySuite extends SparkFunSuite { - val receiverSchedulingPolicy = new LoadBalanceReceiverSchedulingPolicyImpl + val receiverSchedulingPolicy = new ReceiverSchedulingPolicy test("empty executors") { val scheduledLocations = @@ -37,43 +37,26 @@ class LoadBalanceReceiverSchedulingPolicyImplSuite extends SparkFunSuite { assert(scheduledLocations.toSet === Set("host1", "host2")) } - test("choose the idle executor") { - val executors = Seq("host1", "host2", "host3") + test("return all idle executors if more than 3 idle executors") { + val executors = Seq("host1", "host2", "host3", "host4", "host5") // host3 is idle val receiverTrackingInfoMap = Map( - 0 -> ReceiverTrackingInfo(0, ReceiverState.ACTIVE, None, Some("host1")), - 1 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host2")), None)) + 0 -> ReceiverTrackingInfo(0, ReceiverState.ACTIVE, None, Some("host1"))) val scheduledLocations = receiverSchedulingPolicy.scheduleReceiver( - 2, None, receiverTrackingInfoMap, executors) - assert(scheduledLocations.toSet === Set("host3")) + 1, None, receiverTrackingInfoMap, executors) + assert(scheduledLocations.toSet === Set("host2", "host3", "host4", "host5")) } - test("all executors are busy") { - val executors = Seq("host1", "host2", "host3") + test("return 3 best options if less than 3 idle executors") { + val executors = Seq("host1", "host2", "host3", "host4", "host5") // Weights: host1 = 1.5, host2 = 0.5, host3 = 1.0 + // host4 and host5 are idle val receiverTrackingInfoMap = Map( 0 -> ReceiverTrackingInfo(0, ReceiverState.ACTIVE, None, Some("host1")), 1 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host2", "host3")), None), 2 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host1", "host3")), None)) val scheduledLocations = receiverSchedulingPolicy.scheduleReceiver( 3, None, receiverTrackingInfoMap, executors) - assert(scheduledLocations.toSet === Set("host2")) - } - - test("ignore the receiver's info") { - val executors = Seq("host1", "host2", "host3") - // Weights: host1 = 1.0, host2 = 1.5, host3 = 1.5 - // But since we are scheduling the receiver 1, we should ignore - // receiver 1's ReceiverTrackingInfo - // So the new weights are host1 = 1.0, host2 = 0.5, host3 = 1.5 - // Then the scheduled location should be "host2" - val receiverTrackingInfoMap = Map( - 0 -> ReceiverTrackingInfo(0, ReceiverState.ACTIVE, None, Some("host1")), - 1 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host2")), None), - 2 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host3")), None), - 3 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host2", "host3")), None)) - val scheduledLocations = receiverSchedulingPolicy.scheduleReceiver( - 1, None, receiverTrackingInfoMap, executors) - assert(scheduledLocations.toSet === Set("host2")) + assert(scheduledLocations.toSet === Set("host2", "host4", "host5")) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala deleted file mode 100644 index a6e783861dbe..000000000000 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala +++ /dev/null @@ -1,90 +0,0 @@ -/* - * 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.streaming.scheduler - -import org.apache.spark.streaming._ -import org.apache.spark.SparkConf -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.receiver._ -import org.apache.spark.util.Utils - -/** Testsuite for receiver scheduling */ -class ReceiverTrackerSuite extends TestSuiteBase { - val sparkConf = new SparkConf().setMaster("local[8]").setAppName("test") - val ssc = new StreamingContext(sparkConf, Milliseconds(100)) - val tracker = new ReceiverTracker(ssc) - val launcher = new tracker.ReceiverLauncher() - val executors: List[String] = List("0", "1", "2", "3") - - test("receiver scheduling - all or none have preferred location") { - - def parse(s: String): Array[Array[String]] = { - val outerSplit = s.split("\\|") - val loc = new Array[Array[String]](outerSplit.length) - var i = 0 - for (i <- 0 until outerSplit.length) { - loc(i) = outerSplit(i).split("\\,") - } - loc - } - - def testScheduler(numReceivers: Int, preferredLocation: Boolean, allocation: String) { - val receivers = - if (preferredLocation) { - Array.tabulate(numReceivers)(i => new DummyReceiver(host = - Some(((i + 1) % executors.length).toString))) - } else { - Array.tabulate(numReceivers)(_ => new DummyReceiver) - } - val locations = launcher.scheduleReceivers(receivers, executors) - val expectedLocations = parse(allocation) - assert(locations.deep === expectedLocations.deep) - } - - testScheduler(numReceivers = 5, preferredLocation = false, allocation = "0|1|2|3|0") - testScheduler(numReceivers = 3, preferredLocation = false, allocation = "0,3|1|2") - testScheduler(numReceivers = 4, preferredLocation = true, allocation = "1|2|3|0") - } - - test("receiver scheduling - some have preferred location") { - val numReceivers = 4; - val receivers: Seq[Receiver[_]] = Seq(new DummyReceiver(host = Some("1")), - new DummyReceiver, new DummyReceiver, new DummyReceiver) - val locations = launcher.scheduleReceivers(receivers, executors) - assert(locations(0)(0) === "1") - assert(locations(1)(0) === "0") - assert(locations(2)(0) === "1") - assert(locations(0).length === 1) - assert(locations(3).length === 1) - } -} - -/** - * Dummy receiver implementation - */ -private class DummyReceiver(host: Option[String] = None) - extends Receiver[Int](StorageLevel.MEMORY_ONLY) { - - def onStart() { - } - - def onStop() { - } - - override def preferredLocation: Option[String] = host -} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala index 40dc1fb601bd..0891309f956d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala @@ -119,20 +119,20 @@ class StreamingJobProgressListenerSuite extends TestSuiteBase with Matchers { listener.numTotalReceivedRecords should be (600) // onReceiverStarted - val receiverInfoStarted = ReceiverInfo(0, "test", null, true, "localhost") + val receiverInfoStarted = ReceiverInfo(0, "test", true, "localhost") listener.onReceiverStarted(StreamingListenerReceiverStarted(receiverInfoStarted)) listener.receiverInfo(0) should be (Some(receiverInfoStarted)) listener.receiverInfo(1) should be (None) // onReceiverError - val receiverInfoError = ReceiverInfo(1, "test", null, true, "localhost") + val receiverInfoError = ReceiverInfo(1, "test", true, "localhost") listener.onReceiverError(StreamingListenerReceiverError(receiverInfoError)) listener.receiverInfo(0) should be (Some(receiverInfoStarted)) listener.receiverInfo(1) should be (Some(receiverInfoError)) listener.receiverInfo(2) should be (None) // onReceiverStopped - val receiverInfoStopped = ReceiverInfo(2, "test", null, true, "localhost") + val receiverInfoStopped = ReceiverInfo(2, "test", true, "localhost") listener.onReceiverStopped(StreamingListenerReceiverStopped(receiverInfoStopped)) listener.receiverInfo(0) should be (Some(receiverInfoStarted)) listener.receiverInfo(1) should be (Some(receiverInfoError)) From 4e639c4bee03f4cc1dc3c7514ad473e6cb3acabd Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 22 Jul 2015 00:54:29 +0800 Subject: [PATCH 16/27] Fix unintentional changes --- .../streaming/receiver/ReceiverSupervisor.scala | 2 +- .../org/apache/spark/streaming/ReceiverSuite.scala | 12 ------------ 2 files changed, 1 insertion(+), 13 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index 0a88572e8362..9184c8fbcc5f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -101,7 +101,7 @@ private[streaming] abstract class ReceiverSupervisor( protected def onStop(message: String, error: Option[Throwable]) { } /** Called when receiver is started. Return true if the driver accepts us */ - protected def onReceiverStart(): Boolean = true + protected def onReceiverStart(): Boolean /** Called when receiver is stopped */ protected def onReceiverStop(message: String, error: Option[Throwable]) { } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index fe205436014a..13b4d17c8618 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -129,18 +129,6 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { } } - test("restart receiver should consider the scheduled locations") { - val receiver = new FakeReceiver - val executor = new FakeReceiverSupervisor(receiver) - executor.start() - receiver.restart("force the receiver restart") - eventually(timeout(30000 millis), interval(10 millis)) { - // Since the scheduled location is not the current host, the receiver should exit - assert(receiver.onStopCalled) - assert(receiver.isStopped) - } - } - test("block generator") { val blockGeneratorListener = new FakeBlockGeneratorListener val blockIntervalMs = 200 From fab9a01b2c9a83d21ecf5da351ffdb8a07b6910d Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 22 Jul 2015 15:55:47 +0800 Subject: [PATCH 17/27] Move methods back to the outer class --- .../streaming/scheduler/ReceiverTracker.scala | 252 +++++++++--------- 1 file changed, 125 insertions(+), 127 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index c9645c15c936..125535c70508 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -122,6 +122,17 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false // be called. private val receiverJobExitLatch = new CountDownLatch(receiverInputStreams.size) + /** + * Track all receivers' information. The key is the receiver id, the value is the receiver info. + */ + private val receiverTrackingInfos = new HashMap[Int, ReceiverTrackingInfo] + + /** + * Store all preferred locations for all receivers. We need this information to schedule + * receivers + */ + private val receiverPreferredLocations = new HashMap[Int, Option[String]] + /** Start the endpoint and receiver execution thread. */ def start(): Unit = synchronized { if (isTrackerStarted) { @@ -191,9 +202,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** Get the blocks allocated to the given batch and stream. */ def getBlocksOfBatchAndStream(batchTime: Time, streamId: Int): Seq[ReceivedBlockInfo] = { - synchronized { - receivedBlockTracker.getBlocksOfBatchAndStream(batchTime, streamId) - } + receivedBlockTracker.getBlocksOfBatchAndStream(batchTime, streamId) } /** @@ -211,6 +220,119 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } } + /** Register a receiver */ + private def registerReceiver( + streamId: Int, + typ: String, + host: String, + receiverEndpoint: RpcEndpointRef, + senderAddress: RpcAddress + ): Boolean = { + if (!receiverInputStreamIds.contains(streamId)) { + throw new SparkException("Register received for unexpected id " + streamId) + } + + if (isTrackerStopping || isTrackerStopped) { + false + } else if (!ssc.sparkContext.isLocal && // We don't need to schedule it in the local mode + !scheduleReceiver(streamId).contains(host)) { + // Refuse it since it's scheduled to a wrong executor + false + } else { + val name = s"${typ}-${streamId}" + val receiverInfo = ReceiverInfo(streamId, name, true, host) + receiverTrackingInfos.put(streamId, + ReceiverTrackingInfo( + streamId, + ReceiverState.ACTIVE, + scheduledLocations = None, + runningLocation = Some(host), + name = Some(name), + endpoint = Some(receiverEndpoint))) + listenerBus.post(StreamingListenerReceiverStarted(receiverInfo)) + logInfo("Registered receiver for stream " + streamId + " from " + senderAddress) + true + } + } + + /** Deregister a receiver */ + private def deregisterReceiver(streamId: Int, message: String, error: String) { + val lastErrorTime = + if (error == null || error == "") -1 else ssc.scheduler.clock.getTimeMillis() + val errorInfo = ReceiverErrorInfo( + lastErrorMessage = message, lastError = error, lastErrorTime = lastErrorTime) + val newReceiverTrackingInfo = receiverTrackingInfos.get(streamId) match { + case Some(oldInfo) => + oldInfo.copy(errorInfo = Some(errorInfo)) + case None => + logWarning("No prior receiver info") + ReceiverTrackingInfo( + streamId, ReceiverState.INACTIVE, None, None, None, None, Some(errorInfo)) + } + receiverTrackingInfos -= streamId + listenerBus.post(StreamingListenerReceiverStopped(newReceiverTrackingInfo.toReceiverInfo)) + val messageWithError = if (error != null && !error.isEmpty) { + s"$message - $error" + } else { + s"$message" + } + logError(s"Deregistered receiver for stream $streamId: $messageWithError") + } + + /** Add new blocks for the given stream */ + private def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = { + receivedBlockTracker.addBlock(receivedBlockInfo) + } + + /** Report error sent by a receiver */ + private def reportError(streamId: Int, message: String, error: String) { + val newReceiverTrackingInfo = receiverTrackingInfos.get(streamId) match { + case Some(oldInfo) => + val errorInfo = ReceiverErrorInfo(lastErrorMessage = message, lastError = error, + lastErrorTime = oldInfo.errorInfo.map(_.lastErrorTime).getOrElse(-1L)) + oldInfo.copy(errorInfo = Some(errorInfo)) + case None => + logWarning("No prior receiver info") + val errorInfo = ReceiverErrorInfo(lastErrorMessage = message, lastError = error, + lastErrorTime = ssc.scheduler.clock.getTimeMillis()) + ReceiverTrackingInfo( + streamId, ReceiverState.INACTIVE, None, None, None, None, Some(errorInfo)) + } + + receiverTrackingInfos(streamId) = newReceiverTrackingInfo + listenerBus.post(StreamingListenerReceiverError(newReceiverTrackingInfo.toReceiverInfo)) + val messageWithError = if (error != null && !error.isEmpty) { + s"$message - $error" + } else { + s"$message" + } + logWarning(s"Error reported by receiver for stream $streamId: $messageWithError") + } + + private def scheduleReceiver(receiverId: Int): Seq[String] = { + val preferredLocation = receiverPreferredLocations.getOrElse(receiverId, None) + val scheduledLocations = schedulingPolicy.scheduleReceiver( + receiverId, preferredLocation, receiverTrackingInfos, getExecutors(ssc)) + updateReceiverScheduledLocations(receiverId, scheduledLocations) + scheduledLocations + } + + private def updateReceiverScheduledLocations( + receiverId: Int, scheduledLocations: Seq[String]): Unit = { + val newReceiverTrackingInfo = receiverTrackingInfos.get(receiverId) match { + case Some(oldInfo) => + oldInfo.copy(state = ReceiverState.SCHEDULED, + scheduledLocations = Some(scheduledLocations)) + case None => + ReceiverTrackingInfo( + receiverId, + ReceiverState.SCHEDULED, + Some(scheduledLocations), + None) + } + receiverTrackingInfos.put(receiverId, newReceiverTrackingInfo) + } + /** Check if any blocks are left to be processed */ def hasUnallocatedBlocks: Boolean = { receivedBlockTracker.hasUnallocatedReceivedBlocks @@ -237,17 +359,6 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** RpcEndpoint to receive messages from the receivers. */ private class ReceiverTrackerEndpoint(override val rpcEnv: RpcEnv) extends ThreadSafeRpcEndpoint { - /** - * Track all receivers' information. The key is the receiver id, the value is the receiver info. - */ - private val receiverTrackingInfos = new HashMap[Int, ReceiverTrackingInfo] - - /** - * Store all preferred locations for all receivers. We need this information to schedule - * receivers - */ - private val receiverPreferredLocations = new HashMap[Int, Option[String]] - // TODO Remove this thread pool after https://github.com/apache/spark/issues/7385 is merged private val submitJobThreadPool = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("submit-job-thead-pool")) @@ -352,119 +463,6 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } } - /** Register a receiver */ - private def registerReceiver( - streamId: Int, - typ: String, - host: String, - receiverEndpoint: RpcEndpointRef, - senderAddress: RpcAddress - ): Boolean = { - if (!receiverInputStreamIds.contains(streamId)) { - throw new SparkException("Register received for unexpected id " + streamId) - } - - if (isTrackerStopping || isTrackerStopped) { - false - } else if (!ssc.sparkContext.isLocal && // We don't need to schedule it in the local mode - !scheduleReceiver(streamId).contains(host)) { - // Refuse it since it's scheduled to a wrong executor - false - } else { - val name = s"${typ}-${streamId}" - val receiverInfo = ReceiverInfo(streamId, name, true, host) - receiverTrackingInfos.put(streamId, - ReceiverTrackingInfo( - streamId, - ReceiverState.ACTIVE, - scheduledLocations = None, - runningLocation = Some(host), - name = Some(name), - endpoint = Some(receiverEndpoint))) - listenerBus.post(StreamingListenerReceiverStarted(receiverInfo)) - logInfo("Registered receiver for stream " + streamId + " from " + senderAddress) - true - } - } - - /** Deregister a receiver */ - private def deregisterReceiver(streamId: Int, message: String, error: String) { - val lastErrorTime = - if (error == null || error == "") -1 else ssc.scheduler.clock.getTimeMillis() - val errorInfo = ReceiverErrorInfo( - lastErrorMessage = message, lastError = error, lastErrorTime = lastErrorTime) - val newReceiverTrackingInfo = receiverTrackingInfos.get(streamId) match { - case Some(oldInfo) => - oldInfo.copy(errorInfo = Some(errorInfo)) - case None => - logWarning("No prior receiver info") - ReceiverTrackingInfo( - streamId, ReceiverState.INACTIVE, None, None, None, None, Some(errorInfo)) - } - receiverTrackingInfos -= streamId - listenerBus.post(StreamingListenerReceiverStopped(newReceiverTrackingInfo.toReceiverInfo)) - val messageWithError = if (error != null && !error.isEmpty) { - s"$message - $error" - } else { - s"$message" - } - logError(s"Deregistered receiver for stream $streamId: $messageWithError") - } - - /** Add new blocks for the given stream */ - private def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = { - receivedBlockTracker.addBlock(receivedBlockInfo) - } - - /** Report error sent by a receiver */ - private def reportError(streamId: Int, message: String, error: String) { - val newReceiverTrackingInfo = receiverTrackingInfos.get(streamId) match { - case Some(oldInfo) => - val errorInfo = ReceiverErrorInfo(lastErrorMessage = message, lastError = error, - lastErrorTime = oldInfo.errorInfo.map(_.lastErrorTime).getOrElse(-1L)) - oldInfo.copy(errorInfo = Some(errorInfo)) - case None => - logWarning("No prior receiver info") - val errorInfo = ReceiverErrorInfo(lastErrorMessage = message, lastError = error, - lastErrorTime = ssc.scheduler.clock.getTimeMillis()) - ReceiverTrackingInfo( - streamId, ReceiverState.INACTIVE, None, None, None, None, Some(errorInfo)) - } - - receiverTrackingInfos(streamId) = newReceiverTrackingInfo - listenerBus.post(StreamingListenerReceiverError(newReceiverTrackingInfo.toReceiverInfo)) - val messageWithError = if (error != null && !error.isEmpty) { - s"$message - $error" - } else { - s"$message" - } - logWarning(s"Error reported by receiver for stream $streamId: $messageWithError") - } - - private def updateReceiverScheduledLocations( - receiverId: Int, scheduledLocations: Seq[String]): Unit = { - val newReceiverTrackingInfo = receiverTrackingInfos.get(receiverId) match { - case Some(oldInfo) => - oldInfo.copy(state = ReceiverState.SCHEDULED, - scheduledLocations = Some(scheduledLocations)) - case None => - ReceiverTrackingInfo( - receiverId, - ReceiverState.SCHEDULED, - Some(scheduledLocations), - None) - } - receiverTrackingInfos.put(receiverId, newReceiverTrackingInfo) - } - - private def scheduleReceiver(receiverId: Int): Seq[String] = { - val preferredLocation = receiverPreferredLocations.getOrElse(receiverId, None) - val scheduledLocations = schedulingPolicy.scheduleReceiver( - receiverId, preferredLocation, receiverTrackingInfos, getExecutors(ssc)) - updateReceiverScheduledLocations(receiverId, scheduledLocations) - scheduledLocations - } - /** Send stop signal to the receivers. */ private def stopReceivers() { // Signal the receivers to stop From 276a4ac665a5cb637c59b3a52aca8d84adb71ca9 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 22 Jul 2015 16:14:53 +0800 Subject: [PATCH 18/27] Remove "ReceiverLauncher" and move codes to "launchReceivers" --- .../streaming/scheduler/ReceiverTracker.scala | 111 +++++++----------- 1 file changed, 40 insertions(+), 71 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 125535c70508..38737e50724a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.scheduler -import java.util.concurrent.CountDownLatch +import java.util.concurrent.{TimeUnit, CountDownLatch} import scala.collection.mutable.HashMap import scala.concurrent.ExecutionContext @@ -91,7 +91,6 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false private val receiverInputStreams = ssc.graph.getReceiverInputStreams() private val receiverInputStreamIds = receiverInputStreams.map { _.id } - private val receiverExecutor = new ReceiverLauncher() private val receivedBlockTracker = new ReceivedBlockTracker( ssc.sparkContext.conf, ssc.sparkContext.hadoopConfiguration, @@ -116,7 +115,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false // This not being null means the tracker has been started and not stopped private var endpoint: RpcEndpointRef = null - private val schedulingPolicy: ReceiverSchedulingPolicy = new ReceiverSchedulingPolicy() + private val schedulingPolicy = new ReceiverSchedulingPolicy() // Track the active receiver job number. When a receiver job exits ultimately, countDown will // be called. @@ -142,7 +141,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false if (!receiverInputStreams.isEmpty) { endpoint = ssc.env.rpcEnv.setupEndpoint( "ReceiverTracker", new ReceiverTrackerEndpoint(ssc.env.rpcEnv)) - if (!skipReceiverLaunch) receiverExecutor.start() + if (!skipReceiverLaunch) launchReceivers() logInfo("ReceiverTracker started") trackerState = Started } @@ -159,14 +158,11 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false // Wait for the Spark job that runs the receivers to be over // That is, for the receivers to quit gracefully. - receiverExecutor.awaitTermination(10000) + receiverJobExitLatch.await(10, TimeUnit.SECONDS) if (graceful) { - val pollTime = 100 logInfo("Waiting for receiver job to terminate gracefully") - while (receiverExecutor.running) { - Thread.sleep(pollTime) - } + receiverJobExitLatch.await() logInfo("Waited for receiver job to terminate gracefully") } @@ -347,6 +343,41 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false executors.diff(List(driver)) } + /** + * Run the dummy Spark job to ensure that all slaves have registered. This avoids all the + * receivers to be scheduled on the same node. + * + * TODO Should poll the executor number and wait for executors according to + * "spark.scheduler.minRegisteredResourcesRatio" and + * "spark.scheduler.maxRegisteredResourcesWaitingTime" rather than running a dummy job. + */ + private def runDummySparkJob(): Unit = { + if (!ssc.sparkContext.isLocal) { + ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect() + } + } + + /** + * Get the receivers from the ReceiverInputDStreams, distributes them to the + * worker nodes as a parallel collection, and runs them. + */ + private def launchReceivers(): Unit = { + val receivers = receiverInputStreams.map(nis => { + val rcvr = nis.getReceiver() + rcvr.setReceiverId(nis.id) + rcvr + }) + + runDummySparkJob() + + // Distribute the receivers and start them + logInfo("Starting " + receivers.length + " receivers") + + for (receiver <- receivers) { + endpoint.send(StartReceiver(receiver)) + } + } + /** Check if tracker has been marked for starting */ private def isTrackerStarted(): Boolean = trackerState == Started @@ -465,73 +496,11 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** Send stop signal to the receivers. */ private def stopReceivers() { - // Signal the receivers to stop receiverTrackingInfos.values.flatMap(_.endpoint).foreach { _.send(StopReceiver) } logInfo("Sent stop signal to all " + receiverTrackingInfos.size + " receivers") } } - /** This thread class runs all the receivers on the cluster. */ - class ReceiverLauncher { - @transient val env = ssc.env - @volatile @transient var running = false - @transient val thread = new Thread() { - override def run() { - try { - SparkEnv.set(env) - startReceivers() - } catch { - case ie: InterruptedException => logInfo("ReceiverLauncher interrupted") - } - } - } - - def start() { - thread.start() - } - - /** - * Get the receivers from the ReceiverInputDStreams, distributes them to the - * worker nodes as a parallel collection, and runs them. - */ - private def startReceivers() { - val receivers = receiverInputStreams.map(nis => { - val rcvr = nis.getReceiver() - rcvr.setReceiverId(nis.id) - rcvr - }) - - // Run the dummy Spark job to ensure that all slaves have registered. - // This avoids all the receivers to be scheduled on the same node. - if (!ssc.sparkContext.isLocal) { - ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect() - } - - // Distribute the receivers and start them - logInfo("Starting " + receivers.length + " receivers") - running = true - - try { - for (receiver <- receivers) { - endpoint.send(StartReceiver(receiver)) - } - // Wait until all receivers exit - receiverJobExitLatch.await() - logInfo("All of the receivers have been terminated") - } finally { - running = false - } - } - - /** - * Wait until the Spark job that runs the receivers is terminated, or return when - * `milliseconds` elapses - */ - def awaitTermination(milliseconds: Long): Unit = { - thread.join(milliseconds) - } - } - } /** From 075e0a31398eba05bf5340e85dfca18342aff064 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 22 Jul 2015 16:33:17 +0800 Subject: [PATCH 19/27] Add receiver RDD name; use '!isTrackerStarted' instead --- .../spark/streaming/scheduler/ReceiverTracker.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 38737e50724a..5d30b133d586 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -379,13 +379,13 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } /** Check if tracker has been marked for starting */ - private def isTrackerStarted(): Boolean = trackerState == Started + private def isTrackerStarted: Boolean = trackerState == Started /** Check if tracker has been marked for stopping */ - private def isTrackerStopping(): Boolean = trackerState == Stopping + private def isTrackerStopping: Boolean = trackerState == Stopping /** Check if tracker has been marked for stopped */ - private def isTrackerStopped(): Boolean = trackerState == Stopped + private def isTrackerStopped: Boolean = trackerState == Stopped /** RpcEndpoint to receive messages from the receivers. */ private class ReceiverTrackerEndpoint(override val rpcEnv: RpcEnv) extends ThreadSafeRpcEndpoint { @@ -436,7 +436,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false receiverPreferredLocations(receiver.streamId) = receiver.preferredLocation val receiverId = receiver.streamId - if (isTrackerStopping() || isTrackerStopped()) { + if (!isTrackerStarted) { onReceiverJobFinish(receiverId) return } @@ -455,19 +455,20 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } else { ssc.sc.makeRDD(Seq(receiver -> scheduledLocations)) } + receiverRDD.setName(s"Receiver $receiverId") val future = ssc.sparkContext.submitJob[Receiver[_], Unit, Unit]( receiverRDD, startReceiverFunc, Seq(0), (_, _) => Unit, ()) // We will keep restarting the receiver job until ReceiverTracker is stopped future.onComplete { case Success(_) => - if (isTrackerStopping() || isTrackerStopped()) { + if (!isTrackerStarted) { onReceiverJobFinish(receiverId) } else { logInfo(s"Restarting Receiver $receiverId") self.send(StartReceiver(receiver)) } case Failure(e) => - if (isTrackerStopping() || isTrackerStopped()) { + if (!isTrackerStarted) { onReceiverJobFinish(receiverId) } else { logError("Receiver has been stopped. Try to restart it.", e) From 59f88874d9c26329e6097ada1457539ad7b6c763 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 22 Jul 2015 22:33:27 +0800 Subject: [PATCH 20/27] Schedule all receivers at the same time when launching them --- .../scheduler/ReceiverSchedulingPolicy.scala | 73 +++++++++++++++- .../streaming/scheduler/ReceiverTracker.scala | 82 +++++++++++------- .../ReceiverSchedulingPolicySuite.scala | 86 ++++++++++++++++++- 3 files changed, 203 insertions(+), 38 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala index de6373c9e91f..f95a167d7262 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala @@ -20,6 +20,8 @@ package org.apache.spark.streaming.scheduler import scala.collection.Map import scala.collection.mutable +import org.apache.spark.streaming.receiver.Receiver + /** * A ReceiverScheduler trying to balance executors' load. Here is the approach to schedule executors * for a receiver. @@ -46,11 +48,80 @@ import scala.collection.mutable */ private[streaming] class ReceiverSchedulingPolicy { + /** + * Try our best to schedule receivers with evenly distributed. However, if the + * `preferredLocation`s of receivers are not even, we may not be able to schedule them evenly + * because we have to respect them. + * + * This method is called when we start to launch receivers at the first time. + */ + def scheduleReceivers( + receivers: Seq[Receiver[_]], executors: Seq[String]): Map[Int, Seq[String]] = { + if (receivers.isEmpty) { + return Map.empty + } + + require(executors.nonEmpty, "There is no executor up") + + val hostToExecutors = executors.groupBy(_.split(":")(0)) + val locations = new Array[mutable.ArrayBuffer[String]](receivers.length) + val numReceiversOnExecutor = mutable.HashMap[String, Int]() + // Set the initial value to 0 + executors.foreach(numReceiversOnExecutor(_) = 0) + + // Firstly, we need to respect "preferredLocation". So if a receiver has "preferredLocation", + // we need to make sure the "preferredLocation" is in the candidate location list. + for (i <- 0 until receivers.length) { + locations(i) = new mutable.ArrayBuffer[String]() + // Note: preferredLocation is host but executors are host:port + receivers(i).preferredLocation.foreach { host => + hostToExecutors.get(host) match { + case Some(executorsOnHost) => + // preferredLocation is a known host. Select an executor that has the least receivers in + // this host + val scheduledLocation = + executorsOnHost.minBy(executor => numReceiversOnExecutor(executor)) + locations(i) += scheduledLocation + numReceiversOnExecutor(scheduledLocation) = + numReceiversOnExecutor(scheduledLocation) + 1 + case None => + // preferredLocation is an unknown host. + // Note: There are two cases: + // 1. This executor is not up. But it may be up later. + // 2. This executor is dead, or it's not a host in the cluster. + // Currently, simply add host to the scheduled locations + locations(i) += host + } + } + } + + // For those receivers that don't have preferredLocation, make sure we assign at least one + // executor to them. + for (scheduledLocations <- locations.filter(_.isEmpty)) { + // Select the executor that has the least receivers + val (executor, numReceivers) = numReceiversOnExecutor.minBy(_._2) + scheduledLocations += executor + numReceiversOnExecutor(executor) = numReceivers + 1 + } + + // Assign idle executors to receivers that have less executors + val idleExecutors = numReceiversOnExecutor.filter(_._2 == 0).map(_._1) + for (executor <- idleExecutors) { + // Assign an idle executor to the receiver that has least locations. + val scheduledLocations = locations.minBy(_.size) + scheduledLocations += executor + } + + receivers.map(_.streamId).zip(locations).toMap + } + /** * Return a list of candidate executors to run the receiver. If the list is empty, the caller can * run this receiver in arbitrary executor. + * + * This method is called when a receiver is registering with ReceiverTracker or is restarting. */ - def scheduleReceiver( + def rescheduleReceiver( receiverId: Int, preferredLocation: Option[String], receiverTrackingInfoMap: Map[Int, ReceiverTrackingInfo], diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 5d30b133d586..74f9a1211f6d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -62,9 +62,16 @@ private[streaming] case class DeregisterReceiver(streamId: Int, msg: String, err private[streaming] sealed trait ReceiverTrackerLocalMessage /** - * This message will trigger ReceiverTrackerEndpoint to start a Spark job for the receiver. + * This message will trigger ReceiverTrackerEndpoint to restart a Spark job for the receiver. */ -private[streaming] case class StartReceiver(receiver: Receiver[_]) +private[streaming] case class RestartReceiver(receiver: Receiver[_]) + extends ReceiverTrackerLocalMessage + +/** + * This message is sent to ReceiverTrackerEndpoint when we start to launch Spark jobs for receivers + * at the first time. + */ +private[streaming] case class StartAllReceivers(receiver: Seq[Receiver[_]]) extends ReceiverTrackerLocalMessage /** @@ -307,8 +314,8 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false private def scheduleReceiver(receiverId: Int): Seq[String] = { val preferredLocation = receiverPreferredLocations.getOrElse(receiverId, None) - val scheduledLocations = schedulingPolicy.scheduleReceiver( - receiverId, preferredLocation, receiverTrackingInfos, getExecutors(ssc)) + val scheduledLocations = schedulingPolicy.rescheduleReceiver( + receiverId, preferredLocation, receiverTrackingInfos, getExecutors) updateReceiverScheduledLocations(receiverId, scheduledLocations) scheduledLocations } @@ -337,10 +344,14 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** * Get the list of executors excluding driver */ - private def getExecutors(ssc: StreamingContext): List[String] = { - val executors = ssc.sparkContext.getExecutorMemoryStatus.map(_._1.split(":")(0)).toList - val driver = ssc.sparkContext.getConf.get("spark.driver.host") - executors.diff(List(driver)) + private def getExecutors: List[String] = { + if (ssc.sc.isLocal) { + List("localhost") + } else { + val executors = ssc.sparkContext.getExecutorMemoryStatus.map(_._1.split(":")(0)).toList + val driver = ssc.sparkContext.getConf.get("spark.driver.host") + executors.diff(List(driver)) + } } /** @@ -355,6 +366,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false if (!ssc.sparkContext.isLocal) { ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect() } + assert(getExecutors.nonEmpty) } /** @@ -370,12 +382,8 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false runDummySparkJob() - // Distribute the receivers and start them logInfo("Starting " + receivers.length + " receivers") - - for (receiver <- receivers) { - endpoint.send(StartReceiver(receiver)) - } + endpoint.send(StartAllReceivers) } /** Check if tracker has been marked for starting */ @@ -396,8 +404,22 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false override def receive: PartialFunction[Any, Unit] = { // Local messages - case StartReceiver(receiver) => - startReceiver(receiver) + case StartAllReceivers(receivers) => + val scheduledLocations = schedulingPolicy.scheduleReceivers(receivers, getExecutors) + for (receiver <- receivers) { + val locations = scheduledLocations(receiver.streamId) + updateReceiverScheduledLocations(receiver.streamId, locations) + receiverPreferredLocations(receiver.streamId) = receiver.preferredLocation + startReceiver(receiver, locations) + } + case RestartReceiver(receiver) => + val scheduledLocations = schedulingPolicy.rescheduleReceiver( + receiver.streamId, + receiver.preferredLocation, + receiverTrackingInfos, + getExecutors) + updateReceiverScheduledLocations(receiver.streamId, scheduledLocations) + startReceiver(receiver, scheduledLocations) case c @ CleanupOldBlocks(cleanupThreshTime) => receiverTrackingInfos.values.flatMap(_.endpoint).foreach(_.send(c)) // Remote messages @@ -425,28 +447,22 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false context.reply(true) } - private def startReceiver(receiver: Receiver[_]): Unit = { - val checkpointDirOption = Option(ssc.checkpointDir) - val serializableHadoopConf = - new SerializableConfiguration(ssc.sparkContext.hadoopConfiguration) - - // Function to start the receiver on the worker node - val startReceiverFunc = new StartReceiverFunc(checkpointDirOption, serializableHadoopConf) - - receiverPreferredLocations(receiver.streamId) = receiver.preferredLocation + /** + * Start a receiver along with its scheduled locations + */ + private def startReceiver(receiver: Receiver[_], scheduledLocations: Seq[String]): Unit = { val receiverId = receiver.streamId - if (!isTrackerStarted) { onReceiverJobFinish(receiverId) return } - val scheduledLocations = schedulingPolicy.scheduleReceiver( - receiverId, - receiver.preferredLocation, - receiverTrackingInfos, - getExecutors(ssc)) - updateReceiverScheduledLocations(receiver.streamId, scheduledLocations) + val checkpointDirOption = Option(ssc.checkpointDir) + val serializableHadoopConf = + new SerializableConfiguration(ssc.sparkContext.hadoopConfiguration) + + // Function to start the receiver on the worker node + val startReceiverFunc = new StartReceiverFunc(checkpointDirOption, serializableHadoopConf) // Create the RDD using the scheduledLocations to run the receiver in a Spark job val receiverRDD: RDD[Receiver[_]] = @@ -465,7 +481,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false onReceiverJobFinish(receiverId) } else { logInfo(s"Restarting Receiver $receiverId") - self.send(StartReceiver(receiver)) + self.send(RestartReceiver(receiver)) } case Failure(e) => if (!isTrackerStarted) { @@ -473,7 +489,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } else { logError("Receiver has been stopped. Try to restart it.", e) logInfo(s"Restarting Receiver $receiverId") - self.send(StartReceiver(receiver)) + self.send(RestartReceiver(receiver)) } }(submitJobThreadPool) logInfo(s"Receiver ${receiver.streamId} started") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala index a56267eb762c..280835c35d6c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala @@ -17,7 +17,11 @@ package org.apache.spark.streaming.scheduler +import scala.collection.mutable + import org.apache.spark.SparkFunSuite +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.receiver.Receiver class ReceiverSchedulingPolicySuite extends SparkFunSuite { @@ -25,14 +29,14 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { test("empty executors") { val scheduledLocations = - receiverSchedulingPolicy.scheduleReceiver(0, None, Map.empty, executors = Seq.empty) + receiverSchedulingPolicy.rescheduleReceiver(0, None, Map.empty, executors = Seq.empty) assert(scheduledLocations === Seq.empty) } test("receiver preferredLocation") { val receiverTrackingInfoMap = Map( 0 -> ReceiverTrackingInfo(0, ReceiverState.INACTIVE, None, None)) - val scheduledLocations = receiverSchedulingPolicy.scheduleReceiver( + val scheduledLocations = receiverSchedulingPolicy.rescheduleReceiver( 0, Some("host1"), receiverTrackingInfoMap, executors = Seq("host2")) assert(scheduledLocations.toSet === Set("host1", "host2")) } @@ -42,7 +46,7 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { // host3 is idle val receiverTrackingInfoMap = Map( 0 -> ReceiverTrackingInfo(0, ReceiverState.ACTIVE, None, Some("host1"))) - val scheduledLocations = receiverSchedulingPolicy.scheduleReceiver( + val scheduledLocations = receiverSchedulingPolicy.rescheduleReceiver( 1, None, receiverTrackingInfoMap, executors) assert(scheduledLocations.toSet === Set("host2", "host3", "host4", "host5")) } @@ -55,8 +59,82 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { 0 -> ReceiverTrackingInfo(0, ReceiverState.ACTIVE, None, Some("host1")), 1 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host2", "host3")), None), 2 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host1", "host3")), None)) - val scheduledLocations = receiverSchedulingPolicy.scheduleReceiver( + val scheduledLocations = receiverSchedulingPolicy.rescheduleReceiver( 3, None, receiverTrackingInfoMap, executors) assert(scheduledLocations.toSet === Set("host2", "host4", "host5")) } + + test("scheduleReceivers should schedule receivers evenly " + + "when there are more receivers than executors") { + val receivers = (0 until 6).map(new DummyReceiver(_)) + val executors = (10000 until 10003).map(port => s"localhost:${port}") + val scheduledLocations = receiverSchedulingPolicy.scheduleReceivers(receivers, executors) + val numReceiversOnExecutor = mutable.HashMap[String, Int]() + // There should be 2 receivers running on each executor and each receiver has one location + scheduledLocations.foreach { case (receiverId, locations) => + assert(locations.size == 1) + numReceiversOnExecutor(locations(0)) = numReceiversOnExecutor.getOrElse(locations(0), 0) + 1 + } + assert(numReceiversOnExecutor === executors.map(_ -> 2).toMap) + } + + + test("scheduleReceivers should schedule receivers evenly " + + "when there are more executors than receivers") { + val receivers = (0 until 3).map(new DummyReceiver(_)) + val executors = (10000 until 10006).map(port => s"localhost:${port}") + val scheduledLocations = receiverSchedulingPolicy.scheduleReceivers(receivers, executors) + val numReceiversOnExecutor = mutable.HashMap[String, Int]() + // There should be 1 receiver running on each executor and each receiver has two locations + scheduledLocations.foreach { case (receiverId, locations) => + assert(locations.size == 2) + locations.foreach { l => + numReceiversOnExecutor(l) = numReceiversOnExecutor.getOrElse(l, 0) + 1 + } + } + assert(numReceiversOnExecutor === executors.map(_ -> 1).toMap) + } + + test("scheduleReceivers should schedule receivers evenly " + + "when the preferredLocations are even") { + val receivers = (0 until 3).map(new DummyReceiver(_)) ++ + (3 until 6).map(new DummyReceiver(_, Some("localhost"))) + val executors = (10000 until 10003).map(port => s"localhost:${port}") ++ + (10003 until 10006).map(port => s"localhost2:${port}") + val scheduledLocations = receiverSchedulingPolicy.scheduleReceivers(receivers, executors) + val numReceiversOnExecutor = mutable.HashMap[String, Int]() + // There should be 1 receiver running on each executor and each receiver has 1 location + scheduledLocations.foreach { case (receiverId, locations) => + assert(locations.size == 1) + locations.foreach { l => + numReceiversOnExecutor(l) = numReceiversOnExecutor.getOrElse(l, 0) + 1 + } + } + assert(numReceiversOnExecutor === executors.map(_ -> 1).toMap) + // Make sure we schedule the receivers to their preferredLocations + val locationsForReceiversWithPreferredLocation = + scheduledLocations.filter { case (receiverId, locations) => receiverId >= 3 }.flatMap(_._2) + // We can simply check the location set because we only know each receiver only has 1 location + assert(locationsForReceiversWithPreferredLocation.toSet === + (10000 until 10003).map(port => s"localhost:${port}").toSet) + } + + test("scheduleReceivers should return empty if no receiver") { + assert(receiverSchedulingPolicy.scheduleReceivers(Seq.empty, Seq("localhost:10000")).isEmpty) + } +} + +/** + * Dummy receiver implementation + */ +private class DummyReceiver(receiverId: Int, host: Option[String] = None) + extends Receiver[Int](StorageLevel.MEMORY_ONLY) { + + setReceiverId(receiverId) + + override def onStart(): Unit = {} + + override def onStop(): Unit = {} + + override def preferredLocation: Option[String] = host } From 8f93c8d1aec224a279c7f247d95e708072063d66 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 22 Jul 2015 23:14:54 +0800 Subject: [PATCH 21/27] Use hostPort as the receiver location rather than host; fix comments and unit tests --- .../receiver/ReceiverSupervisor.scala | 4 +- .../receiver/ReceiverSupervisorImpl.scala | 4 +- .../scheduler/ReceiverSchedulingPolicy.scala | 50 +++++++++---------- .../streaming/scheduler/ReceiverTracker.scala | 32 ++++++------ .../ReceiverSchedulingPolicySuite.scala | 29 +++++++---- 5 files changed, 63 insertions(+), 56 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index 9184c8fbcc5f..a6e5ab6e53a0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import scala.concurrent._ import scala.util.control.NonFatal -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{SparkEnv, Logging, SparkConf} import org.apache.spark.storage.StreamBlockId import org.apache.spark.util.{Utils, ThreadUtils} @@ -65,8 +65,6 @@ private[streaming] abstract class ReceiverSupervisor( /** State of the receiver */ @volatile private[streaming] var receiverState = Initialized - protected val host = Utils.localHostName() - /** Push a single data item to backend data store. */ def pushSingle(data: Any) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 3ee648ad9ad3..e0504cc584cf 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -46,6 +46,8 @@ private[streaming] class ReceiverSupervisorImpl( checkpointDirOption: Option[String] ) extends ReceiverSupervisor(receiver, env.conf) with Logging { + private val hostPort = SparkEnv.get.blockManager.blockManagerId.hostPort + private val receivedBlockHandler: ReceivedBlockHandler = { if (WriteAheadLogUtils.enableReceiverLog(env.conf)) { if (checkpointDirOption.isEmpty) { @@ -164,7 +166,7 @@ private[streaming] class ReceiverSupervisorImpl( override protected def onReceiverStart(): Boolean = { val msg = RegisterReceiver( - streamId, receiver.getClass.getSimpleName, host, endpoint) + streamId, receiver.getClass.getSimpleName, hostPort, endpoint) trackerEndpoint.askWithRetry[Boolean](msg) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala index f95a167d7262..da8109ee6623 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala @@ -22,30 +22,6 @@ import scala.collection.mutable import org.apache.spark.streaming.receiver.Receiver -/** - * A ReceiverScheduler trying to balance executors' load. Here is the approach to schedule executors - * for a receiver. - *
    - *
  1. - * If preferredLocation is set, preferredLocation should be one of the candidate executors. - *
  2. - *
  3. - * Every executor will be assigned to a weight according to the receivers running or scheduling - * on it. - *
      - *
    • - * If a receiver is running on an executor, it contributes 1.0 to the executor's weight. - *
    • - *
    • - * If a receiver is scheduled to an executor but has not yet run, it contributes - * `1.0 / #candidate_executors_of_this_receiver` to the executor's weight.
    • - *
    - * At last, if there are more than 3 idle executors (weight = 0), returns all idle executors. - * Otherwise, we only return 3 best options according to the weights. - *
  4. - *
- * - */ private[streaming] class ReceiverSchedulingPolicy { /** @@ -61,7 +37,9 @@ private[streaming] class ReceiverSchedulingPolicy { return Map.empty } - require(executors.nonEmpty, "There is no executor up") + if (executors.isEmpty) { + return receivers.map(_.streamId -> Seq.empty).toMap + } val hostToExecutors = executors.groupBy(_.split(":")(0)) val locations = new Array[mutable.ArrayBuffer[String]](receivers.length) @@ -119,6 +97,28 @@ private[streaming] class ReceiverSchedulingPolicy { * Return a list of candidate executors to run the receiver. If the list is empty, the caller can * run this receiver in arbitrary executor. * + * This method tries to balance executors' load. Here is the approach to schedule executors + * for a receiver. + *
    + *
  1. + * If preferredLocation is set, preferredLocation should be one of the candidate executors. + *
  2. + *
  3. + * Every executor will be assigned to a weight according to the receivers running or + * scheduling on it. + *
      + *
    • + * If a receiver is running on an executor, it contributes 1.0 to the executor's weight. + *
    • + *
    • + * If a receiver is scheduled to an executor but has not yet run, it contributes + * `1.0 / #candidate_executors_of_this_receiver` to the executor's weight.
    • + *
    + * At last, if there are more than 3 idle executors (weight = 0), returns all idle executors. + * Otherwise, we only return 3 best options according to the weights. + *
  4. + *
+ * * This method is called when a receiver is registering with ReceiverTracker or is restarting. */ def rescheduleReceiver( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 74f9a1211f6d..0ef01942d10d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -25,7 +25,7 @@ import scala.language.existentials import scala.util.{Failure, Success} import org.apache.spark.streaming.util.WriteAheadLogUtils -import org.apache.spark.{TaskContext, Logging, SparkEnv, SparkException} +import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.rpc._ import org.apache.spark.streaming.{StreamingContext, Time} @@ -47,7 +47,7 @@ private[streaming] sealed trait ReceiverTrackerMessage private[streaming] case class RegisterReceiver( streamId: Int, typ: String, - host: String, + hostPort: String, receiverEndpoint: RpcEndpointRef ) extends ReceiverTrackerMessage private[streaming] case class AddBlock(receivedBlockInfo: ReceivedBlockInfo) @@ -130,12 +130,13 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** * Track all receivers' information. The key is the receiver id, the value is the receiver info. + * It's only accessed in ReceiverTrackerEndpoint. */ private val receiverTrackingInfos = new HashMap[Int, ReceiverTrackingInfo] /** * Store all preferred locations for all receivers. We need this information to schedule - * receivers + * receivers. It's only accessed in ReceiverTrackerEndpoint. */ private val receiverPreferredLocations = new HashMap[Int, Option[String]] @@ -227,7 +228,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false private def registerReceiver( streamId: Int, typ: String, - host: String, + hostPort: String, receiverEndpoint: RpcEndpointRef, senderAddress: RpcAddress ): Boolean = { @@ -237,19 +238,18 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false if (isTrackerStopping || isTrackerStopped) { false - } else if (!ssc.sparkContext.isLocal && // We don't need to schedule it in the local mode - !scheduleReceiver(streamId).contains(host)) { + } else if (!scheduleReceiver(streamId).contains(hostPort)) { // Refuse it since it's scheduled to a wrong executor false } else { val name = s"${typ}-${streamId}" - val receiverInfo = ReceiverInfo(streamId, name, true, host) + val receiverInfo = ReceiverInfo(streamId, name, true, hostPort) receiverTrackingInfos.put(streamId, ReceiverTrackingInfo( streamId, ReceiverState.ACTIVE, scheduledLocations = None, - runningLocation = Some(host), + runningLocation = Some(hostPort), name = Some(name), endpoint = Some(receiverEndpoint))) listenerBus.post(StreamingListenerReceiverStarted(receiverInfo)) @@ -344,13 +344,13 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** * Get the list of executors excluding driver */ - private def getExecutors: List[String] = { + private def getExecutors: Seq[String] = { if (ssc.sc.isLocal) { - List("localhost") + Seq(ssc.sparkContext.env.blockManager.blockManagerId.hostPort) } else { - val executors = ssc.sparkContext.getExecutorMemoryStatus.map(_._1.split(":")(0)).toList - val driver = ssc.sparkContext.getConf.get("spark.driver.host") - executors.diff(List(driver)) + ssc.sparkContext.env.blockManager.master.getMemoryStatus.filter { case (blockManagerId, _) => + blockManagerId.executorId != SparkContext.DRIVER_IDENTIFIER // Ignore the driver location + }.map { case (blockManagerId, _) => blockManagerId.hostPort }.toSeq } } @@ -383,7 +383,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false runDummySparkJob() logInfo("Starting " + receivers.length + " receivers") - endpoint.send(StartAllReceivers) + endpoint.send(StartAllReceivers(receivers)) } /** Check if tracker has been marked for starting */ @@ -429,9 +429,9 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { // Remote messages - case RegisterReceiver(streamId, typ, host, receiverEndpoint) => + case RegisterReceiver(streamId, typ, hostPort, receiverEndpoint) => val successful = - registerReceiver(streamId, typ, host, receiverEndpoint, context.sender.address) + registerReceiver(streamId, typ, hostPort, receiverEndpoint, context.sender.address) context.reply(successful) case AddBlock(receivedBlockInfo) => context.reply(addBlock(receivedBlockInfo)) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala index 280835c35d6c..fe7064689b1f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala @@ -27,13 +27,13 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { val receiverSchedulingPolicy = new ReceiverSchedulingPolicy - test("empty executors") { + test("rescheduleReceiver: empty executors") { val scheduledLocations = receiverSchedulingPolicy.rescheduleReceiver(0, None, Map.empty, executors = Seq.empty) assert(scheduledLocations === Seq.empty) } - test("receiver preferredLocation") { + test("rescheduleReceiver: receiver preferredLocation") { val receiverTrackingInfoMap = Map( 0 -> ReceiverTrackingInfo(0, ReceiverState.INACTIVE, None, None)) val scheduledLocations = receiverSchedulingPolicy.rescheduleReceiver( @@ -41,7 +41,7 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { assert(scheduledLocations.toSet === Set("host1", "host2")) } - test("return all idle executors if more than 3 idle executors") { + test("rescheduleReceiver: return all idle executors if more than 3 idle executors") { val executors = Seq("host1", "host2", "host3", "host4", "host5") // host3 is idle val receiverTrackingInfoMap = Map( @@ -51,7 +51,7 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { assert(scheduledLocations.toSet === Set("host2", "host3", "host4", "host5")) } - test("return 3 best options if less than 3 idle executors") { + test("rescheduleReceiver: return 3 best options if less than 3 idle executors") { val executors = Seq("host1", "host2", "host3", "host4", "host5") // Weights: host1 = 1.5, host2 = 0.5, host3 = 1.0 // host4 and host5 are idle @@ -64,8 +64,8 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { assert(scheduledLocations.toSet === Set("host2", "host4", "host5")) } - test("scheduleReceivers should schedule receivers evenly " + - "when there are more receivers than executors") { + test("scheduleReceivers: " + + "schedule receivers evenly when there are more receivers than executors") { val receivers = (0 until 6).map(new DummyReceiver(_)) val executors = (10000 until 10003).map(port => s"localhost:${port}") val scheduledLocations = receiverSchedulingPolicy.scheduleReceivers(receivers, executors) @@ -79,8 +79,8 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { } - test("scheduleReceivers should schedule receivers evenly " + - "when there are more executors than receivers") { + test("scheduleReceivers: " + + "schedule receivers evenly when there are more executors than receivers") { val receivers = (0 until 3).map(new DummyReceiver(_)) val executors = (10000 until 10006).map(port => s"localhost:${port}") val scheduledLocations = receiverSchedulingPolicy.scheduleReceivers(receivers, executors) @@ -95,8 +95,7 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { assert(numReceiversOnExecutor === executors.map(_ -> 1).toMap) } - test("scheduleReceivers should schedule receivers evenly " + - "when the preferredLocations are even") { + test("scheduleReceivers: schedule receivers evenly when the preferredLocations are even") { val receivers = (0 until 3).map(new DummyReceiver(_)) ++ (3 until 6).map(new DummyReceiver(_, Some("localhost"))) val executors = (10000 until 10003).map(port => s"localhost:${port}") ++ @@ -119,9 +118,17 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { (10000 until 10003).map(port => s"localhost:${port}").toSet) } - test("scheduleReceivers should return empty if no receiver") { + test("scheduleReceivers: return empty if no receiver") { assert(receiverSchedulingPolicy.scheduleReceivers(Seq.empty, Seq("localhost:10000")).isEmpty) } + + test("scheduleReceivers: return empty locations if no executors") { + val receivers = (0 until 3).map(new DummyReceiver(_)) + val scheduledLocations = receiverSchedulingPolicy.scheduleReceivers(receivers, Seq.empty) + scheduledLocations.foreach { case (receiverId, locations) => + assert(locations.isEmpty) + } + } } /** From 05daf9c829c6c17d1e7ebd4042e687c6627b0b90 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 23 Jul 2015 09:15:55 +0800 Subject: [PATCH 22/27] Use receiverTrackingInfo.toReceiverInfo --- .../streaming/scheduler/ReceiverTracker.scala | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 6b5a53c9021b..301f26b72cf9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -246,16 +246,15 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false false } else { val name = s"${typ}-${streamId}" - val receiverInfo = ReceiverInfo(streamId, name, true, hostPort) - receiverTrackingInfos.put(streamId, - ReceiverTrackingInfo( - streamId, - ReceiverState.ACTIVE, - scheduledLocations = None, - runningLocation = Some(hostPort), - name = Some(name), - endpoint = Some(receiverEndpoint))) - listenerBus.post(StreamingListenerReceiverStarted(receiverInfo)) + val receiverTrackingInfo = ReceiverTrackingInfo( + streamId, + ReceiverState.ACTIVE, + scheduledLocations = None, + runningLocation = Some(hostPort), + name = Some(name), + endpoint = Some(receiverEndpoint)) + receiverTrackingInfos.put(streamId, receiverTrackingInfo) + listenerBus.post(StreamingListenerReceiverStarted(receiverTrackingInfo.toReceiverInfo)) logInfo("Registered receiver for stream " + streamId + " from " + senderAddress) true } From 715ef9caf7cef85022a09062ae68cd2f5cbf7e73 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 23 Jul 2015 16:11:18 +0800 Subject: [PATCH 23/27] Rename: scheduledLocations -> scheduledExecutors; locations -> executors --- .../scheduler/ReceiverSchedulingPolicy.scala | 51 ++++++++------- .../streaming/scheduler/ReceiverTracker.scala | 42 ++++++------- .../scheduler/ReceiverTrackingInfo.scala | 10 +-- .../ReceiverSchedulingPolicySuite.scala | 62 +++++++++---------- 4 files changed, 82 insertions(+), 83 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala index da8109ee6623..0a51a7048f33 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala @@ -42,55 +42,54 @@ private[streaming] class ReceiverSchedulingPolicy { } val hostToExecutors = executors.groupBy(_.split(":")(0)) - val locations = new Array[mutable.ArrayBuffer[String]](receivers.length) + val scheduledExecutors = Array.fill(receivers.length)(new mutable.ArrayBuffer[String]) val numReceiversOnExecutor = mutable.HashMap[String, Int]() // Set the initial value to 0 executors.foreach(numReceiversOnExecutor(_) = 0) // Firstly, we need to respect "preferredLocation". So if a receiver has "preferredLocation", - // we need to make sure the "preferredLocation" is in the candidate location list. + // we need to make sure the "preferredLocation" is in the candidate scheduled executor list. for (i <- 0 until receivers.length) { - locations(i) = new mutable.ArrayBuffer[String]() // Note: preferredLocation is host but executors are host:port receivers(i).preferredLocation.foreach { host => hostToExecutors.get(host) match { case Some(executorsOnHost) => // preferredLocation is a known host. Select an executor that has the least receivers in // this host - val scheduledLocation = + val leastScheduledExecutor = executorsOnHost.minBy(executor => numReceiversOnExecutor(executor)) - locations(i) += scheduledLocation - numReceiversOnExecutor(scheduledLocation) = - numReceiversOnExecutor(scheduledLocation) + 1 + scheduledExecutors(i) += leastScheduledExecutor + numReceiversOnExecutor(leastScheduledExecutor) = + numReceiversOnExecutor(leastScheduledExecutor) + 1 case None => // preferredLocation is an unknown host. // Note: There are two cases: // 1. This executor is not up. But it may be up later. // 2. This executor is dead, or it's not a host in the cluster. - // Currently, simply add host to the scheduled locations - locations(i) += host + // Currently, simply add host to the scheduled executors. + scheduledExecutors(i) += host } } } // For those receivers that don't have preferredLocation, make sure we assign at least one // executor to them. - for (scheduledLocations <- locations.filter(_.isEmpty)) { + for (scheduledExecutorsForOneReceiver <- scheduledExecutors.filter(_.isEmpty)) { // Select the executor that has the least receivers - val (executor, numReceivers) = numReceiversOnExecutor.minBy(_._2) - scheduledLocations += executor - numReceiversOnExecutor(executor) = numReceivers + 1 + val (leastScheduledExecutor, numReceivers) = numReceiversOnExecutor.minBy(_._2) + scheduledExecutorsForOneReceiver += leastScheduledExecutor + numReceiversOnExecutor(leastScheduledExecutor) = numReceivers + 1 } // Assign idle executors to receivers that have less executors val idleExecutors = numReceiversOnExecutor.filter(_._2 == 0).map(_._1) for (executor <- idleExecutors) { - // Assign an idle executor to the receiver that has least locations. - val scheduledLocations = locations.minBy(_.size) - scheduledLocations += executor + // Assign an idle executor to the receiver that has least candidate executors. + val leastScheduledExecutors = scheduledExecutors.minBy(_.size) + leastScheduledExecutors += executor } - receivers.map(_.streamId).zip(locations).toMap + receivers.map(_.streamId).zip(scheduledExecutors).toMap } /** @@ -131,31 +130,31 @@ private[streaming] class ReceiverSchedulingPolicy { } // Always try to schedule to the preferred locations - val locations = mutable.Set[String]() - locations ++= preferredLocation + val scheduledExecutors = mutable.Set[String]() + scheduledExecutors ++= preferredLocation val executorWeights = receiverTrackingInfoMap.values.flatMap { receiverTrackingInfo => receiverTrackingInfo.state match { case ReceiverState.INACTIVE => Nil case ReceiverState.SCHEDULED => - val scheduledLocations = receiverTrackingInfo.scheduledLocations.get + val scheduledExecutors = receiverTrackingInfo.scheduledExecutors.get // The probability that a scheduled receiver will run in an executor is // 1.0 / scheduledLocations.size - scheduledLocations.map(location => location -> (1.0 / scheduledLocations.size)) - case ReceiverState.ACTIVE => Seq(receiverTrackingInfo.runningLocation.get -> 1.0) + scheduledExecutors.map(location => location -> (1.0 / scheduledExecutors.size)) + case ReceiverState.ACTIVE => Seq(receiverTrackingInfo.runningExecutor.get -> 1.0) } }.groupBy(_._1).mapValues(_.map(_._2).sum) // Sum weights for each executor val idleExecutors = (executors.toSet -- executorWeights.keys).toSeq if (idleExecutors.size >= 3) { // If there are more than 3 idle executors, return all of them - locations ++= idleExecutors + scheduledExecutors ++= idleExecutors } else { // If there are less than 3 idle executors, return 3 best options - locations ++= idleExecutors + scheduledExecutors ++= idleExecutors val sortedExecutors = executorWeights.toSeq.sortBy(_._2).map(_._1) - locations ++= (idleExecutors ++ sortedExecutors).take(3) + scheduledExecutors ++= (idleExecutors ++ sortedExecutors).take(3) } - locations.toSeq + scheduledExecutors.toSeq } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 301f26b72cf9..c14aa3d529bc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -249,8 +249,8 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false val receiverTrackingInfo = ReceiverTrackingInfo( streamId, ReceiverState.ACTIVE, - scheduledLocations = None, - runningLocation = Some(hostPort), + scheduledExecutors = None, + runningExecutor = Some(hostPort), name = Some(name), endpoint = Some(receiverEndpoint)) receiverTrackingInfos.put(streamId, receiverTrackingInfo) @@ -321,23 +321,23 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false private def scheduleReceiver(receiverId: Int): Seq[String] = { val preferredLocation = receiverPreferredLocations.getOrElse(receiverId, None) - val scheduledLocations = schedulingPolicy.rescheduleReceiver( + val scheduledExecutors = schedulingPolicy.rescheduleReceiver( receiverId, preferredLocation, receiverTrackingInfos, getExecutors) - updateReceiverScheduledLocations(receiverId, scheduledLocations) - scheduledLocations + updateReceiverScheduledExecutors(receiverId, scheduledExecutors) + scheduledExecutors } - private def updateReceiverScheduledLocations( - receiverId: Int, scheduledLocations: Seq[String]): Unit = { + private def updateReceiverScheduledExecutors( + receiverId: Int, scheduledExecutors: Seq[String]): Unit = { val newReceiverTrackingInfo = receiverTrackingInfos.get(receiverId) match { case Some(oldInfo) => oldInfo.copy(state = ReceiverState.SCHEDULED, - scheduledLocations = Some(scheduledLocations)) + scheduledExecutors = Some(scheduledExecutors)) case None => ReceiverTrackingInfo( receiverId, ReceiverState.SCHEDULED, - Some(scheduledLocations), + Some(scheduledExecutors), None) } receiverTrackingInfos.put(receiverId, newReceiverTrackingInfo) @@ -412,21 +412,21 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false override def receive: PartialFunction[Any, Unit] = { // Local messages case StartAllReceivers(receivers) => - val scheduledLocations = schedulingPolicy.scheduleReceivers(receivers, getExecutors) + val scheduledExecutors = schedulingPolicy.scheduleReceivers(receivers, getExecutors) for (receiver <- receivers) { - val locations = scheduledLocations(receiver.streamId) - updateReceiverScheduledLocations(receiver.streamId, locations) + val executors = scheduledExecutors(receiver.streamId) + updateReceiverScheduledExecutors(receiver.streamId, executors) receiverPreferredLocations(receiver.streamId) = receiver.preferredLocation - startReceiver(receiver, locations) + startReceiver(receiver, executors) } case RestartReceiver(receiver) => - val scheduledLocations = schedulingPolicy.rescheduleReceiver( + val scheduledExecutors = schedulingPolicy.rescheduleReceiver( receiver.streamId, receiver.preferredLocation, receiverTrackingInfos, getExecutors) - updateReceiverScheduledLocations(receiver.streamId, scheduledLocations) - startReceiver(receiver, scheduledLocations) + updateReceiverScheduledExecutors(receiver.streamId, scheduledExecutors) + startReceiver(receiver, scheduledExecutors) case c: CleanupOldBlocks => receiverTrackingInfos.values.flatMap(_.endpoint).foreach(_.send(c)) case UpdateReceiverRateLimit(streamUID, newRate) => @@ -459,9 +459,9 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } /** - * Start a receiver along with its scheduled locations + * Start a receiver along with its scheduled executors */ - private def startReceiver(receiver: Receiver[_], scheduledLocations: Seq[String]): Unit = { + private def startReceiver(receiver: Receiver[_], scheduledExecutors: Seq[String]): Unit = { val receiverId = receiver.streamId if (!isTrackerStarted) { onReceiverJobFinish(receiverId) @@ -475,12 +475,12 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false // Function to start the receiver on the worker node val startReceiverFunc = new StartReceiverFunc(checkpointDirOption, serializableHadoopConf) - // Create the RDD using the scheduledLocations to run the receiver in a Spark job + // Create the RDD using the scheduledExecutors to run the receiver in a Spark job val receiverRDD: RDD[Receiver[_]] = - if (scheduledLocations.isEmpty) { + if (scheduledExecutors.isEmpty) { ssc.sc.makeRDD(Seq(receiver), 1) } else { - ssc.sc.makeRDD(Seq(receiver -> scheduledLocations)) + ssc.sc.makeRDD(Seq(receiver -> scheduledExecutors)) } receiverRDD.setName(s"Receiver $receiverId") val future = ssc.sparkContext.submitJob[Receiver[_], Unit, Unit]( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTrackingInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTrackingInfo.scala index 75afc1ad42b5..043ff4d0ff05 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTrackingInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTrackingInfo.scala @@ -28,8 +28,8 @@ private[streaming] case class ReceiverErrorInfo( * * @param receiverId the unique receiver id * @param state the current Receiver state - * @param scheduledLocations the scheduled locations provided by ReceiverSchedulingPolicy - * @param runningLocation the running location if the receiver is active + * @param scheduledExecutors the scheduled executors provided by ReceiverSchedulingPolicy + * @param runningExecutor the running executor if the receiver is active * @param name the receiver name * @param endpoint the receiver endpoint. It can be used to send messages to the receiver * @param errorInfo the receiver error information if it fails @@ -37,8 +37,8 @@ private[streaming] case class ReceiverErrorInfo( private[streaming] case class ReceiverTrackingInfo( receiverId: Int, state: ReceiverState, - scheduledLocations: Option[Seq[String]], - runningLocation: Option[String], + scheduledExecutors: Option[Seq[String]], + runningExecutor: Option[String], name: Option[String] = None, endpoint: Option[RpcEndpointRef] = None, errorInfo: Option[ReceiverErrorInfo] = None) { @@ -47,7 +47,7 @@ private[streaming] case class ReceiverTrackingInfo( receiverId, name.getOrElse(""), state == ReceiverState.ACTIVE, - location = runningLocation.getOrElse(""), + location = runningExecutor.getOrElse(""), lastErrorMessage = errorInfo.map(_.lastErrorMessage).getOrElse(""), lastError = errorInfo.map(_.lastError).getOrElse(""), lastErrorTime = errorInfo.map(_.lastErrorTime).getOrElse(-1L) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala index fe7064689b1f..a03cc8f5f0c4 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala @@ -28,17 +28,17 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { val receiverSchedulingPolicy = new ReceiverSchedulingPolicy test("rescheduleReceiver: empty executors") { - val scheduledLocations = + val scheduledExecutors = receiverSchedulingPolicy.rescheduleReceiver(0, None, Map.empty, executors = Seq.empty) - assert(scheduledLocations === Seq.empty) + assert(scheduledExecutors === Seq.empty) } test("rescheduleReceiver: receiver preferredLocation") { val receiverTrackingInfoMap = Map( 0 -> ReceiverTrackingInfo(0, ReceiverState.INACTIVE, None, None)) - val scheduledLocations = receiverSchedulingPolicy.rescheduleReceiver( + val scheduledExecutors = receiverSchedulingPolicy.rescheduleReceiver( 0, Some("host1"), receiverTrackingInfoMap, executors = Seq("host2")) - assert(scheduledLocations.toSet === Set("host1", "host2")) + assert(scheduledExecutors.toSet === Set("host1", "host2")) } test("rescheduleReceiver: return all idle executors if more than 3 idle executors") { @@ -46,9 +46,9 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { // host3 is idle val receiverTrackingInfoMap = Map( 0 -> ReceiverTrackingInfo(0, ReceiverState.ACTIVE, None, Some("host1"))) - val scheduledLocations = receiverSchedulingPolicy.rescheduleReceiver( + val scheduledExecutors = receiverSchedulingPolicy.rescheduleReceiver( 1, None, receiverTrackingInfoMap, executors) - assert(scheduledLocations.toSet === Set("host2", "host3", "host4", "host5")) + assert(scheduledExecutors.toSet === Set("host2", "host3", "host4", "host5")) } test("rescheduleReceiver: return 3 best options if less than 3 idle executors") { @@ -59,21 +59,21 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { 0 -> ReceiverTrackingInfo(0, ReceiverState.ACTIVE, None, Some("host1")), 1 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host2", "host3")), None), 2 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host1", "host3")), None)) - val scheduledLocations = receiverSchedulingPolicy.rescheduleReceiver( + val scheduledExecutors = receiverSchedulingPolicy.rescheduleReceiver( 3, None, receiverTrackingInfoMap, executors) - assert(scheduledLocations.toSet === Set("host2", "host4", "host5")) + assert(scheduledExecutors.toSet === Set("host2", "host4", "host5")) } test("scheduleReceivers: " + "schedule receivers evenly when there are more receivers than executors") { val receivers = (0 until 6).map(new DummyReceiver(_)) val executors = (10000 until 10003).map(port => s"localhost:${port}") - val scheduledLocations = receiverSchedulingPolicy.scheduleReceivers(receivers, executors) + val scheduledExecutors = receiverSchedulingPolicy.scheduleReceivers(receivers, executors) val numReceiversOnExecutor = mutable.HashMap[String, Int]() - // There should be 2 receivers running on each executor and each receiver has one location - scheduledLocations.foreach { case (receiverId, locations) => - assert(locations.size == 1) - numReceiversOnExecutor(locations(0)) = numReceiversOnExecutor.getOrElse(locations(0), 0) + 1 + // There should be 2 receivers running on each executor and each receiver has one executor + scheduledExecutors.foreach { case (receiverId, executors) => + assert(executors.size == 1) + numReceiversOnExecutor(executors(0)) = numReceiversOnExecutor.getOrElse(executors(0), 0) + 1 } assert(numReceiversOnExecutor === executors.map(_ -> 2).toMap) } @@ -83,12 +83,12 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { "schedule receivers evenly when there are more executors than receivers") { val receivers = (0 until 3).map(new DummyReceiver(_)) val executors = (10000 until 10006).map(port => s"localhost:${port}") - val scheduledLocations = receiverSchedulingPolicy.scheduleReceivers(receivers, executors) + val scheduledExecutors = receiverSchedulingPolicy.scheduleReceivers(receivers, executors) val numReceiversOnExecutor = mutable.HashMap[String, Int]() - // There should be 1 receiver running on each executor and each receiver has two locations - scheduledLocations.foreach { case (receiverId, locations) => - assert(locations.size == 2) - locations.foreach { l => + // There should be 1 receiver running on each executor and each receiver has two executors + scheduledExecutors.foreach { case (receiverId, executors) => + assert(executors.size == 2) + executors.foreach { l => numReceiversOnExecutor(l) = numReceiversOnExecutor.getOrElse(l, 0) + 1 } } @@ -100,21 +100,21 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { (3 until 6).map(new DummyReceiver(_, Some("localhost"))) val executors = (10000 until 10003).map(port => s"localhost:${port}") ++ (10003 until 10006).map(port => s"localhost2:${port}") - val scheduledLocations = receiverSchedulingPolicy.scheduleReceivers(receivers, executors) + val scheduledExecutors = receiverSchedulingPolicy.scheduleReceivers(receivers, executors) val numReceiversOnExecutor = mutable.HashMap[String, Int]() - // There should be 1 receiver running on each executor and each receiver has 1 location - scheduledLocations.foreach { case (receiverId, locations) => - assert(locations.size == 1) - locations.foreach { l => + // There should be 1 receiver running on each executor and each receiver has 1 executor + scheduledExecutors.foreach { case (receiverId, executors) => + assert(executors.size == 1) + executors.foreach { l => numReceiversOnExecutor(l) = numReceiversOnExecutor.getOrElse(l, 0) + 1 } } assert(numReceiversOnExecutor === executors.map(_ -> 1).toMap) // Make sure we schedule the receivers to their preferredLocations - val locationsForReceiversWithPreferredLocation = - scheduledLocations.filter { case (receiverId, locations) => receiverId >= 3 }.flatMap(_._2) - // We can simply check the location set because we only know each receiver only has 1 location - assert(locationsForReceiversWithPreferredLocation.toSet === + val executorsForReceiversWithPreferredLocation = + scheduledExecutors.filter { case (receiverId, executors) => receiverId >= 3 }.flatMap(_._2) + // We can simply check the executor set because we only know each receiver only has 1 executor + assert(executorsForReceiversWithPreferredLocation.toSet === (10000 until 10003).map(port => s"localhost:${port}").toSet) } @@ -122,11 +122,11 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { assert(receiverSchedulingPolicy.scheduleReceivers(Seq.empty, Seq("localhost:10000")).isEmpty) } - test("scheduleReceivers: return empty locations if no executors") { + test("scheduleReceivers: return empty scheduled executors if no executors") { val receivers = (0 until 3).map(new DummyReceiver(_)) - val scheduledLocations = receiverSchedulingPolicy.scheduleReceivers(receivers, Seq.empty) - scheduledLocations.foreach { case (receiverId, locations) => - assert(locations.isEmpty) + val scheduledExecutors = receiverSchedulingPolicy.scheduleReceivers(receivers, Seq.empty) + scheduledExecutors.foreach { case (receiverId, executors) => + assert(executors.isEmpty) } } } From 74514985d1a1d172b843beee54c64b0287266642 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 23 Jul 2015 16:17:22 +0800 Subject: [PATCH 24/27] Move DummyReceiver back to ReceiverTrackerSuite --- .../ReceiverSchedulingPolicySuite.scala | 17 ----------------- .../scheduler/ReceiverTrackerSuite.scala | 19 ++++++++++++++++++- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala index a03cc8f5f0c4..93f920fdc71f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala @@ -20,8 +20,6 @@ package org.apache.spark.streaming.scheduler import scala.collection.mutable import org.apache.spark.SparkFunSuite -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.receiver.Receiver class ReceiverSchedulingPolicySuite extends SparkFunSuite { @@ -130,18 +128,3 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { } } } - -/** - * Dummy receiver implementation - */ -private class DummyReceiver(receiverId: Int, host: Option[String] = None) - extends Receiver[Int](StorageLevel.MEMORY_ONLY) { - - setReceiverId(receiverId) - - override def onStart(): Unit = {} - - override def onStop(): Unit = {} - - override def preferredLocation: Option[String] = host -} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala index 2c4aac62d838..e2159bd4f225 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala @@ -19,10 +19,12 @@ package org.apache.spark.streaming.scheduler import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.streaming._ + import org.apache.spark.SparkConf +import org.apache.spark.streaming._ import org.apache.spark.streaming.receiver._ import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.storage.StorageLevel /** Testsuite for receiver scheduling */ class ReceiverTrackerSuite extends TestSuiteBase { @@ -85,3 +87,18 @@ private class RateLimitInputDStream(@transient ssc_ : StreamingContext) * one on the executor side and we won't be able to read its rate limit. */ private object SingletonDummyReceiver extends DummyReceiver(0) + +/** + * Dummy receiver implementation + */ +private class DummyReceiver(receiverId: Int, host: Option[String] = None) + extends Receiver[Int](StorageLevel.MEMORY_ONLY) { + + setReceiverId(receiverId) + + override def onStart(): Unit = {} + + override def onStop(): Unit = {} + + override def preferredLocation: Option[String] = host +} From 5e1fa488943e654efe07e19f7c240316095e91bc Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 23 Jul 2015 16:23:15 +0800 Subject: [PATCH 25/27] Fix the code style --- .../streaming/scheduler/ReceiverSchedulingPolicy.scala | 10 +++++++++- .../spark/streaming/scheduler/ReceiverTracker.scala | 2 +- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala index 0a51a7048f33..a904d40e34fb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala @@ -29,6 +29,14 @@ private[streaming] class ReceiverSchedulingPolicy { * `preferredLocation`s of receivers are not even, we may not be able to schedule them evenly * because we have to respect them. * + * Here is the approach to schedule executors: + *
    + *
  1. First, schedule all the receivers with preferred locations (hosts), evenly among the + * executors running on those host.
  2. + *
  3. Then, schedule all other receivers evenly among all the executors such that overall + * distribution over all the receivers is even.
  4. + *
+ * * This method is called when we start to launch receivers at the first time. */ def scheduleReceivers( @@ -45,7 +53,7 @@ private[streaming] class ReceiverSchedulingPolicy { val scheduledExecutors = Array.fill(receivers.length)(new mutable.ArrayBuffer[String]) val numReceiversOnExecutor = mutable.HashMap[String, Int]() // Set the initial value to 0 - executors.foreach(numReceiversOnExecutor(_) = 0) + executors.foreach(e => numReceiversOnExecutor(e) = 0) // Firstly, we need to respect "preferredLocation". So if a receiver has "preferredLocation", // we need to make sure the "preferredLocation" is in the candidate scheduled executor list. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index c14aa3d529bc..7a61743dd658 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -338,7 +338,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false receiverId, ReceiverState.SCHEDULED, Some(scheduledExecutors), - None) + runningExecutor = None) } receiverTrackingInfos.put(receiverId, newReceiverTrackingInfo) } From 61a6c3f4c3b290e93618209419eca7dcf472b529 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 23 Jul 2015 16:27:33 +0800 Subject: [PATCH 26/27] Set state to ReceiverState.INACTIVE in deregisterReceiver --- .../org/apache/spark/streaming/scheduler/ReceiverTracker.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 7a61743dd658..6270137951b5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -268,7 +268,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false lastErrorMessage = message, lastError = error, lastErrorTime = lastErrorTime) val newReceiverTrackingInfo = receiverTrackingInfos.get(streamId) match { case Some(oldInfo) => - oldInfo.copy(errorInfo = Some(errorInfo)) + oldInfo.copy(state = ReceiverState.INACTIVE, errorInfo = Some(errorInfo)) case None => logWarning("No prior receiver info") ReceiverTrackingInfo( From 137b257fabaa198706528a008394f41ca247782a Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 23 Jul 2015 16:30:33 +0800 Subject: [PATCH 27/27] Add preferredNumExecutors to rescheduleReceiver --- .../scheduler/ReceiverSchedulingPolicy.scala | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala index a904d40e34fb..ef5b687b5831 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala @@ -102,7 +102,8 @@ private[streaming] class ReceiverSchedulingPolicy { /** * Return a list of candidate executors to run the receiver. If the list is empty, the caller can - * run this receiver in arbitrary executor. + * run this receiver in arbitrary executor. The caller can use `preferredNumExecutors` to require + * returning `preferredNumExecutors` executors if possible. * * This method tries to balance executors' load. Here is the approach to schedule executors * for a receiver. @@ -121,8 +122,9 @@ private[streaming] class ReceiverSchedulingPolicy { * If a receiver is scheduled to an executor but has not yet run, it contributes * `1.0 / #candidate_executors_of_this_receiver` to the executor's weight. * - * At last, if there are more than 3 idle executors (weight = 0), returns all idle executors. - * Otherwise, we only return 3 best options according to the weights. + * At last, if there are more than `preferredNumExecutors` idle executors (weight = 0), + * returns all idle executors. Otherwise, we only return `preferredNumExecutors` best options + * according to the weights. * * * @@ -132,7 +134,8 @@ private[streaming] class ReceiverSchedulingPolicy { receiverId: Int, preferredLocation: Option[String], receiverTrackingInfoMap: Map[Int, ReceiverTrackingInfo], - executors: Seq[String]): Seq[String] = { + executors: Seq[String], + preferredNumExecutors: Int = 3): Seq[String] = { if (executors.isEmpty) { return Seq.empty } @@ -154,14 +157,14 @@ private[streaming] class ReceiverSchedulingPolicy { }.groupBy(_._1).mapValues(_.map(_._2).sum) // Sum weights for each executor val idleExecutors = (executors.toSet -- executorWeights.keys).toSeq - if (idleExecutors.size >= 3) { - // If there are more than 3 idle executors, return all of them + if (idleExecutors.size >= preferredNumExecutors) { + // If there are more than `preferredNumExecutors` idle executors, return all of them scheduledExecutors ++= idleExecutors } else { - // If there are less than 3 idle executors, return 3 best options + // If there are less than `preferredNumExecutors` idle executors, return 3 best options scheduledExecutors ++= idleExecutors val sortedExecutors = executorWeights.toSeq.sortBy(_._2).map(_._1) - scheduledExecutors ++= (idleExecutors ++ sortedExecutors).take(3) + scheduledExecutors ++= (idleExecutors ++ sortedExecutors).take(preferredNumExecutors) } scheduledExecutors.toSeq }