-
Notifications
You must be signed in to change notification settings - Fork 28.3k
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 8 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 |
---|---|---|
@@ -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") { | ||
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.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 |
---|---|---|
|
@@ -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 { | ||
|
@@ -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 { | ||
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 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) | ||
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() | ||
|
||
// 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) | ||
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 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 | ||
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 notifications@github.com 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 <notifications@github.com
Sent from Gmail Mobile |
||
|
||
/** | ||
* Dummy receiver implementation | ||
*/ | ||
|
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.
nit: Need a scala doc, for consistency