Skip to content
This repository was archived by the owner on May 9, 2024. It is now read-only.

[SPARK-8834] Add backpressure-based dynamic throttling to Spark Streaming #14

Closed
wants to merge 4 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -73,12 +73,16 @@ class DirectKafkaInputDStream[

protected val kc = new KafkaCluster(kafkaParams)

protected val maxMessagesPerPartition: Option[Long] = {
val ratePerSec = context.sparkContext.getConf.getInt(
private val ratePerSec: Int = context.sparkContext.getConf.getInt(
"spark.streaming.kafka.maxRatePerPartition", 0)
if (ratePerSec > 0) {
protected def maxMessagesPerPartition: Option[Long] = {
val effectiveRatePerSec =
rateController.map(_.getLatestRate().toInt).filter(_ > 0).fold(ratePerSec){
ratePerSec min _
}
if (effectiveRatePerSec > 0) {
val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000
Some((secsPerBatch * ratePerSec).toLong)
Some((secsPerBatch * effectiveRatePerSec).toLong)
} else {
None
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,8 @@ import scala.reflect.ClassTag
import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDDOperationScope
import org.apache.spark.streaming.{Time, Duration, StreamingContext}
import org.apache.spark.streaming.scheduler.rate._
import org.apache.spark.streaming.scheduler.RateController
import org.apache.spark.util.Utils

/**
Expand All @@ -47,6 +49,18 @@ abstract class InputDStream[T: ClassTag] (@transient ssc_ : StreamingContext)
/** This is an unique identifier for the input stream. */
val id = ssc.getNewInputStreamId()

/**
* A StreamingListener that maintains a maximum ingestion speed for this
* stream. Initialized with `attachRateEstimator`.
*/
private[streaming] var rateController: Option[RateController] = None

def attachRateEstimator(rateEstimator: RateEstimator): Unit = {
if (ssc.scheduler.isStarted == false) {
rateController = Some(new RateController(id, rateEstimator))
}
}

/** A human-readable name of this InputDStream */
private[streaming] def name: String = {
// e.g. FlumePollingDStream -> "Flume polling stream"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@ import org.apache.spark.streaming._
import org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD
import org.apache.spark.streaming.receiver.Receiver
import org.apache.spark.streaming.scheduler.StreamInputInfo
import org.apache.spark.streaming.scheduler.RateController
import org.apache.spark.streaming.scheduler.rate.RateEstimator
import org.apache.spark.streaming.util.WriteAheadLogUtils

/**
Expand All @@ -40,6 +42,13 @@ import org.apache.spark.streaming.util.WriteAheadLogUtils
abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingContext)
extends InputDStream[T](ssc_) {

override def attachRateEstimator(rateEstimator: RateEstimator): Unit = {
if (ssc.scheduler.isStarted == false) {
def publish(rate: Long): Unit = ssc.scheduler.receiverTracker.updateStreamRate(id, rate)
rateController = Some(new RateController(id, rateEstimator, publish))
}
}

/**
* Gets the receiver object that will be sent to the worker nodes
* to receive data. This method needs to defined by any specific implementation
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.spark.streaming.receiver

import java.util.concurrent.atomic.AtomicInteger

import com.google.common.util.concurrent.{RateLimiter => GuavaRateLimiter}

import org.apache.spark.{Logging, SparkConf}
Expand All @@ -34,12 +36,26 @@ import org.apache.spark.{Logging, SparkConf}
*/
private[receiver] abstract class RateLimiter(conf: SparkConf) extends Logging {

// treated as an upper limit
private val desiredRate = conf.getInt("spark.streaming.receiver.maxRate", 0)
private lazy val rateLimiter = GuavaRateLimiter.create(desiredRate)
private var currentRate = new AtomicInteger(desiredRate)
private lazy val rateLimiter = GuavaRateLimiter.create(currentRate.get())

def waitToPush() {
if (desiredRate > 0) {
if (currentRate.get() > 0) {
rateLimiter.acquire()
}
}

private[receiver] def updateRate(newRate: Int): Unit =
if (newRate > 0) {
try {
if (desiredRate > 0) {
currentRate.set(newRate min desiredRate)
}
else currentRate.set(newRate)
} finally {
rateLimiter.setRate(currentRate.get())
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,4 +23,5 @@ import org.apache.spark.streaming.Time
private[streaming] sealed trait ReceiverMessage extends Serializable
private[streaming] object StopReceiver extends ReceiverMessage
private[streaming] case class CleanupOldBlocks(threshTime: Time) extends ReceiverMessage

private[streaming] case class RateLimitUpdate(elementsPerSecond: Long)
extends ReceiverMessage
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,10 @@ private[streaming] class ReceiverSupervisorImpl(
case CleanupOldBlocks(threshTime) =>
logDebug("Received delete old batch signal")
cleanupOldBlocks(threshTime)
case RateLimitUpdate(eps) => {
blockGenerator.updateRate(eps.toInt)

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why the coercion from long to int?

logDebug(s"Received update for $streamId : $eps")
}
}
})

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,8 @@ class JobScheduler(val ssc: StreamingContext) extends Logging {
eventLoop.start()

listenerBus.start(ssc.sparkContext)
// Estimators receive updates from batch completion
ssc.graph.getInputStreams.flatMap(_.rateController).foreach(ssc.addStreamingListener(_))
receiverTracker = new ReceiverTracker(ssc)
inputInfoTracker = new InputInfoTracker(ssc)
receiverTracker.start()
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* 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 java.util.concurrent.atomic.AtomicLong

import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.streaming.scheduler.rate.RateEstimator
import org.apache.spark.util.ThreadUtils

import scala.concurrent.{ExecutionContext, Future}

/**
* :: DeveloperApi ::
* A StreamingListener that receives batch completion updates, and maintains
* an estimate of the speed at which this stream should ingest messages,
* given an estimate computation from a `RateEstimator`
*/
@DeveloperApi
class RateController(val streamUID: Int,
rateEstimator: RateEstimator,
publisher: Long => Unit = (l) => ())
extends StreamingListener {

private val rateUpdateExecutionContext = ExecutionContext.fromExecutorService(

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Couldn't/Shouldn't this be passed into the constructor so that there's a choice in creating new threads and which ThreadFactory and ThreadGroup etc they will use?

ThreadUtils.newDaemonSingleThreadExecutor("stream-rate-update"))

private val speedLimit : AtomicLong = new AtomicLong(-1L)

def rateUpdate(time: Long, elems: Long, workDelay: Long, waitDelay: Long): Unit =
Future[Unit] {
val newSpeed = rateEstimator.compute(time, elems, workDelay, waitDelay)
newSpeed foreach { s =>
speedLimit.set(s.toLong)
publisher(getLatestRate())
}
} (rateUpdateExecutionContext)

def getLatestRate(): Long = speedLimit.get()

override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted){
val elements = batchCompleted.batchInfo.streamIdToNumRecords

for {
processingEnd <- batchCompleted.batchInfo.processingEndTime
workDelay <- batchCompleted.batchInfo.processingDelay
waitDelay <- batchCompleted.batchInfo.schedulingDelay
elems <- elements.get(streamUID)
} rateUpdate(processingEnd, elems, workDelay, waitDelay)
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ import org.apache.spark.{Logging, SparkEnv, SparkException}
import org.apache.spark.rpc._
import org.apache.spark.streaming.{StreamingContext, Time}
import org.apache.spark.streaming.receiver.{CleanupOldBlocks, Receiver, ReceiverSupervisorImpl,
StopReceiver}
StopReceiver, RateLimitUpdate}
import org.apache.spark.util.SerializableConfiguration

/**
Expand Down Expand Up @@ -180,6 +180,13 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
logError(s"Deregistered receiver for stream $streamId: $messageWithError")
}

/** Update a receiver's maximum rate from an estimator's update */
def updateStreamRate(streamUID: Int, newRate: Long): Unit = {
for {info <- receiverInfo.get(streamUID)
eP <- Option(info.endpoint)}
eP.send(RateLimitUpdate(newRate))
}

/** Add new blocks for the given stream */
private def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = {
receivedBlockTracker.addBlock(receivedBlockInfo)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
/*
* 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.rate

class PIDRateEstimator(proportional: Double = -1D,
integral: Double = -.2D,
derivative: Double = 0D)
extends RateEstimator{

var init: Boolean = true
var latestTime : Long = -1L
var latestSpeed : Double = -1D
var latestError : Double = -1L

def compute(time: Long, // in milliseconds
elements: Long,
processingDelay: Long, // in milliseconds
schedulingDelay: Long // in milliseconds
): Option[Double] = {

this.synchronized {
if (time > latestTime && processingDelay > 0) {

// in seconds, should be close to batchDuration
val delaySinceUpdate = (time - latestTime).toDouble / 1000

// in elements/second
val processingSpeed = elements.toDouble / processingDelay * 1000

// in elements/second
val error = latestSpeed - processingSpeed

// in elements/second
val sumError = schedulingDelay.toDouble / 1000 * processingSpeed / delaySinceUpdate

// in elements/(second ^ 2)
val dError = (error - latestError) / delaySinceUpdate

val newSpeed = (latestSpeed + proportional * error +
integral * sumError +
derivative * dError) max 0D
latestTime = time
if (init) {
latestSpeed = processingSpeed
latestError = 0D
init = false

None
} else {
latestSpeed = newSpeed
latestError = error

Some(newSpeed)
}
} else None
}
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
/*
* 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.rate

import org.apache.spark.annotation.DeveloperApi

/**
* :: DeveloperApi ::
* A component that estimates the rate at wich an InputDStream should ingest
* elements, based on updates at every batch completion.
*/
@DeveloperApi
trait RateEstimator {

/**
* Computes the number of elements the stream attached to this `RateEstimator`
* should ingest per second, given an update on the size and completion
* times of the latest batch.
*/
def compute(time: Long, elements: Long,
processingDelay: Long, schedulingDelay: Long): Option[Double]

}
Original file line number Diff line number Diff line change
Expand Up @@ -21,15 +21,16 @@ import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer}
import scala.concurrent.Future
import scala.concurrent.ExecutionContext.Implicits.global

import org.apache.spark.Logging
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.dstream.DStream
import org.apache.spark.streaming.receiver.Receiver
import org.apache.spark.streaming.scheduler._
import org.apache.spark.streaming.scheduler.rate._

import org.scalatest.Matchers
import org.scalatest.concurrent.Eventually._
import org.scalatest.time.SpanSugar._
import org.apache.spark.Logging

class StreamingListenerSuite extends TestSuiteBase with Matchers {

Expand Down Expand Up @@ -131,6 +132,30 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers {
}
}

// This test is long to run an may be dependent on your machine's
// characteristics (high variance in estimating processing speed on a
// small batch)
ignore("latest speed reporting") {
val midInput = (1 to 100).map(Seq(_)).toSeq
val midSsc = setupStreams(midInput, operation)
val midLatestRate = new RateController(0, new PIDRateEstimator(-1, 0, 0))
midSsc.addStreamingListener(midLatestRate)
runStreams(midSsc, input.size, input.size)

val midSp = midLatestRate.getLatestRate()

// between two batch sizes that are both below the system's limits,
// the estimate of elements processed per batch should be comparable
val bigInput = (1 to 1000).map(Seq(_)).toSeq
val bigSsc = setupStreams(bigInput, operation)
val bigLatestRate = new RateController(0, new PIDRateEstimator(-1, 0, 0))
bigSsc.addStreamingListener(bigLatestRate)
runStreams(bigSsc, bigInput.size, bigInput.size)

val bigSp = bigLatestRate.getLatestRate()
bigSp should (be >= (midSp / 2) and be <= (midSp * 2))
}

/** Check if a sequence of numbers is in increasing order */
def isInIncreasingOrder(seq: Seq[Long]): Boolean = {
for (i <- 1 until seq.size) {
Expand Down
Loading