-
Notifications
You must be signed in to change notification settings - Fork 29k
[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
[SPARK-8975][Streaming] Adds a mechanism to send a new rate from the driver to the block generator #7471
Changes from 6 commits
4721c7d
d15de42
6369b30
cd1397d
261a051
0c51959
210f495
162d9e5
8941cf9
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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} | ||
|
|
@@ -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 = | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 |
|---|---|---|
|
|
@@ -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 | ||
|
|
||
| /** | ||
|
|
@@ -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)) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: add |
||
| eP.send(UpdateRateLimit(newRate)) | ||
| } | ||
|
|
||
| /** Add new blocks for the given stream */ | ||
| private def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = { | ||
| receivedBlockTracker.addBlock(receivedBlockInfo) | ||
|
|
||
| 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") { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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") { | ||
|
||
| 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" | ||
| ) | ||
| } | ||
| } | ||
There was a problem hiding this comment.
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.