Split Time to Time (absolute instant of time) and Duration (duration of time).

This commit is contained in:
Tathagata Das 2013-01-09 12:42:10 -08:00
parent 6c502e3793
commit 156e8b47ef
25 changed files with 174 additions and 152 deletions

View file

@ -17,7 +17,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
val jars = ssc.sc.jars val jars = ssc.sc.jars
val graph = ssc.graph val graph = ssc.graph
val checkpointDir = ssc.checkpointDir val checkpointDir = ssc.checkpointDir
val checkpointInterval = ssc.checkpointInterval val checkpointInterval: Duration = ssc.checkpointInterval
def validate() { def validate() {
assert(master != null, "Checkpoint.master is null") assert(master != null, "Checkpoint.master is null")

View file

@ -2,7 +2,7 @@ package spark.streaming
import spark.streaming.dstream._ import spark.streaming.dstream._
import StreamingContext._ import StreamingContext._
import Time._ //import Time._
import spark.{RDD, Logging} import spark.{RDD, Logging}
import spark.storage.StorageLevel import spark.storage.StorageLevel
@ -47,7 +47,7 @@ abstract class DStream[T: ClassManifest] (
// ======================================================================= // =======================================================================
/** Time interval after which the DStream generates a RDD */ /** Time interval after which the DStream generates a RDD */
def slideTime: Time def slideTime: Duration
/** List of parent DStreams on which this DStream depends on */ /** List of parent DStreams on which this DStream depends on */
def dependencies: List[DStream[_]] def dependencies: List[DStream[_]]
@ -67,14 +67,14 @@ abstract class DStream[T: ClassManifest] (
protected[streaming] var zeroTime: Time = null protected[streaming] var zeroTime: Time = null
// Duration for which the DStream will remember each RDD created // Duration for which the DStream will remember each RDD created
protected[streaming] var rememberDuration: Time = null protected[streaming] var rememberDuration: Duration = null
// Storage level of the RDDs in the stream // Storage level of the RDDs in the stream
protected[streaming] var storageLevel: StorageLevel = StorageLevel.NONE protected[streaming] var storageLevel: StorageLevel = StorageLevel.NONE
// Checkpoint details // Checkpoint details
protected[streaming] val mustCheckpoint = false protected[streaming] val mustCheckpoint = false
protected[streaming] var checkpointInterval: Time = null protected[streaming] var checkpointInterval: Duration = null
protected[streaming] var checkpointData = new DStreamCheckpointData(HashMap[Time, Any]()) protected[streaming] var checkpointData = new DStreamCheckpointData(HashMap[Time, Any]())
// Reference to whole DStream graph // Reference to whole DStream graph
@ -108,7 +108,7 @@ abstract class DStream[T: ClassManifest] (
* Enable periodic checkpointing of RDDs of this DStream * Enable periodic checkpointing of RDDs of this DStream
* @param interval Time interval after which generated RDD will be checkpointed * @param interval Time interval after which generated RDD will be checkpointed
*/ */
def checkpoint(interval: Time): DStream[T] = { def checkpoint(interval: Duration): DStream[T] = {
if (isInitialized) { if (isInitialized) {
throw new UnsupportedOperationException( throw new UnsupportedOperationException(
"Cannot change checkpoint interval of an DStream after streaming context has started") "Cannot change checkpoint interval of an DStream after streaming context has started")
@ -224,7 +224,7 @@ abstract class DStream[T: ClassManifest] (
dependencies.foreach(_.setGraph(graph)) dependencies.foreach(_.setGraph(graph))
} }
protected[streaming] def remember(duration: Time) { protected[streaming] def remember(duration: Duration) {
if (duration != null && duration > rememberDuration) { if (duration != null && duration > rememberDuration) {
rememberDuration = duration rememberDuration = duration
logInfo("Duration for remembering RDDs set to " + rememberDuration + " for " + this) logInfo("Duration for remembering RDDs set to " + rememberDuration + " for " + this)
@ -531,7 +531,7 @@ abstract class DStream[T: ClassManifest] (
* @param windowTime width of the window; must be a multiple of this DStream's interval. * @param windowTime width of the window; must be a multiple of this DStream's interval.
* @return * @return
*/ */
def window(windowTime: Time): DStream[T] = window(windowTime, this.slideTime) def window(windowTime: Duration): DStream[T] = window(windowTime, this.slideTime)
/** /**
* Return a new DStream which is computed based on windowed batches of this DStream. * Return a new DStream which is computed based on windowed batches of this DStream.
@ -541,7 +541,7 @@ abstract class DStream[T: ClassManifest] (
* the new DStream will generate RDDs); must be a multiple of this * the new DStream will generate RDDs); must be a multiple of this
* DStream's interval * DStream's interval
*/ */
def window(windowTime: Time, slideTime: Time): DStream[T] = { def window(windowTime: Duration, slideTime: Duration): DStream[T] = {
new WindowedDStream(this, windowTime, slideTime) new WindowedDStream(this, windowTime, slideTime)
} }
@ -550,22 +550,22 @@ abstract class DStream[T: ClassManifest] (
* This is equivalent to window(batchTime, batchTime). * This is equivalent to window(batchTime, batchTime).
* @param batchTime tumbling window duration; must be a multiple of this DStream's interval * @param batchTime tumbling window duration; must be a multiple of this DStream's interval
*/ */
def tumble(batchTime: Time): DStream[T] = window(batchTime, batchTime) def tumble(batchTime: Duration): DStream[T] = window(batchTime, batchTime)
/** /**
* Returns a new DStream in which each RDD has a single element generated by reducing all * Returns a new DStream in which each RDD has a single element generated by reducing all
* elements in a window over this DStream. windowTime and slideTime are as defined in the * elements in a window over this DStream. windowTime and slideTime are as defined in the
* window() operation. This is equivalent to window(windowTime, slideTime).reduce(reduceFunc) * window() operation. This is equivalent to window(windowTime, slideTime).reduce(reduceFunc)
*/ */
def reduceByWindow(reduceFunc: (T, T) => T, windowTime: Time, slideTime: Time): DStream[T] = { def reduceByWindow(reduceFunc: (T, T) => T, windowTime: Duration, slideTime: Duration): DStream[T] = {
this.window(windowTime, slideTime).reduce(reduceFunc) this.window(windowTime, slideTime).reduce(reduceFunc)
} }
def reduceByWindow( def reduceByWindow(
reduceFunc: (T, T) => T, reduceFunc: (T, T) => T,
invReduceFunc: (T, T) => T, invReduceFunc: (T, T) => T,
windowTime: Time, windowTime: Duration,
slideTime: Time slideTime: Duration
): DStream[T] = { ): DStream[T] = {
this.map(x => (1, x)) this.map(x => (1, x))
.reduceByKeyAndWindow(reduceFunc, invReduceFunc, windowTime, slideTime, 1) .reduceByKeyAndWindow(reduceFunc, invReduceFunc, windowTime, slideTime, 1)
@ -577,7 +577,7 @@ abstract class DStream[T: ClassManifest] (
* of elements in a window over this DStream. windowTime and slideTime are as defined in the * of elements in a window over this DStream. windowTime and slideTime are as defined in the
* window() operation. This is equivalent to window(windowTime, slideTime).count() * window() operation. This is equivalent to window(windowTime, slideTime).count()
*/ */
def countByWindow(windowTime: Time, slideTime: Time): DStream[Int] = { def countByWindow(windowTime: Duration, slideTime: Duration): DStream[Int] = {
this.map(_ => 1).reduceByWindow(_ + _, _ - _, windowTime, slideTime) this.map(_ => 1).reduceByWindow(_ + _, _ - _, windowTime, slideTime)
} }

View file

@ -12,8 +12,8 @@ final private[streaming] class DStreamGraph extends Serializable with Logging {
private val outputStreams = new ArrayBuffer[DStream[_]]() private val outputStreams = new ArrayBuffer[DStream[_]]()
private[streaming] var zeroTime: Time = null private[streaming] var zeroTime: Time = null
private[streaming] var batchDuration: Time = null private[streaming] var batchDuration: Duration = null
private[streaming] var rememberDuration: Time = null private[streaming] var rememberDuration: Duration = null
private[streaming] var checkpointInProgress = false private[streaming] var checkpointInProgress = false
private[streaming] def start(time: Time) { private[streaming] def start(time: Time) {
@ -41,7 +41,7 @@ final private[streaming] class DStreamGraph extends Serializable with Logging {
} }
} }
private[streaming] def setBatchDuration(duration: Time) { private[streaming] def setBatchDuration(duration: Duration) {
this.synchronized { this.synchronized {
if (batchDuration != null) { if (batchDuration != null) {
throw new Exception("Batch duration already set as " + batchDuration + throw new Exception("Batch duration already set as " + batchDuration +
@ -51,7 +51,7 @@ final private[streaming] class DStreamGraph extends Serializable with Logging {
batchDuration = duration batchDuration = duration
} }
private[streaming] def remember(duration: Time) { private[streaming] def remember(duration: Duration) {
this.synchronized { this.synchronized {
if (rememberDuration != null) { if (rememberDuration != null) {
throw new Exception("Batch duration already set as " + batchDuration + throw new Exception("Batch duration already set as " + batchDuration +

View file

@ -0,0 +1,62 @@
package spark.streaming
class Duration (private val millis: Long) {
def < (that: Duration): Boolean = (this.millis < that.millis)
def <= (that: Duration): Boolean = (this.millis <= that.millis)
def > (that: Duration): Boolean = (this.millis > that.millis)
def >= (that: Duration): Boolean = (this.millis >= that.millis)
def + (that: Duration): Duration = new Duration(millis + that.millis)
def - (that: Duration): Duration = new Duration(millis - that.millis)
def * (times: Int): Duration = new Duration(millis * times)
def / (that: Duration): Long = millis / that.millis
def isMultipleOf(that: Duration): Boolean =
(this.millis % that.millis == 0)
def min(that: Duration): Duration = if (this < that) this else that
def max(that: Duration): Duration = if (this > that) this else that
def isZero: Boolean = (this.millis == 0)
override def toString: String = (millis.toString + " ms")
def toFormattedString: String = millis.toString
def milliseconds: Long = millis
}
/**
* Helper object that creates instance of [[spark.streaming.Duration]] representing
* a given number of milliseconds.
*/
object Milliseconds {
def apply(milliseconds: Long) = new Duration(milliseconds)
}
/**
* Helper object that creates instance of [[spark.streaming.Duration]] representing
* a given number of seconds.
*/
object Seconds {
def apply(seconds: Long) = new Duration(seconds * 1000)
}
/**
* Helper object that creates instance of [[spark.streaming.Duration]] representing
* a given number of minutes.
*/
object Minutes {
def apply(minutes: Long) = new Duration(minutes * 60000)
}

View file

@ -1,16 +1,16 @@
package spark.streaming package spark.streaming
private[streaming] private[streaming]
case class Interval(beginTime: Time, endTime: Time) { class Interval(val beginTime: Time, val endTime: Time) {
def this(beginMs: Long, endMs: Long) = this(Time(beginMs), new Time(endMs)) def this(beginMs: Long, endMs: Long) = this(new Time(beginMs), new Time(endMs))
def duration(): Time = endTime - beginTime def duration(): Duration = endTime - beginTime
def + (time: Time): Interval = { def + (time: Duration): Interval = {
new Interval(beginTime + time, endTime + time) new Interval(beginTime + time, endTime + time)
} }
def - (time: Time): Interval = { def - (time: Duration): Interval = {
new Interval(beginTime - time, endTime - time) new Interval(beginTime - time, endTime - time)
} }
@ -27,24 +27,14 @@ case class Interval(beginTime: Time, endTime: Time) {
def >= (that: Interval) = !(this < that) def >= (that: Interval) = !(this < that)
def next(): Interval = { override def toString = "[" + beginTime + ", " + endTime + "]"
this + (endTime - beginTime)
}
def isZero = (beginTime.isZero && endTime.isZero)
def toFormattedString = beginTime.toFormattedString + "-" + endTime.toFormattedString
override def toString = "[" + beginTime + ", " + endTime + "]"
} }
object Interval { object Interval {
def zero() = new Interval (Time.zero, Time.zero) def currentInterval(duration: Duration): Interval = {
val time = new Time(System.currentTimeMillis)
def currentInterval(intervalDuration: Time): Interval = { val intervalBegin = time.floor(duration)
val time = Time(System.currentTimeMillis) new Interval(intervalBegin, intervalBegin + duration)
val intervalBegin = time.floor(intervalDuration)
Interval(intervalBegin, intervalBegin + intervalDuration)
} }
} }

View file

@ -69,21 +69,21 @@ extends Serializable {
self.map(x => (x._1, 1L)).reduceByKey((x: Long, y: Long) => x + y, numPartitions) self.map(x => (x._1, 1L)).reduceByKey((x: Long, y: Long) => x + y, numPartitions)
} }
def groupByKeyAndWindow(windowTime: Time, slideTime: Time): DStream[(K, Seq[V])] = { def groupByKeyAndWindow(windowTime: Duration, slideTime: Duration): DStream[(K, Seq[V])] = {
groupByKeyAndWindow(windowTime, slideTime, defaultPartitioner()) groupByKeyAndWindow(windowTime, slideTime, defaultPartitioner())
} }
def groupByKeyAndWindow( def groupByKeyAndWindow(
windowTime: Time, windowTime: Duration,
slideTime: Time, slideTime: Duration,
numPartitions: Int numPartitions: Int
): DStream[(K, Seq[V])] = { ): DStream[(K, Seq[V])] = {
groupByKeyAndWindow(windowTime, slideTime, defaultPartitioner(numPartitions)) groupByKeyAndWindow(windowTime, slideTime, defaultPartitioner(numPartitions))
} }
def groupByKeyAndWindow( def groupByKeyAndWindow(
windowTime: Time, windowTime: Duration,
slideTime: Time, slideTime: Duration,
partitioner: Partitioner partitioner: Partitioner
): DStream[(K, Seq[V])] = { ): DStream[(K, Seq[V])] = {
self.window(windowTime, slideTime).groupByKey(partitioner) self.window(windowTime, slideTime).groupByKey(partitioner)
@ -91,23 +91,23 @@ extends Serializable {
def reduceByKeyAndWindow( def reduceByKeyAndWindow(
reduceFunc: (V, V) => V, reduceFunc: (V, V) => V,
windowTime: Time windowTime: Duration
): DStream[(K, V)] = { ): DStream[(K, V)] = {
reduceByKeyAndWindow(reduceFunc, windowTime, self.slideTime, defaultPartitioner()) reduceByKeyAndWindow(reduceFunc, windowTime, self.slideTime, defaultPartitioner())
} }
def reduceByKeyAndWindow( def reduceByKeyAndWindow(
reduceFunc: (V, V) => V, reduceFunc: (V, V) => V,
windowTime: Time, windowTime: Duration,
slideTime: Time slideTime: Duration
): DStream[(K, V)] = { ): DStream[(K, V)] = {
reduceByKeyAndWindow(reduceFunc, windowTime, slideTime, defaultPartitioner()) reduceByKeyAndWindow(reduceFunc, windowTime, slideTime, defaultPartitioner())
} }
def reduceByKeyAndWindow( def reduceByKeyAndWindow(
reduceFunc: (V, V) => V, reduceFunc: (V, V) => V,
windowTime: Time, windowTime: Duration,
slideTime: Time, slideTime: Duration,
numPartitions: Int numPartitions: Int
): DStream[(K, V)] = { ): DStream[(K, V)] = {
reduceByKeyAndWindow(reduceFunc, windowTime, slideTime, defaultPartitioner(numPartitions)) reduceByKeyAndWindow(reduceFunc, windowTime, slideTime, defaultPartitioner(numPartitions))
@ -115,8 +115,8 @@ extends Serializable {
def reduceByKeyAndWindow( def reduceByKeyAndWindow(
reduceFunc: (V, V) => V, reduceFunc: (V, V) => V,
windowTime: Time, windowTime: Duration,
slideTime: Time, slideTime: Duration,
partitioner: Partitioner partitioner: Partitioner
): DStream[(K, V)] = { ): DStream[(K, V)] = {
val cleanedReduceFunc = ssc.sc.clean(reduceFunc) val cleanedReduceFunc = ssc.sc.clean(reduceFunc)
@ -134,8 +134,8 @@ extends Serializable {
def reduceByKeyAndWindow( def reduceByKeyAndWindow(
reduceFunc: (V, V) => V, reduceFunc: (V, V) => V,
invReduceFunc: (V, V) => V, invReduceFunc: (V, V) => V,
windowTime: Time, windowTime: Duration,
slideTime: Time slideTime: Duration
): DStream[(K, V)] = { ): DStream[(K, V)] = {
reduceByKeyAndWindow( reduceByKeyAndWindow(
@ -145,8 +145,8 @@ extends Serializable {
def reduceByKeyAndWindow( def reduceByKeyAndWindow(
reduceFunc: (V, V) => V, reduceFunc: (V, V) => V,
invReduceFunc: (V, V) => V, invReduceFunc: (V, V) => V,
windowTime: Time, windowTime: Duration,
slideTime: Time, slideTime: Duration,
numPartitions: Int numPartitions: Int
): DStream[(K, V)] = { ): DStream[(K, V)] = {
@ -157,8 +157,8 @@ extends Serializable {
def reduceByKeyAndWindow( def reduceByKeyAndWindow(
reduceFunc: (V, V) => V, reduceFunc: (V, V) => V,
invReduceFunc: (V, V) => V, invReduceFunc: (V, V) => V,
windowTime: Time, windowTime: Duration,
slideTime: Time, slideTime: Duration,
partitioner: Partitioner partitioner: Partitioner
): DStream[(K, V)] = { ): DStream[(K, V)] = {
@ -169,8 +169,8 @@ extends Serializable {
} }
def countByKeyAndWindow( def countByKeyAndWindow(
windowTime: Time, windowTime: Duration,
slideTime: Time, slideTime: Duration,
numPartitions: Int = self.ssc.sc.defaultParallelism numPartitions: Int = self.ssc.sc.defaultParallelism
): DStream[(K, Long)] = { ): DStream[(K, Long)] = {

View file

@ -22,7 +22,8 @@ class Scheduler(ssc: StreamingContext) extends Logging {
val clockClass = System.getProperty("spark.streaming.clock", "spark.streaming.util.SystemClock") val clockClass = System.getProperty("spark.streaming.clock", "spark.streaming.util.SystemClock")
val clock = Class.forName(clockClass).newInstance().asInstanceOf[Clock] val clock = Class.forName(clockClass).newInstance().asInstanceOf[Clock]
val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds, generateRDDs(_)) val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds,
longTime => generateRDDs(new Time(longTime)))
def start() { def start() {
// If context was started from checkpoint, then restart timer such that // If context was started from checkpoint, then restart timer such that
@ -41,7 +42,7 @@ class Scheduler(ssc: StreamingContext) extends Logging {
timer.restart(graph.zeroTime.milliseconds) timer.restart(graph.zeroTime.milliseconds)
logInfo("Scheduler's timer restarted") logInfo("Scheduler's timer restarted")
} else { } else {
val firstTime = Time(timer.start()) val firstTime = new Time(timer.start())
graph.start(firstTime - ssc.graph.batchDuration) graph.start(firstTime - ssc.graph.batchDuration)
logInfo("Scheduler's timer started") logInfo("Scheduler's timer started")
} }

View file

@ -26,7 +26,7 @@ import java.util.UUID
class StreamingContext private ( class StreamingContext private (
sc_ : SparkContext, sc_ : SparkContext,
cp_ : Checkpoint, cp_ : Checkpoint,
batchDur_ : Time batchDur_ : Duration
) extends Logging { ) extends Logging {
/** /**
@ -34,7 +34,7 @@ class StreamingContext private (
* @param sparkContext Existing SparkContext * @param sparkContext Existing SparkContext
* @param batchDuration The time interval at which streaming data will be divided into batches * @param batchDuration The time interval at which streaming data will be divided into batches
*/ */
def this(sparkContext: SparkContext, batchDuration: Time) = this(sparkContext, null, batchDuration) def this(sparkContext: SparkContext, batchDuration: Duration) = this(sparkContext, null, batchDuration)
/** /**
* Creates a StreamingContext by providing the details necessary for creating a new SparkContext. * Creates a StreamingContext by providing the details necessary for creating a new SparkContext.
@ -42,7 +42,7 @@ class StreamingContext private (
* @param frameworkName A name for your job, to display on the cluster web UI * @param frameworkName A name for your job, to display on the cluster web UI
* @param batchDuration The time interval at which streaming data will be divided into batches * @param batchDuration The time interval at which streaming data will be divided into batches
*/ */
def this(master: String, frameworkName: String, batchDuration: Time) = def this(master: String, frameworkName: String, batchDuration: Duration) =
this(StreamingContext.createNewSparkContext(master, frameworkName), null, batchDuration) this(StreamingContext.createNewSparkContext(master, frameworkName), null, batchDuration)
/** /**
@ -96,7 +96,7 @@ class StreamingContext private (
} }
} }
protected[streaming] var checkpointInterval: Time = if (isCheckpointPresent) cp_.checkpointInterval else null protected[streaming] var checkpointInterval: Duration = if (isCheckpointPresent) cp_.checkpointInterval else null
protected[streaming] var receiverJobThread: Thread = null protected[streaming] var receiverJobThread: Thread = null
protected[streaming] var scheduler: Scheduler = null protected[streaming] var scheduler: Scheduler = null
@ -107,7 +107,7 @@ class StreamingContext private (
* if the developer wishes to query old data outside the DStream computation). * if the developer wishes to query old data outside the DStream computation).
* @param duration Minimum duration that each DStream should remember its RDDs * @param duration Minimum duration that each DStream should remember its RDDs
*/ */
def remember(duration: Time) { def remember(duration: Duration) {
graph.remember(duration) graph.remember(duration)
} }
@ -117,7 +117,7 @@ class StreamingContext private (
* @param directory HDFS-compatible directory where the checkpoint data will be reliably stored * @param directory HDFS-compatible directory where the checkpoint data will be reliably stored
* @param interval checkpoint interval * @param interval checkpoint interval
*/ */
def checkpoint(directory: String, interval: Time = null) { def checkpoint(directory: String, interval: Duration = null) {
if (directory != null) { if (directory != null) {
sc.setCheckpointDir(StreamingContext.getSparkCheckpointDir(directory)) sc.setCheckpointDir(StreamingContext.getSparkCheckpointDir(directory))
checkpointDir = directory checkpointDir = directory

View file

@ -7,7 +7,7 @@ package spark.streaming
* @param millis Time in UTC. * @param millis Time in UTC.
*/ */
case class Time(private val millis: Long) { class Time(private val millis: Long) {
def < (that: Time): Boolean = (this.millis < that.millis) def < (that: Time): Boolean = (this.millis < that.millis)
@ -17,63 +17,32 @@ case class Time(private val millis: Long) {
def >= (that: Time): Boolean = (this.millis >= that.millis) def >= (that: Time): Boolean = (this.millis >= that.millis)
def + (that: Time): Time = Time(millis + that.millis) def + (that: Duration): Time = new Time(millis + that.milliseconds)
def - (that: Time): Time = Time(millis - that.millis)
def * (times: Int): Time = Time(millis * times)
def / (that: Time): Long = millis / that.millis def - (that: Time): Duration = new Duration(millis - that.millis)
def floor(that: Time): Time = { def - (that: Duration): Time = new Time(millis - that.milliseconds)
val t = that.millis
def floor(that: Duration): Time = {
val t = that.milliseconds
val m = math.floor(this.millis / t).toLong val m = math.floor(this.millis / t).toLong
Time(m * t) new Time(m * t)
} }
def isMultipleOf(that: Time): Boolean = def isMultipleOf(that: Duration): Boolean =
(this.millis % that.millis == 0) (this.millis % that.milliseconds == 0)
def min(that: Time): Time = if (this < that) this else that def min(that: Time): Time = if (this < that) this else that
def max(that: Time): Time = if (this > that) this else that def max(that: Time): Time = if (this > that) this else that
def isZero: Boolean = (this.millis == 0)
override def toString: String = (millis.toString + " ms") override def toString: String = (millis.toString + " ms")
def toFormattedString: String = millis.toString
def milliseconds: Long = millis def milliseconds: Long = millis
} }
private[streaming] object Time { /*private[streaming] object Time {
val zero = Time(0)
implicit def toTime(long: Long) = Time(long) implicit def toTime(long: Long) = Time(long)
} }
*/
/**
* Helper object that creates instance of [[spark.streaming.Time]] representing
* a given number of milliseconds.
*/
object Milliseconds {
def apply(milliseconds: Long) = Time(milliseconds)
}
/**
* Helper object that creates instance of [[spark.streaming.Time]] representing
* a given number of seconds.
*/
object Seconds {
def apply(seconds: Long) = Time(seconds * 1000)
}
/**
* Helper object that creates instance of [[spark.streaming.Time]] representing
* a given number of minutes.
*/
object Minutes {
def apply(minutes: Long) = Time(minutes * 60000)
}

View file

@ -2,7 +2,7 @@ package spark.streaming.dstream
import spark.{RDD, Partitioner} import spark.{RDD, Partitioner}
import spark.rdd.CoGroupedRDD import spark.rdd.CoGroupedRDD
import spark.streaming.{Time, DStream} import spark.streaming.{Time, DStream, Duration}
private[streaming] private[streaming]
class CoGroupedDStream[K : ClassManifest]( class CoGroupedDStream[K : ClassManifest](
@ -24,7 +24,7 @@ class CoGroupedDStream[K : ClassManifest](
override def dependencies = parents.toList override def dependencies = parents.toList
override def slideTime = parents.head.slideTime override def slideTime: Duration = parents.head.slideTime
override def compute(validTime: Time): Option[RDD[(K, Seq[Seq[_]])]] = { override def compute(validTime: Time): Option[RDD[(K, Seq[Seq[_]])]] = {
val part = partitioner val part = partitioner

View file

@ -1,6 +1,6 @@
package spark.streaming.dstream package spark.streaming.dstream
import spark.streaming.{DStream, Time} import spark.streaming.{Duration, DStream, Time}
import spark.RDD import spark.RDD
private[streaming] private[streaming]
@ -11,7 +11,7 @@ class FilteredDStream[T: ClassManifest](
override def dependencies = List(parent) override def dependencies = List(parent)
override def slideTime: Time = parent.slideTime override def slideTime: Duration = parent.slideTime
override def compute(validTime: Time): Option[RDD[T]] = { override def compute(validTime: Time): Option[RDD[T]] = {
parent.getOrCompute(validTime).map(_.filter(filterFunc)) parent.getOrCompute(validTime).map(_.filter(filterFunc))

View file

@ -1,6 +1,6 @@
package spark.streaming.dstream package spark.streaming.dstream
import spark.streaming.{DStream, Time} import spark.streaming.{Duration, DStream, Time}
import spark.RDD import spark.RDD
import spark.SparkContext._ import spark.SparkContext._
@ -12,7 +12,7 @@ class FlatMapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest]
override def dependencies = List(parent) override def dependencies = List(parent)
override def slideTime: Time = parent.slideTime override def slideTime: Duration = parent.slideTime
override def compute(validTime: Time): Option[RDD[(K, U)]] = { override def compute(validTime: Time): Option[RDD[(K, U)]] = {
parent.getOrCompute(validTime).map(_.flatMapValues[U](flatMapValueFunc)) parent.getOrCompute(validTime).map(_.flatMapValues[U](flatMapValueFunc))

View file

@ -1,6 +1,6 @@
package spark.streaming.dstream package spark.streaming.dstream
import spark.streaming.{DStream, Time} import spark.streaming.{Duration, DStream, Time}
import spark.RDD import spark.RDD
private[streaming] private[streaming]
@ -11,7 +11,7 @@ class FlatMappedDStream[T: ClassManifest, U: ClassManifest](
override def dependencies = List(parent) override def dependencies = List(parent)
override def slideTime: Time = parent.slideTime override def slideTime: Duration = parent.slideTime
override def compute(validTime: Time): Option[RDD[U]] = { override def compute(validTime: Time): Option[RDD[U]] = {
parent.getOrCompute(validTime).map(_.flatMap(flatMapFunc)) parent.getOrCompute(validTime).map(_.flatMap(flatMapFunc))

View file

@ -1,7 +1,7 @@
package spark.streaming.dstream package spark.streaming.dstream
import spark.RDD import spark.RDD
import spark.streaming.{DStream, Job, Time} import spark.streaming.{Duration, DStream, Job, Time}
private[streaming] private[streaming]
class ForEachDStream[T: ClassManifest] ( class ForEachDStream[T: ClassManifest] (
@ -11,7 +11,7 @@ class ForEachDStream[T: ClassManifest] (
override def dependencies = List(parent) override def dependencies = List(parent)
override def slideTime: Time = parent.slideTime override def slideTime: Duration = parent.slideTime
override def compute(validTime: Time): Option[RDD[Unit]] = None override def compute(validTime: Time): Option[RDD[Unit]] = None

View file

@ -1,6 +1,6 @@
package spark.streaming.dstream package spark.streaming.dstream
import spark.streaming.{DStream, Time} import spark.streaming.{Duration, DStream, Time}
import spark.RDD import spark.RDD
private[streaming] private[streaming]
@ -9,7 +9,7 @@ class GlommedDStream[T: ClassManifest](parent: DStream[T])
override def dependencies = List(parent) override def dependencies = List(parent)
override def slideTime: Time = parent.slideTime override def slideTime: Duration = parent.slideTime
override def compute(validTime: Time): Option[RDD[Array[T]]] = { override def compute(validTime: Time): Option[RDD[Array[T]]] = {
parent.getOrCompute(validTime).map(_.glom()) parent.getOrCompute(validTime).map(_.glom())

View file

@ -1,13 +1,13 @@
package spark.streaming.dstream package spark.streaming.dstream
import spark.streaming.{StreamingContext, DStream} import spark.streaming.{Duration, StreamingContext, DStream}
abstract class InputDStream[T: ClassManifest] (@transient ssc_ : StreamingContext) abstract class InputDStream[T: ClassManifest] (@transient ssc_ : StreamingContext)
extends DStream[T](ssc_) { extends DStream[T](ssc_) {
override def dependencies = List() override def dependencies = List()
override def slideTime = { override def slideTime: Duration = {
if (ssc == null) throw new Exception("ssc is null") if (ssc == null) throw new Exception("ssc is null")
if (ssc.graph.batchDuration == null) throw new Exception("batchDuration is null") if (ssc.graph.batchDuration == null) throw new Exception("batchDuration is null")
ssc.graph.batchDuration ssc.graph.batchDuration

View file

@ -1,6 +1,6 @@
package spark.streaming.dstream package spark.streaming.dstream
import spark.streaming.{DStream, Time} import spark.streaming.{Duration, DStream, Time}
import spark.RDD import spark.RDD
private[streaming] private[streaming]
@ -12,7 +12,7 @@ class MapPartitionedDStream[T: ClassManifest, U: ClassManifest](
override def dependencies = List(parent) override def dependencies = List(parent)
override def slideTime: Time = parent.slideTime override def slideTime: Duration = parent.slideTime
override def compute(validTime: Time): Option[RDD[U]] = { override def compute(validTime: Time): Option[RDD[U]] = {
parent.getOrCompute(validTime).map(_.mapPartitions[U](mapPartFunc, preservePartitioning)) parent.getOrCompute(validTime).map(_.mapPartitions[U](mapPartFunc, preservePartitioning))

View file

@ -1,6 +1,6 @@
package spark.streaming.dstream package spark.streaming.dstream
import spark.streaming.{DStream, Time} import spark.streaming.{Duration, DStream, Time}
import spark.RDD import spark.RDD
import spark.SparkContext._ import spark.SparkContext._
@ -12,7 +12,7 @@ class MapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest](
override def dependencies = List(parent) override def dependencies = List(parent)
override def slideTime: Time = parent.slideTime override def slideTime: Duration = parent.slideTime
override def compute(validTime: Time): Option[RDD[(K, U)]] = { override def compute(validTime: Time): Option[RDD[(K, U)]] = {
parent.getOrCompute(validTime).map(_.mapValues[U](mapValueFunc)) parent.getOrCompute(validTime).map(_.mapValues[U](mapValueFunc))

View file

@ -1,6 +1,6 @@
package spark.streaming.dstream package spark.streaming.dstream
import spark.streaming.{DStream, Time} import spark.streaming.{Duration, DStream, Time}
import spark.RDD import spark.RDD
private[streaming] private[streaming]
@ -11,7 +11,7 @@ class MappedDStream[T: ClassManifest, U: ClassManifest] (
override def dependencies = List(parent) override def dependencies = List(parent)
override def slideTime: Time = parent.slideTime override def slideTime: Duration = parent.slideTime
override def compute(validTime: Time): Option[RDD[U]] = { override def compute(validTime: Time): Option[RDD[U]] = {
parent.getOrCompute(validTime).map(_.map[U](mapFunc)) parent.getOrCompute(validTime).map(_.map[U](mapFunc))

View file

@ -9,15 +9,15 @@ import spark.SparkContext._
import spark.storage.StorageLevel import spark.storage.StorageLevel
import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.ArrayBuffer
import spark.streaming.{Interval, Time, DStream} import spark.streaming.{Duration, Interval, Time, DStream}
private[streaming] private[streaming]
class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest]( class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
parent: DStream[(K, V)], parent: DStream[(K, V)],
reduceFunc: (V, V) => V, reduceFunc: (V, V) => V,
invReduceFunc: (V, V) => V, invReduceFunc: (V, V) => V,
_windowTime: Time, _windowTime: Duration,
_slideTime: Time, _slideTime: Duration,
partitioner: Partitioner partitioner: Partitioner
) extends DStream[(K,V)](parent.ssc) { ) extends DStream[(K,V)](parent.ssc) {
@ -39,15 +39,15 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
super.persist(StorageLevel.MEMORY_ONLY_SER) super.persist(StorageLevel.MEMORY_ONLY_SER)
reducedStream.persist(StorageLevel.MEMORY_ONLY_SER) reducedStream.persist(StorageLevel.MEMORY_ONLY_SER)
def windowTime: Time = _windowTime def windowTime: Duration = _windowTime
override def dependencies = List(reducedStream) override def dependencies = List(reducedStream)
override def slideTime: Time = _slideTime override def slideTime: Duration = _slideTime
override val mustCheckpoint = true override val mustCheckpoint = true
override def parentRememberDuration: Time = rememberDuration + windowTime override def parentRememberDuration: Duration = rememberDuration + windowTime
override def persist(storageLevel: StorageLevel): DStream[(K,V)] = { override def persist(storageLevel: StorageLevel): DStream[(K,V)] = {
super.persist(storageLevel) super.persist(storageLevel)
@ -55,7 +55,7 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
this this
} }
override def checkpoint(interval: Time): DStream[(K, V)] = { override def checkpoint(interval: Duration): DStream[(K, V)] = {
super.checkpoint(interval) super.checkpoint(interval)
//reducedStream.checkpoint(interval) //reducedStream.checkpoint(interval)
this this
@ -66,7 +66,7 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
val invReduceF = invReduceFunc val invReduceF = invReduceFunc
val currentTime = validTime val currentTime = validTime
val currentWindow = Interval(currentTime - windowTime + parent.slideTime, currentTime) val currentWindow = new Interval(currentTime - windowTime + parent.slideTime, currentTime)
val previousWindow = currentWindow - slideTime val previousWindow = currentWindow - slideTime
logDebug("Window time = " + windowTime) logDebug("Window time = " + windowTime)

View file

@ -2,7 +2,7 @@ package spark.streaming.dstream
import spark.{RDD, Partitioner} import spark.{RDD, Partitioner}
import spark.SparkContext._ import spark.SparkContext._
import spark.streaming.{DStream, Time} import spark.streaming.{Duration, DStream, Time}
private[streaming] private[streaming]
class ShuffledDStream[K: ClassManifest, V: ClassManifest, C: ClassManifest]( class ShuffledDStream[K: ClassManifest, V: ClassManifest, C: ClassManifest](
@ -15,7 +15,7 @@ class ShuffledDStream[K: ClassManifest, V: ClassManifest, C: ClassManifest](
override def dependencies = List(parent) override def dependencies = List(parent)
override def slideTime: Time = parent.slideTime override def slideTime: Duration = parent.slideTime
override def compute(validTime: Time): Option[RDD[(K,C)]] = { override def compute(validTime: Time): Option[RDD[(K,C)]] = {
parent.getOrCompute(validTime) match { parent.getOrCompute(validTime) match {

View file

@ -4,7 +4,7 @@ import spark.RDD
import spark.Partitioner import spark.Partitioner
import spark.SparkContext._ import spark.SparkContext._
import spark.storage.StorageLevel import spark.storage.StorageLevel
import spark.streaming.{Time, DStream} import spark.streaming.{Duration, Time, DStream}
private[streaming] private[streaming]
class StateDStream[K: ClassManifest, V: ClassManifest, S <: AnyRef : ClassManifest]( class StateDStream[K: ClassManifest, V: ClassManifest, S <: AnyRef : ClassManifest](
@ -18,7 +18,7 @@ class StateDStream[K: ClassManifest, V: ClassManifest, S <: AnyRef : ClassManife
override def dependencies = List(parent) override def dependencies = List(parent)
override def slideTime = parent.slideTime override def slideTime: Duration = parent.slideTime
override val mustCheckpoint = true override val mustCheckpoint = true

View file

@ -1,7 +1,7 @@
package spark.streaming.dstream package spark.streaming.dstream
import spark.RDD import spark.RDD
import spark.streaming.{DStream, Time} import spark.streaming.{Duration, DStream, Time}
private[streaming] private[streaming]
class TransformedDStream[T: ClassManifest, U: ClassManifest] ( class TransformedDStream[T: ClassManifest, U: ClassManifest] (
@ -11,7 +11,7 @@ class TransformedDStream[T: ClassManifest, U: ClassManifest] (
override def dependencies = List(parent) override def dependencies = List(parent)
override def slideTime: Time = parent.slideTime override def slideTime: Duration = parent.slideTime
override def compute(validTime: Time): Option[RDD[U]] = { override def compute(validTime: Time): Option[RDD[U]] = {
parent.getOrCompute(validTime).map(transformFunc(_, validTime)) parent.getOrCompute(validTime).map(transformFunc(_, validTime))

View file

@ -1,6 +1,6 @@
package spark.streaming.dstream package spark.streaming.dstream
import spark.streaming.{DStream, Time} import spark.streaming.{Duration, DStream, Time}
import spark.RDD import spark.RDD
import collection.mutable.ArrayBuffer import collection.mutable.ArrayBuffer
import spark.rdd.UnionRDD import spark.rdd.UnionRDD
@ -23,7 +23,7 @@ class UnionDStream[T: ClassManifest](parents: Array[DStream[T]])
override def dependencies = parents.toList override def dependencies = parents.toList
override def slideTime: Time = parents.head.slideTime override def slideTime: Duration = parents.head.slideTime
override def compute(validTime: Time): Option[RDD[T]] = { override def compute(validTime: Time): Option[RDD[T]] = {
val rdds = new ArrayBuffer[RDD[T]]() val rdds = new ArrayBuffer[RDD[T]]()

View file

@ -3,13 +3,13 @@ package spark.streaming.dstream
import spark.RDD import spark.RDD
import spark.rdd.UnionRDD import spark.rdd.UnionRDD
import spark.storage.StorageLevel import spark.storage.StorageLevel
import spark.streaming.{Interval, Time, DStream} import spark.streaming.{Duration, Interval, Time, DStream}
private[streaming] private[streaming]
class WindowedDStream[T: ClassManifest]( class WindowedDStream[T: ClassManifest](
parent: DStream[T], parent: DStream[T],
_windowTime: Time, _windowTime: Duration,
_slideTime: Time) _slideTime: Duration)
extends DStream[T](parent.ssc) { extends DStream[T](parent.ssc) {
if (!_windowTime.isMultipleOf(parent.slideTime)) if (!_windowTime.isMultipleOf(parent.slideTime))
@ -22,16 +22,16 @@ class WindowedDStream[T: ClassManifest](
parent.persist(StorageLevel.MEMORY_ONLY_SER) parent.persist(StorageLevel.MEMORY_ONLY_SER)
def windowTime: Time = _windowTime def windowTime: Duration = _windowTime
override def dependencies = List(parent) override def dependencies = List(parent)
override def slideTime: Time = _slideTime override def slideTime: Duration = _slideTime
override def parentRememberDuration: Time = rememberDuration + windowTime override def parentRememberDuration: Duration = rememberDuration + windowTime
override def compute(validTime: Time): Option[RDD[T]] = { override def compute(validTime: Time): Option[RDD[T]] = {
val currentWindow = Interval(validTime - windowTime + parent.slideTime, validTime) val currentWindow = new Interval(validTime - windowTime + parent.slideTime, validTime)
Some(new UnionRDD(ssc.sc, parent.slice(currentWindow))) Some(new UnionRDD(ssc.sc, parent.slice(currentWindow)))
} }
} }