Skip to content

[SPARK-8975][Streaming] Adds a mechanism to send a new rate from the driver to the block generator #7471

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
Closed
Original file line number Diff line number Diff line change
Expand Up @@ -34,12 +34,32 @@ import org.apache.spark.{Logging, SparkConf}
*/
private[receiver] abstract class RateLimiter(conf: SparkConf) extends Logging {

private val desiredRate = conf.getInt("spark.streaming.receiver.maxRate", 0)
private lazy val rateLimiter = GuavaRateLimiter.create(desiredRate)
// treated as an upper limit
private val maxRateLimit = conf.getLong("spark.streaming.receiver.maxRate", Long.MaxValue)
private lazy val rateLimiter = GuavaRateLimiter.create(maxRateLimit.toDouble)

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

/**
* Return the current rate limit. If no limit has been set so far, it returns {{{Long.MaxValue}}}.
*/
def getCurrentLimit: Long =
rateLimiter.getRate.toLong

/**
* Set the rate limit to `newRate`. The new rate will not exceed the maximum rate configured by
* {{{spark.streaming.receiver.maxRate}}}, even if `newRate` is higher than that.
*
* @param newRate A new rate in events per second. It has no effect if it's 0 or negative.
*/
private[receiver] def updateRate(newRate: Long): Unit =
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: Need a scala doc, for consistency

if (newRate > 0) {
if (maxRateLimit > 0) {
rateLimiter.setRate(newRate.min(maxRateLimit))
} else {
rateLimiter.setRate(newRate)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -271,7 +271,7 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable
}

/** Get the attached executor. */
private def executor = {
private def executor: ReceiverSupervisor = {
assert(executor_ != null, "Executor has not been attached to this receiver")
executor_
}
Expand Down
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 UpdateRateLimit(elementsPerSecond: Long)
extends ReceiverMessage
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,9 @@ private[streaming] abstract class ReceiverSupervisor(
/** Time between a receiver is stopped and started again */
private val defaultRestartDelay = conf.getInt("spark.streaming.receiverRestartDelay", 2000)

/** The current maximum rate limit for this receiver. */
private[streaming] def getCurrentRateLimit: Option[Long] = None

/** Exception associated with the stopping of the receiver */
@volatile protected var stoppingError: Throwable = null

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,9 @@ private[streaming] class ReceiverSupervisorImpl(
case CleanupOldBlocks(threshTime) =>
logDebug("Received delete old batch signal")
cleanupOldBlocks(threshTime)
case UpdateRateLimit(eps) =>
logInfo(s"Received a new rate limit: $eps.")
blockGenerator.updateRate(eps)
}
})

Expand All @@ -98,6 +101,9 @@ private[streaming] class ReceiverSupervisorImpl(
}
}, streamId, env.conf)

override private[streaming] def getCurrentRateLimit: Option[Long] =
Some(blockGenerator.getCurrentLimit)

/** Push a single record of received data into block generator. */
def pushSingle(data: Any) {
blockGenerator.addData(data)
Expand Down
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, UpdateRateLimit}
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 ingestion rate */
def sendRateUpdate(streamUID: Int, newRate: Long): Unit = {
for (info <- receiverInfo.get(streamUID); eP <- Option(info.endpoint)) {
eP.send(UpdateRateLimit(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,46 @@
/*
* 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.receiver

import org.apache.spark.SparkConf
import org.apache.spark.SparkFunSuite

/** Testsuite for testing the network receiver behavior */
class RateLimiterSuite extends SparkFunSuite {

test("rate limiter initializes even without a maxRate set") {
Copy link
Contributor

Choose a reason for hiding this comment

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

@dragos I had mentioned in the earlier PR that we need these test to actually send data through the rateLimiter to see whether the updated rate is taking effect. Otherwise this test to see test whether a == 10 after setting a = 10 is nice but not very useful.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

We could add an integration test. On the other hand, I personally don't see great use in testing Guava's code, I assume we take the same stance towards other libraries too. So, my assumption here is that if the update reaches the receiver, Guava's RateLimiter is working just fine.

Copy link
Contributor

Choose a reason for hiding this comment

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

I was not sure whether the Guava rate limiter behaves in the desired way when the rate is updated. There are several ways of implementing a updateable rate limiter, and I wanted to understand this's one behavior, and cover it in tests. However, I just digged deeper in Guava RateLimiter code, and realized that there are fundamentally bigger problems in using the Guava rate limiter, even for stable rate. It works effectively using token bucket philosophy, which allows a large burst of data to be allowed, if there has been no data for while. We DONT want that. This means that we need to implement our own rate limiter (which was the case, until someone replaced my implementation with Guava). Anyways, thats not something outside the scope of this PR. So I am okay not testing the rate limiting.

So please revert ReceiverSuite, and remove the block manager unit tests in this testsuite. Basically, revert it to the previous iteration (sorry :) )

val conf = new SparkConf()
val rateLimiter = new RateLimiter(conf){}
rateLimiter.updateRate(105)
assert(rateLimiter.getCurrentLimit == 105)
}

test("rate limiter updates when below maxRate") {
val conf = new SparkConf().set("spark.streaming.receiver.maxRate", "110")
val rateLimiter = new RateLimiter(conf){}
rateLimiter.updateRate(105)
assert(rateLimiter.getCurrentLimit == 105)
}

test("rate limiter stays below maxRate despite large updates") {
val conf = new SparkConf().set("spark.streaming.receiver.maxRate", "100")
val rateLimiter = new RateLimiter(conf){}
rateLimiter.updateRate(105)
assert(rateLimiter.getCurrentLimit === 100)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,11 +17,17 @@

package org.apache.spark.streaming.scheduler

import org.scalatest.concurrent.Eventually._
import org.scalatest.concurrent.Timeouts
import org.scalatest.time.SpanSugar._
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
import org.apache.spark.streaming.dstream.InputDStream
import scala.reflect.ClassTag
import org.apache.spark.streaming.dstream.ReceiverInputDStream

/** Testsuite for receiver scheduling */
class ReceiverTrackerSuite extends TestSuiteBase {
Expand Down Expand Up @@ -72,8 +78,64 @@ class ReceiverTrackerSuite extends TestSuiteBase {
assert(locations(0).length === 1)
assert(locations(3).length === 1)
}

test("Receiver tracker - propagates rate limit") {
object ReceiverStartedWaiter extends StreamingListener {
@volatile
var started = false

override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted): Unit = {
started = true
}
}

ssc.addStreamingListener(ReceiverStartedWaiter)
ssc.scheduler.listenerBus.start(ssc.sc)

val newRateLimit = 100L
val inputDStream = new RateLimitInputDStream(ssc)
val tracker = new ReceiverTracker(ssc)
tracker.start()

// we wait until the Receiver has registered with the tracker,
// otherwise our rate update is lost
eventually(timeout(5 seconds)) {
assert(ReceiverStartedWaiter.started)
}
tracker.sendRateUpdate(inputDStream.id, newRateLimit)
// this is an async message, we need to wait a bit for it to be processed
eventually(timeout(3 seconds)) {
assert(inputDStream.getCurrentRateLimit.get === newRateLimit)
}
}
}

/** An input DStream with a hard-coded receiver that gives access to internals for testing. */
private class RateLimitInputDStream(@transient ssc_ : StreamingContext)
extends ReceiverInputDStream[Int](ssc_) {

override def getReceiver(): DummyReceiver = SingletonDummyReceiver

def getCurrentRateLimit: Option[Long] = {
invokeExecutorMethod.getCurrentRateLimit
}

private def invokeExecutorMethod: ReceiverSupervisor = {
val c = classOf[Receiver[_]]
val ex = c.getDeclaredMethod("executor")
ex.setAccessible(true)
ex.invoke(SingletonDummyReceiver).asInstanceOf[ReceiverSupervisor]
}
}

/**
* A Receiver as an object so we can read its rate limit.
*
* @note It's necessary to be a top-level object, or else serialization would create another
* one on the executor side and we won't be able to read its rate limit.
*/
private object SingletonDummyReceiver extends DummyReceiver

/**
* Dummy receiver implementation
*/
Expand Down