Skip to content
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

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

Closed
Original file line number Diff line number Diff line change
Expand Up @@ -34,12 +34,26 @@ 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

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,62 @@ class ReceiverTrackerSuite extends TestSuiteBase {
assert(locations(0).length === 1)
assert(locations(3).length === 1)
}

test("Receiver tracker - propagates rate limit") {
object streamingListener extends StreamingListener {
Copy link
Contributor

Choose a reason for hiding this comment

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

This should be capitalized ... say.. TestStreamingListener, or even better ReceiverWaiter.

@volatile
var started = false

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

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

val newRateLimit = 100L
val ids = new TestReceiverInputDStream(ssc)
Copy link
Contributor

Choose a reason for hiding this comment

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

ids --> dstream. I got confused with ids being IDs

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(streamingListener.started)
}
tracker.sendRateUpdate(ids.id, newRateLimit)
// this is an async message, we need to wait a bit for it to be processed
eventually(timeout(3 seconds)) {
assert(ids.getCurrentRateLimit.get === newRateLimit)
}
}
}

/** An input DStream with a hard-coded receiver that gives access to internals for testing. */
private class TestReceiverInputDStream(@transient ssc_ : StreamingContext)
Copy link
Contributor

Choose a reason for hiding this comment

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

TestReceiverInputDStream --> DummyReceiverInputDStream

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I'll find a better name. IMO Dummy means nothing, so if names are important we could probably find a better one here. This isn't a dummy input dstream at all, it's giving access to its rate limit.

extends ReceiverInputDStream[Int](ssc_) {

override def getReceiver(): DummyReceiver = TestDummyReceiver

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

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

/**
* We need the receiver to be an object, otherwise serialization will create another one
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: make it more scala docs style to begin with. "A receiver as an object, so that we can read its rate limit. Otherwise,.."

* and we won't be able to read its rate limit.
*/
private object TestDummyReceiver extends DummyReceiver
Copy link
Contributor

Choose a reason for hiding this comment

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

I said this in another comment thread, but since that thread is hidden let me put it here as well.

You could use reflection to get the private method "executor". Just make the DummyReceiver class have a method that gets the underlying supervisor using reflection. Then, instead of another class/object TestDummyReceiver, you can just use object DummyReceiver extends DummyReceiver

Copy link
Contributor

Choose a reason for hiding this comment

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

Why did you not remove TestDummyReceiver? You could only make a object DummyReceiver extends DummyReceiver and rename TestReceiverInputDStream to DummyReceiverInputDStream. No need to introduce another Test*** (Test and Dummy is really redundant).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Is your issue only about naming? I personally find it weird to have the companion object of a class extend the class itself. It's true that I could do that, but I prefer not to. If you feel strongly about it I can do the renaming you wish, but I find it unnecessary.

Copy link
Contributor

Choose a reason for hiding this comment

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

Then let's call it SingletonReceiver. I find using test and dummy both in
the name very superfluous

On Wednesday, July 22, 2015, Iulian Dragos notifications@github.com wrote:

In
streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala
#7471 (comment):

+/** An input DStream with a hard-coded receiver that gives access to internals for testing. */
+private class TestReceiverInputDStream(@transient ssc_ : StreamingContext)

  • extends ReceiverInputDStreamInt {
  • override def getReceiver(): DummyReceiver = TestDummyReceiver
  • def getCurrentRateLimit: Option[Long] = {
  • TestDummyReceiver.executor.getCurrentRateLimit
  • }
    +}

+/**

  • * We need the receiver to be an object, otherwise serialization will create another one
  • * and we won't be able to read its rate limit.
  • */
    +private object TestDummyReceiver extends DummyReceiver

Is your issue only about naming? I personally find it weird to have the
companion object of a class extend the class itself. It's true that I
could do that, but I prefer not to. If you feel strongly about it I can
do the renaming you wish, but I find it unnecessary.


Reply to this email directly or view it on GitHub
https://github.com/apache/spark/pull/7471/files#r35208599.

Sent from Gmail Mobile

Copy link
Contributor

Choose a reason for hiding this comment

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

Then let's call it SingletonReceiver. I find using test and dummy both in
the name very superfl

On Wednesday, July 22, 2015, Iulian Dragos <notifications@github.com
javascript:_e(%7B%7D,'cvml','notifications@github.com');> wrote:

In
streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala
#7471 (comment):

+/** An input DStream with a hard-coded receiver that gives access to internals for testing. */
+private class TestReceiverInputDStream(@transient ssc_ : StreamingContext)

  • extends ReceiverInputDStreamInt {
  • override def getReceiver(): DummyReceiver = TestDummyReceiver
  • def getCurrentRateLimit: Option[Long] = {
  • TestDummyReceiver.executor.getCurrentRateLimit
  • }
    +}

+/**

  • * We need the receiver to be an object, otherwise serialization will create another one
  • * and we won't be able to read its rate limit.
  • */
    +private object TestDummyReceiver extends DummyReceiver

Is your issue only about naming? I personally find it weird to have the
companion object of a class extend the class itself. It's true that I
could do that, but I prefer not to. If you feel strongly about it I can
do the renaming you wish, but I find it unnecessary.


Reply to this email directly or view it on GitHub
https://github.com/apache/spark/pull/7471/files#r35208599.

Sent from Gmail Mobile


/**
* Dummy receiver implementation
*/
Expand Down