forked from alteryx/spark
-
Notifications
You must be signed in to change notification settings - Fork 0
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 impl…
…ements the ReceiverRateController
- Loading branch information
1 parent
8941cf9
commit d32ca36
Showing
5 changed files
with
144 additions
and
1 deletion.
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
69 changes: 69 additions & 0 deletions
69
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,69 @@ | ||
/* | ||
* 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 | ||
private [streaming] abstract class RateController(val streamUID: Int, rateEstimator: RateEstimator) | ||
extends StreamingListener with Serializable { | ||
|
||
protected def publish(rate: Long): Unit | ||
|
||
// Used to compute & publish the rate update asynchronously | ||
@transient private val executionContext = ExecutionContext.fromExecutorService( | ||
ThreadUtils.newDaemonSingleThreadExecutor("stream-rate-update")) | ||
|
||
private val rateLimit : AtomicLong = new AtomicLong(-1L) | ||
|
||
// Asynchronous computation of the rate update | ||
private def computeAndPublish(time: Long, elems: Long, workDelay: Long, waitDelay: Long): Unit = | ||
Future[Unit] { | ||
val newSpeed = rateEstimator.compute(time, elems, workDelay, waitDelay) | ||
newSpeed foreach { s => | ||
rateLimit.set(s.toLong) | ||
publish(getLatestRate()) | ||
} | ||
} (executionContext) | ||
|
||
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) | ||
} | ||
|
||
} |
46 changes: 46 additions & 0 deletions
46
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,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.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 | ||
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. | ||
*/ | ||
def compute(time: Long, elements: Long, | ||
processingDelay: Long, schedulingDelay: Long): Option[Double] | ||
} | ||
|
||
/** | ||
* The trivial rate estimator never sends an update | ||
*/ | ||
private[streaming] class NoopRateEstimator extends RateEstimator { | ||
|
||
def compute(time: Long, elements: Long, | ||
processingDelay: Long, schedulingDelay: Long): Option[Double] = None | ||
} |