-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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
[SPARK-8975][Streaming] Adds a mechanism to send a new rate from the driver to the block generator #7471
Changes from 4 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,28 @@ 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.getInt("spark.streaming.receiver.maxRate", 0) | ||
private[receiver] var currentRateLimit = new AtomicInteger(maxRateLimit) | ||
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. What is this variable for? Seems like the only place its value is referenced outside this class is RateLimiterSuite. Rather than storing the current rate in the guava RateLimiter as well as this atomicInt, isnt it easier to just keep it one location, that is, the guava rate limiter? If you want to access the rate limit externally, then just have a def that returns, guava rateLimiter.getRate(). |
||
private lazy val rateLimiter = GuavaRateLimiter.create(currentRateLimit.get()) | ||
|
||
def waitToPush() { | ||
if (desiredRate > 0) { | ||
if (currentRateLimit.get() > 0) { | ||
rateLimiter.acquire() | ||
} | ||
} | ||
|
||
private[receiver] def updateRate(newRate: Int): Unit = | ||
if (newRate > 0) { | ||
try { | ||
if (maxRateLimit > 0) { | ||
currentRateLimit.set(newRate.min(maxRateLimit)) | ||
} | ||
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 use |
||
else { | ||
currentRateLimit.set(newRate) | ||
} | ||
} finally { | ||
rateLimiter.setRate(currentRateLimit.get()) | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -271,7 +271,7 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable | |
} | ||
|
||
/** Get the attached executor. */ | ||
private def executor = { | ||
private[streaming] def executor = { | ||
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. Why this change? If you want to access private members, its best to use the 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 didn't know about 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 could probably code something along those lines, but I think it's better to use qualified private. Are there any downsides to relying on language support for visibility? I doubt any user-code would declare things in 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. Aah, I see. The reason I wanted to use PrivateTest is because marking private[streaming] make it invisible through Scala compilation, but its visibly publicly in the bytecode, and therefore through Java. And since Receivers can be written in Java, I dont want to expose that. PrivateTester wont work, but the underlying mechanism will working using reflection.
You could use this do the testing. Just make the DummyReceiver have a method that gets the underlying supervisor. Then an object DummyReceiver (no need for TestDummyReceiver class and object) for access to the supervisor. |
||
assert(executor_ != null, "Executor has not been attached to this receiver") | ||
executor_ | ||
} | ||
|
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 rate from an estimator's update */ | ||
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: There should be no references to estimator in this scala doc. Just "Update a receiver's max ingestion rate" is good enough. |
||
def sendRateUpdate(streamUID: Int, newRate: Long): Unit = { | ||
for (info <- receiverInfo.get(streamUID); eP <- Option(info.endpoint)) | ||
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 |
---|---|---|
|
@@ -537,4 +537,19 @@ trait TestSuiteBase extends SparkFunSuite with BeforeAndAfter with Logging { | |
verifyOutput[W](output, expectedOutput, useSet) | ||
} | ||
} | ||
|
||
/** | ||
* Wait until `cond` becomes true, or timeout ms have passed. This method checks the condition | ||
* every 100ms, so it won't wait more than 100ms more than necessary. | ||
* | ||
* @param cond A boolean that should become `true` | ||
* @param timemout How many millis to wait before giving up | ||
*/ | ||
def waitUntil(cond: => Boolean, timeout: Int): Unit = { | ||
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. This method is general enough that probably deserves a better place. I couldn't find anything similar in spark-core Utils or streaming utilities (the closest one was in BatchCounter, but won't do the same thing, and relies on wait/notify). I'm open to suggestions where to move it. 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 have used No need to add another method here. 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. cool, thanks! |
||
val start = System.currentTimeMillis() | ||
val end = start + timeout | ||
while ((System.currentTimeMillis() < end) && !cond) { | ||
Thread.sleep(100) | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,47 @@ | ||
/* | ||
* 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") { | ||
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 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. 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.currentRateLimit.get == 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.currentRateLimit.get == 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.currentRateLimit.get == 100) | ||
} | ||
|
||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -22,6 +22,9 @@ 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 { | ||
|
@@ -72,15 +75,46 @@ class ReceiverTrackerSuite extends TestSuiteBase { | |
assert(locations(0).length === 1) | ||
assert(locations(3).length === 1) | ||
} | ||
|
||
test("Receiver tracker - propagates rate limit") { | ||
val newRateLimit = 100 | ||
val ids = new TestReceiverInputDStream(ssc) | ||
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. ids --> dstream. I got confused with ids being IDs |
||
val tracker = new ReceiverTracker(ssc) | ||
tracker.start() | ||
waitUntil(TestDummyReceiver.started, 5000) | ||
tracker.sendRateUpdate(ids.id, newRateLimit) | ||
// this is an async message, we need to wait a bit for it to be processed | ||
waitUntil(ids.getRateLimit.get == newRateLimit, 1000) | ||
assert(ids.getRateLimit.get === newRateLimit) | ||
} | ||
} | ||
|
||
/** An input DStream with a hard-coded receiver that gives access to internals for testing. */ | ||
private class TestReceiverInputDStream(@transient ssc_ : StreamingContext) | ||
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. TestReceiverInputDStream --> DummyReceiverInputDStream 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'll find a better name. IMO |
||
extends ReceiverInputDStream[Int](ssc_) { | ||
|
||
override def getReceiver(): DummyReceiver = TestDummyReceiver | ||
|
||
def getRateLimit: Option[Int] = | ||
TestDummyReceiver.executor.getCurrentRateLimit | ||
} | ||
|
||
/** | ||
* We need the receiver to be an object, otherwise serialization will create another one | ||
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: 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 | ||
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 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 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. Why did you not remove TestDummyReceiver? You could only make a 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. 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. 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. Then let's call it SingletonReceiver. I find using test and dummy both in On Wednesday, July 22, 2015, Iulian Dragos [email protected] wrote:
Sent from Gmail Mobile 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. Then let's call it SingletonReceiver. I find using test and dummy both in On Wednesday, July 22, 2015, Iulian Dragos <[email protected]
Sent from Gmail Mobile |
||
|
||
/** | ||
* Dummy receiver implementation | ||
*/ | ||
private class DummyReceiver(host: Option[String] = None) | ||
extends Receiver[Int](StorageLevel.MEMORY_ONLY) { | ||
|
||
var started = false | ||
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. This needs to be @volatile, otherwise waiting threads (one calling |
||
|
||
def onStart() { | ||
started = true | ||
} | ||
|
||
def onStop() { | ||
|
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.