This repository was archived by the owner on May 9, 2024. It is now read-only.
forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 3
[SPARK-8834] Add backpressure-based dynamic throttling to Spark Streaming #14
Closed
Closed
Changes from all commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
f7cbae9
Adds basic rate estimation and speed listening for streams
huitseeker 2d20eb8
Adds facilities to attach a RateController to each InputDStream
huitseeker ac1795c
Applies throttling for the DirectKafkaInputDStream
huitseeker a62a31e
Add log message on rate update
huitseeker File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or 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 hidden or 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 hidden or 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 hidden or 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 hidden or 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 hidden or 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 hidden or 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
67 changes: 67 additions & 0 deletions
67
streaming/src/main/scala/org/apache/spark/streaming/scheduler/RateController.scala
This file contains hidden or 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,67 @@ | ||
/* | ||
* 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.util.concurrent.atomic.AtomicLong | ||
|
||
import org.apache.spark.annotation.DeveloperApi | ||
import org.apache.spark.streaming.scheduler.rate.RateEstimator | ||
import org.apache.spark.util.ThreadUtils | ||
|
||
import scala.concurrent.{ExecutionContext, Future} | ||
|
||
/** | ||
* :: DeveloperApi :: | ||
* 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` | ||
*/ | ||
@DeveloperApi | ||
class RateController(val streamUID: Int, | ||
rateEstimator: RateEstimator, | ||
publisher: Long => Unit = (l) => ()) | ||
extends StreamingListener { | ||
|
||
private val rateUpdateExecutionContext = ExecutionContext.fromExecutorService( | ||
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. Couldn't/Shouldn't this be passed into the constructor so that there's a choice in creating new threads and which ThreadFactory and ThreadGroup etc they will use? |
||
ThreadUtils.newDaemonSingleThreadExecutor("stream-rate-update")) | ||
|
||
private val speedLimit : AtomicLong = new AtomicLong(-1L) | ||
|
||
def rateUpdate(time: Long, elems: Long, workDelay: Long, waitDelay: Long): Unit = | ||
Future[Unit] { | ||
val newSpeed = rateEstimator.compute(time, elems, workDelay, waitDelay) | ||
newSpeed foreach { s => | ||
speedLimit.set(s.toLong) | ||
publisher(getLatestRate()) | ||
} | ||
} (rateUpdateExecutionContext) | ||
|
||
def getLatestRate(): Long = speedLimit.get() | ||
|
||
override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted){ | ||
val elements = batchCompleted.batchInfo.streamIdToNumRecords | ||
|
||
for { | ||
processingEnd <- batchCompleted.batchInfo.processingEndTime | ||
workDelay <- batchCompleted.batchInfo.processingDelay | ||
waitDelay <- batchCompleted.batchInfo.schedulingDelay | ||
elems <- elements.get(streamUID) | ||
} rateUpdate(processingEnd, elems, workDelay, waitDelay) | ||
} | ||
|
||
} |
This file contains hidden or 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
74 changes: 74 additions & 0 deletions
74
streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/PIDRateEstimator.scala
This file contains hidden or 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,74 @@ | ||
/* | ||
* 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 | ||
|
||
class PIDRateEstimator(proportional: Double = -1D, | ||
integral: Double = -.2D, | ||
derivative: Double = 0D) | ||
extends RateEstimator{ | ||
|
||
var init: Boolean = true | ||
var latestTime : Long = -1L | ||
var latestSpeed : Double = -1D | ||
var latestError : Double = -1L | ||
|
||
def compute(time: Long, // in milliseconds | ||
elements: Long, | ||
processingDelay: Long, // in milliseconds | ||
schedulingDelay: Long // in milliseconds | ||
): Option[Double] = { | ||
|
||
this.synchronized { | ||
if (time > latestTime && processingDelay > 0) { | ||
|
||
// in seconds, should be close to batchDuration | ||
val delaySinceUpdate = (time - latestTime).toDouble / 1000 | ||
|
||
// in elements/second | ||
val processingSpeed = elements.toDouble / processingDelay * 1000 | ||
|
||
// in elements/second | ||
val error = latestSpeed - processingSpeed | ||
|
||
// in elements/second | ||
val sumError = schedulingDelay.toDouble / 1000 * processingSpeed / delaySinceUpdate | ||
|
||
// in elements/(second ^ 2) | ||
val dError = (error - latestError) / delaySinceUpdate | ||
|
||
val newSpeed = (latestSpeed + proportional * error + | ||
integral * sumError + | ||
derivative * dError) max 0D | ||
latestTime = time | ||
if (init) { | ||
latestSpeed = processingSpeed | ||
latestError = 0D | ||
init = false | ||
|
||
None | ||
} else { | ||
latestSpeed = newSpeed | ||
latestError = error | ||
|
||
Some(newSpeed) | ||
} | ||
} else None | ||
} | ||
} | ||
|
||
} |
38 changes: 38 additions & 0 deletions
38
streaming/src/main/scala/org/apache/spark/streaming/scheduler/rate/RateEstimator.scala
This file contains hidden or 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,38 @@ | ||
/* | ||
* 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.annotation.DeveloperApi | ||
|
||
/** | ||
* :: DeveloperApi :: | ||
* A component that estimates the rate at wich an InputDStream should ingest | ||
* elements, based on updates at every batch completion. | ||
*/ | ||
@DeveloperApi | ||
trait RateEstimator { | ||
|
||
/** | ||
* 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. | ||
*/ | ||
def compute(time: Long, elements: Long, | ||
processingDelay: Long, schedulingDelay: Long): Option[Double] | ||
|
||
} |
This file contains hidden or 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
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Why the coercion from long to int?