Skip to content
Closed
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
Copy link
Contributor

Choose a reason for hiding this comment

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

This is not needed any more. Make sure to remove the empty line below too.


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 {

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[streaming] 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,12 @@ 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))
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: add { and } around the for-loop body

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
Expand Up @@ -155,63 +155,6 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable {
assert(recordedData.toSet === generatedData.toSet)
}

ignore("block generator throttling") {
val blockGeneratorListener = new FakeBlockGeneratorListener
val blockIntervalMs = 100
val maxRate = 1001
val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockIntervalMs}ms").
set("spark.streaming.receiver.maxRate", maxRate.toString)
val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf)
val expectedBlocks = 20
val waitTime = expectedBlocks * blockIntervalMs
val expectedMessages = maxRate * waitTime / 1000
val expectedMessagesPerBlock = maxRate * blockIntervalMs / 1000
val generatedData = new ArrayBuffer[Int]

// Generate blocks
val startTime = System.currentTimeMillis()
blockGenerator.start()
var count = 0
while(System.currentTimeMillis - startTime < waitTime) {
blockGenerator.addData(count)
generatedData += count
count += 1
}
blockGenerator.stop()

val recordedBlocks = blockGeneratorListener.arrayBuffers
val recordedData = recordedBlocks.flatten
assert(blockGeneratorListener.arrayBuffers.size > 0, "No blocks received")
assert(recordedData.toSet === generatedData.toSet, "Received data not same")

// recordedData size should be close to the expected rate; use an error margin proportional to
// the value, so that rate changes don't cause a brittle test
val minExpectedMessages = expectedMessages - 0.05 * expectedMessages
val maxExpectedMessages = expectedMessages + 0.05 * expectedMessages
val numMessages = recordedData.size
assert(
numMessages >= minExpectedMessages && numMessages <= maxExpectedMessages,
s"#records received = $numMessages, not between $minExpectedMessages and $maxExpectedMessages"
)

// XXX Checking every block would require an even distribution of messages across blocks,
// which throttling code does not control. Therefore, test against the average.
val minExpectedMessagesPerBlock = expectedMessagesPerBlock - 0.05 * expectedMessagesPerBlock
val maxExpectedMessagesPerBlock = expectedMessagesPerBlock + 0.05 * expectedMessagesPerBlock
val receivedBlockSizes = recordedBlocks.map { _.size }.mkString(",")

// the first and last block may be incomplete, so we slice them out
val validBlocks = recordedBlocks.drop(1).dropRight(1)
val averageBlockSize = validBlocks.map(block => block.size).sum / validBlocks.size

assert(
averageBlockSize >= minExpectedMessagesPerBlock &&
averageBlockSize <= maxExpectedMessagesPerBlock,
s"# records in received blocks = [$receivedBlockSizes], not between " +
s"$minExpectedMessagesPerBlock and $maxExpectedMessagesPerBlock, on average"
)
}

/**
* Test whether write ahead logs are generated by received,
* and automatically cleaned up. The clean up must be aware of the
Expand Down Expand Up @@ -347,28 +290,33 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable {
errors += throwable
}
}
}

/**
* An implementation of BlockGeneratorListener that is used to test the BlockGenerator.
*/
class FakeBlockGeneratorListener(pushDelay: Long = 0) extends BlockGeneratorListener {
// buffer of data received as ArrayBuffers
val arrayBuffers = new ArrayBuffer[ArrayBuffer[Int]]
val errors = new ArrayBuffer[Throwable]
/**
* An implementation of BlockGeneratorListener that is used to test the BlockGenerator.
*/
class FakeBlockGeneratorListener(pushDelay: Long = 0) extends BlockGeneratorListener {
// buffer of data received as ArrayBuffers
val arrayBuffers = new ArrayBuffer[ArrayBuffer[Int]]
val errors = new ArrayBuffer[Throwable]

def onAddData(data: Any, metadata: Any) { }
def onAddData(data: Any, metadata: Any) {}

def onGenerateBlock(blockId: StreamBlockId) { }
def onGenerateBlock(blockId: StreamBlockId) {}

def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]) {
val bufferOfInts = arrayBuffer.map(_.asInstanceOf[Int])
arrayBuffers += bufferOfInts
Thread.sleep(0)
}
def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]) {
val bufferOfInts = arrayBuffer.map(_.asInstanceOf[Int])
arrayBuffers += bufferOfInts
Thread.sleep(0)
}

def onError(message: String, throwable: Throwable) {
errors += throwable
}
def onError(message: String, throwable: Throwable) {
errors += throwable
}

def reset(): Unit = {
arrayBuffers.clear()
errors.clear()
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,154 @@
/*
* 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 scala.collection.mutable.ArrayBuffer

import org.apache.spark.SparkConf
import org.apache.spark.SparkFunSuite
import org.apache.spark.streaming.StreamingContext
import org.apache.spark.streaming.FakeBlockGeneratorListener

/** 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)
}

def setupGenerator(blockInterval: Int): (BlockGenerator, FakeBlockGeneratorListener) = {
val blockGeneratorListener = new FakeBlockGeneratorListener
val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockInterval}ms")
val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf)
(blockGenerator, blockGeneratorListener)
}

test("throttling block generator") {
Copy link
Contributor

Choose a reason for hiding this comment

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

This is a good attempt, but this is not the right attempt. This is a RateLimiterSuite, so ideally it should not be using something like a BlockGenerator to test the RateLimiter. So I would update the first three tests ("rate limiter **") to use a dummy rate limiter tester, that tests the upper limit that has been set, before and after any rate update has been made. The tester can be a simple integer increment. Note that only the upper limit needs to be tested, unlike the flaky block generator tests which test for both upper and lower limit.

Copy link
Contributor

Choose a reason for hiding this comment

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

Ignore this. See comment in the above thread.

val blockIntervalMs = 100
val (blockGenerator, blockGeneratorListener) = setupGenerator(blockIntervalMs)
val maxRate = 1000
blockGenerator.updateRate(maxRate)
blockGenerator.start()
throttlingTest(maxRate, blockGenerator, blockGeneratorListener, blockIntervalMs)
blockGenerator.stop()
}

test("throttling block generator changes rate up") {
val blockIntervalMs = 100
val (blockGenerator, blockGeneratorListener) = setupGenerator(blockIntervalMs)
val maxRate1 = 1000
blockGenerator.start()
blockGenerator.updateRate(maxRate1)
throttlingTest(maxRate1, blockGenerator, blockGeneratorListener, blockIntervalMs)

blockGeneratorListener.reset()
val maxRate2 = 5000
blockGenerator.updateRate(maxRate2)
throttlingTest(maxRate2, blockGenerator, blockGeneratorListener, blockIntervalMs)
blockGenerator.stop()
}

test("throttling block generator changes rate up and down") {
val blockIntervalMs = 100
val (blockGenerator, blockGeneratorListener) = setupGenerator(blockIntervalMs)
val maxRate1 = 1000
blockGenerator.updateRate(maxRate1)
blockGenerator.start()
throttlingTest(maxRate1, blockGenerator, blockGeneratorListener, blockIntervalMs)

blockGeneratorListener.reset()
val maxRate2 = 5000
blockGenerator.updateRate(maxRate2)
throttlingTest(maxRate2, blockGenerator, blockGeneratorListener, blockIntervalMs)

blockGeneratorListener.reset()
val maxRate3 = 1000
blockGenerator.updateRate(maxRate3)
throttlingTest(maxRate3, blockGenerator, blockGeneratorListener, blockIntervalMs)
blockGenerator.stop()
}

def throttlingTest(
maxRate: Long,
blockGenerator: BlockGenerator,
blockGeneratorListener: FakeBlockGeneratorListener,
blockIntervalMs: Int) {
val expectedBlocks = 20
val waitTime = expectedBlocks * blockIntervalMs
val expectedMessages = maxRate * waitTime / 1000
val expectedMessagesPerBlock = maxRate * blockIntervalMs / 1000
val generatedData = new ArrayBuffer[Int]

// Generate blocks
val startTime = System.currentTimeMillis()
var count = 0
while(System.currentTimeMillis - startTime < waitTime) {
blockGenerator.addData(count)
generatedData += count
count += 1
}

val recordedBlocks = blockGeneratorListener.arrayBuffers
val recordedData = recordedBlocks.flatten
assert(blockGeneratorListener.arrayBuffers.size > 0, "No blocks received")

// recordedData size should be close to the expected rate; use an error margin proportional to
// the value, so that rate changes don't cause a brittle test
val minExpectedMessages = expectedMessages - 0.05 * expectedMessages
val maxExpectedMessages = expectedMessages + 0.05 * expectedMessages
val numMessages = recordedData.size
assert(
numMessages >= minExpectedMessages && numMessages <= maxExpectedMessages,
s"#records received = $numMessages, not between $minExpectedMessages and $maxExpectedMessages"
)

// XXX Checking every block would require an even distribution of messages across blocks,
// which throttling code does not control. Therefore, test against the average.
val minExpectedMessagesPerBlock = expectedMessagesPerBlock - 0.05 * expectedMessagesPerBlock
val maxExpectedMessagesPerBlock = expectedMessagesPerBlock + 0.05 * expectedMessagesPerBlock
val receivedBlockSizes = recordedBlocks.map { _.size }.mkString(",")

// the first and last block may be incomplete, so we slice them out
val validBlocks = recordedBlocks.drop(1).dropRight(1)
val averageBlockSize = validBlocks.map(block => block.size).sum / validBlocks.size

assert(
averageBlockSize >= minExpectedMessagesPerBlock &&
averageBlockSize <= maxExpectedMessagesPerBlock,
s"# records in received blocks = [$receivedBlockSizes], not between " +
s"$minExpectedMessagesPerBlock and $maxExpectedMessagesPerBlock, on average"
)
}
}
Loading