forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-8977] [STREAMING] Defines the RateEstimator interface, and imp…
…ements the RateController Based on apache#7471. - [x] add a test that exercises the publish path from driver to receiver - [ ] remove Serializable from `RateController` and `RateEstimator` Author: Iulian Dragos <[email protected]> Author: François Garillot <[email protected]> Closes apache#7600 from dragos/topic/streaming-bp/rate-controller and squashes the following commits: f168c94 [Iulian Dragos] Latest review round. 5125e60 [Iulian Dragos] Fix style. a2eb3b9 [Iulian Dragos] Merge remote-tracking branch 'upstream/master' into topic/streaming-bp/rate-controller 475e346 [Iulian Dragos] Latest round of reviews. e9fb45e [Iulian Dragos] - Add a test for checkpointing - fixed serialization for RateController.executionContext 715437a [Iulian Dragos] Review comments and added a `reset` call in ReceiverTrackerTest. e57c66b [Iulian Dragos] Added a couple of tests for the full scenario from driver to receivers, with several rate updates. b425d32 [Iulian Dragos] Removed DeveloperAPI, removed rateEstimator field, removed Noop rate estimator, changed logic for initialising rate estimator. 238cfc6 [Iulian Dragos] Merge remote-tracking branch 'upstream/master' into topic/streaming-bp/rate-controller 34a389d [Iulian Dragos] Various style changes and a first test for the rate controller. d32ca36 [François Garillot] [SPARK-8977][Streaming] Defines the RateEstimator interface, and implements the ReceiverRateController 8941cf9 [Iulian Dragos] Renames and other nitpicks. 162d9e5 [Iulian Dragos] Use Reflection for accessing truly private `executor` method and use the listener bus to know when receivers have registered (`onStart` is called before receivers have registered, leading to flaky behavior). 210f495 [Iulian Dragos] Revert "Added a few tests that measure the receiver’s rate." 0c51959 [Iulian Dragos] Added a few tests that measure the receiver’s rate. 261a051 [Iulian Dragos] - removed field to hold the current rate limit in rate limiter - made rate limit a Long and default to Long.MaxValue (consequence of the above) - removed custom `waitUntil` and replaced it by `eventually` cd1397d [Iulian Dragos] Add a test for the propagation of a new rate limit from driver to receivers. 6369b30 [Iulian Dragos] Merge pull request #15 from huitseeker/SPARK-8975 d15de42 [François Garillot] [SPARK-8975][Streaming] Adds Ratelimiter unit tests w.r.t. spark.streaming.receiver.maxRate 4721c7d [François Garillot] [SPARK-8975][Streaming] Add a mechanism to send a new rate from the driver to the block generator
- Loading branch information
Showing
9 changed files
with
355 additions
and
15 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
90 changes: 90 additions & 0 deletions
90
streaming/src/main/scala/org/apache/spark/streaming/scheduler/RateController.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,90 @@ | ||
/* | ||
* 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.scheduler | ||
|
||
import java.io.ObjectInputStream | ||
import java.util.concurrent.atomic.AtomicLong | ||
|
||
import scala.concurrent.{ExecutionContext, Future} | ||
|
||
import org.apache.spark.SparkConf | ||
import org.apache.spark.streaming.scheduler.rate.RateEstimator | ||
import org.apache.spark.util.{ThreadUtils, Utils} | ||
|
||
/** | ||
* A StreamingListener that receives batch completion updates, and maintains | ||
* an estimate of the speed at which this stream should ingest messages, | ||
* given an estimate computation from a `RateEstimator` | ||
*/ | ||
private[streaming] abstract class RateController(val streamUID: Int, rateEstimator: RateEstimator) | ||
extends StreamingListener with Serializable { | ||
|
||
init() | ||
|
||
protected def publish(rate: Long): Unit | ||
|
||
@transient | ||
implicit private var executionContext: ExecutionContext = _ | ||
|
||
@transient | ||
private var rateLimit: AtomicLong = _ | ||
|
||
/** | ||
* An initialization method called both from the constructor and Serialization code. | ||
*/ | ||
private def init() { | ||
executionContext = ExecutionContext.fromExecutorService( | ||
ThreadUtils.newDaemonSingleThreadExecutor("stream-rate-update")) | ||
rateLimit = new AtomicLong(-1L) | ||
} | ||
|
||
private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { | ||
ois.defaultReadObject() | ||
init() | ||
} | ||
|
||
/** | ||
* Compute the new rate limit and publish it asynchronously. | ||
*/ | ||
private def computeAndPublish(time: Long, elems: Long, workDelay: Long, waitDelay: Long): Unit = | ||
Future[Unit] { | ||
val newRate = rateEstimator.compute(time, elems, workDelay, waitDelay) | ||
newRate.foreach { s => | ||
rateLimit.set(s.toLong) | ||
publish(getLatestRate()) | ||
} | ||
} | ||
|
||
def getLatestRate(): Long = rateLimit.get() | ||
|
||
override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { | ||
val elements = batchCompleted.batchInfo.streamIdToInputInfo | ||
|
||
for { | ||
processingEnd <- batchCompleted.batchInfo.processingEndTime; | ||
workDelay <- batchCompleted.batchInfo.processingDelay; | ||
waitDelay <- batchCompleted.batchInfo.schedulingDelay; | ||
elems <- elements.get(streamUID).map(_.numRecords) | ||
} computeAndPublish(processingEnd, elems, workDelay, waitDelay) | ||
} | ||
} | ||
|
||
object RateController { | ||
def isBackPressureEnabled(conf: SparkConf): Boolean = | ||
conf.getBoolean("spark.streaming.backpressure.enable", false) | ||
} |
59 changes: 59 additions & 0 deletions
59
streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/RateEstimator.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,59 @@ | ||
/* | ||
* 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.scheduler.rate | ||
|
||
import org.apache.spark.SparkConf | ||
import org.apache.spark.SparkException | ||
|
||
/** | ||
* A component that estimates the rate at wich an InputDStream should ingest | ||
* elements, based on updates at every batch completion. | ||
*/ | ||
private[streaming] trait RateEstimator extends Serializable { | ||
|
||
/** | ||
* Computes the number of elements the stream attached to this `RateEstimator` | ||
* should ingest per second, given an update on the size and completion | ||
* times of the latest batch. | ||
* | ||
* @param time The timetamp of the current batch interval that just finished | ||
* @param elements The number of elements that were processed in this batch | ||
* @param processingDelay The time in ms that took for the job to complete | ||
* @param schedulingDelay The time in ms that the job spent in the scheduling queue | ||
*/ | ||
def compute( | ||
time: Long, | ||
elements: Long, | ||
processingDelay: Long, | ||
schedulingDelay: Long): Option[Double] | ||
} | ||
|
||
object RateEstimator { | ||
|
||
/** | ||
* Return a new RateEstimator based on the value of `spark.streaming.RateEstimator`. | ||
* | ||
* @return None if there is no configured estimator, otherwise an instance of RateEstimator | ||
* @throws IllegalArgumentException if there is a configured RateEstimator that doesn't match any | ||
* known estimators. | ||
*/ | ||
def create(conf: SparkConf): Option[RateEstimator] = | ||
conf.getOption("spark.streaming.backpressure.rateEstimator").map { estimator => | ||
throw new IllegalArgumentException(s"Unkown rate estimator: $estimator") | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
103 changes: 103 additions & 0 deletions
103
streaming/src/test/scala/org/apache/spark/streaming/scheduler/RateControllerSuite.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,103 @@ | ||
/* | ||
* 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.scheduler | ||
|
||
import scala.collection.mutable | ||
import scala.reflect.ClassTag | ||
import scala.util.control.NonFatal | ||
|
||
import org.scalatest.Matchers._ | ||
import org.scalatest.concurrent.Eventually._ | ||
import org.scalatest.time.SpanSugar._ | ||
|
||
import org.apache.spark.streaming._ | ||
import org.apache.spark.streaming.scheduler.rate.RateEstimator | ||
|
||
class RateControllerSuite extends TestSuiteBase { | ||
|
||
override def useManualClock: Boolean = false | ||
|
||
test("rate controller publishes updates") { | ||
val ssc = new StreamingContext(conf, batchDuration) | ||
withStreamingContext(ssc) { ssc => | ||
val dstream = new RateLimitInputDStream(ssc) | ||
dstream.register() | ||
ssc.start() | ||
|
||
eventually(timeout(10.seconds)) { | ||
assert(dstream.publishCalls > 0) | ||
} | ||
} | ||
} | ||
|
||
test("publish rates reach receivers") { | ||
val ssc = new StreamingContext(conf, batchDuration) | ||
withStreamingContext(ssc) { ssc => | ||
val dstream = new RateLimitInputDStream(ssc) { | ||
override val rateController = | ||
Some(new ReceiverRateController(id, new ConstantEstimator(200.0))) | ||
} | ||
dstream.register() | ||
SingletonTestRateReceiver.reset() | ||
ssc.start() | ||
|
||
eventually(timeout(10.seconds)) { | ||
assert(dstream.getCurrentRateLimit === Some(200)) | ||
} | ||
} | ||
} | ||
|
||
test("multiple publish rates reach receivers") { | ||
val ssc = new StreamingContext(conf, batchDuration) | ||
withStreamingContext(ssc) { ssc => | ||
val rates = Seq(100L, 200L, 300L) | ||
|
||
val dstream = new RateLimitInputDStream(ssc) { | ||
override val rateController = | ||
Some(new ReceiverRateController(id, new ConstantEstimator(rates.map(_.toDouble): _*))) | ||
} | ||
SingletonTestRateReceiver.reset() | ||
dstream.register() | ||
|
||
val observedRates = mutable.HashSet.empty[Long] | ||
ssc.start() | ||
|
||
eventually(timeout(20.seconds)) { | ||
dstream.getCurrentRateLimit.foreach(observedRates += _) | ||
// Long.MaxValue (essentially, no rate limit) is the initial rate limit for any Receiver | ||
observedRates should contain theSameElementsAs (rates :+ Long.MaxValue) | ||
} | ||
} | ||
} | ||
} | ||
|
||
private[streaming] class ConstantEstimator(rates: Double*) extends RateEstimator { | ||
private var idx: Int = 0 | ||
|
||
private def nextRate(): Double = { | ||
val rate = rates(idx) | ||
idx = (idx + 1) % rates.size | ||
rate | ||
} | ||
|
||
def compute( | ||
time: Long, | ||
elements: Long, | ||
processingDelay: Long, | ||
schedulingDelay: Long): Option[Double] = Some(nextRate()) | ||
} |
Oops, something went wrong.