aboutsummaryrefslogtreecommitdiff
path: root/streaming/src
diff options
context:
space:
mode:
authorTathagata Das <tathagata.das1565@gmail.com>2013-01-13 21:08:35 -0800
committerTathagata Das <tathagata.das1565@gmail.com>2013-01-13 21:08:35 -0800
commit0dbd411a562396e024c513936fde46b0d2f6d59d (patch)
treea73a7874d1c46bfe90bb31bc51179aac9de513b0 /streaming/src
parent0a2e33334125cb3ae5e54f8333ea5608779399fc (diff)
downloadspark-0dbd411a562396e024c513936fde46b0d2f6d59d.tar.gz
spark-0dbd411a562396e024c513936fde46b0d2f6d59d.tar.bz2
spark-0dbd411a562396e024c513936fde46b0d2f6d59d.zip
Added documentation for PairDStreamFunctions.
Diffstat (limited to 'streaming/src')
-rw-r--r--streaming/src/main/scala/spark/streaming/DStream.scala35
-rw-r--r--streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala293
-rw-r--r--streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala2
3 files changed, 303 insertions, 27 deletions
diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala
index c89fb7723e..d94548a4f3 100644
--- a/streaming/src/main/scala/spark/streaming/DStream.scala
+++ b/streaming/src/main/scala/spark/streaming/DStream.scala
@@ -471,7 +471,7 @@ abstract class DStream[T: ClassManifest] (
* Returns a new DStream in which each RDD has a single element generated by counting each RDD
* of this DStream.
*/
- def count(): DStream[Int] = this.map(_ => 1).reduce(_ + _)
+ def count(): DStream[Long] = this.map(_ => 1L).reduce(_ + _)
/**
* Applies a function to each RDD in this DStream. This is an output operator, so
@@ -529,17 +529,16 @@ abstract class DStream[T: ClassManifest] (
* Return a new DStream which is computed based on windowed batches of this DStream.
* The new DStream generates RDDs with the same interval as this DStream.
* @param windowDuration width of the window; must be a multiple of this DStream's interval.
- * @return
*/
def window(windowDuration: Duration): DStream[T] = window(windowDuration, this.slideDuration)
/**
* Return a new DStream which is computed based on windowed batches of this DStream.
- * @param windowDuration duration (i.e., width) of the window;
- * must be a multiple of this DStream's interval
+ * @param windowDuration width of the window; must be a multiple of this DStream's
+ * batching interval
* @param slideDuration sliding interval of the window (i.e., the interval after which
- * the new DStream will generate RDDs); must be a multiple of this
- * DStream's interval
+ * the new DStream will generate RDDs); must be a multiple of this
+ * DStream's batching interval
*/
def window(windowDuration: Duration, slideDuration: Duration): DStream[T] = {
new WindowedDStream(this, windowDuration, slideDuration)
@@ -548,16 +547,22 @@ abstract class DStream[T: ClassManifest] (
/**
* Returns a new DStream which computed based on tumbling window on this DStream.
* 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
+ * batching interval
*/
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
- * elements in a window over this DStream. windowDuration and slideDuration are as defined in the
- * window() operation. This is equivalent to window(windowDuration, slideDuration).reduce(reduceFunc)
+ * elements in a window over this DStream. windowDuration and slideDuration are as defined
+ * in the window() operation. This is equivalent to
+ * window(windowDuration, slideDuration).reduce(reduceFunc)
*/
- def reduceByWindow(reduceFunc: (T, T) => T, windowDuration: Duration, slideDuration: Duration): DStream[T] = {
+ def reduceByWindow(
+ reduceFunc: (T, T) => T,
+ windowDuration: Duration,
+ slideDuration: Duration
+ ): DStream[T] = {
this.window(windowDuration, slideDuration).reduce(reduceFunc)
}
@@ -577,8 +582,8 @@ abstract class DStream[T: ClassManifest] (
* of elements in a window over this DStream. windowDuration and slideDuration are as defined in the
* window() operation. This is equivalent to window(windowDuration, slideDuration).count()
*/
- def countByWindow(windowDuration: Duration, slideDuration: Duration): DStream[Int] = {
- this.map(_ => 1).reduceByWindow(_ + _, _ - _, windowDuration, slideDuration)
+ def countByWindow(windowDuration: Duration, slideDuration: Duration): DStream[Long] = {
+ this.map(_ => 1L).reduceByWindow(_ + _, _ - _, windowDuration, slideDuration)
}
/**
@@ -612,6 +617,8 @@ abstract class DStream[T: ClassManifest] (
/**
* Saves each RDD in this DStream as a Sequence file of serialized objects.
+ * The file name at each batch interval is generated based on `prefix` and
+ * `suffix`: "prefix-TIME_IN_MS.suffix".
*/
def saveAsObjectFiles(prefix: String, suffix: String = "") {
val saveFunc = (rdd: RDD[T], time: Time) => {
@@ -622,7 +629,9 @@ abstract class DStream[T: ClassManifest] (
}
/**
- * Saves each RDD in this DStream as at text file, using string representation of elements.
+ * Saves each RDD in this DStream as at text file, using string representation
+ * of elements. The file name at each batch interval is generated based on
+ * `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
*/
def saveAsTextFiles(prefix: String, suffix: String = "") {
val saveFunc = (rdd: RDD[T], time: Time) => {
diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala
index 482d01300d..3952457339 100644
--- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala
+++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala
@@ -25,34 +25,76 @@ extends Serializable {
new HashPartitioner(numPartitions)
}
+ /**
+ * Creates a new DStream by applying `groupByKey` on each RDD of `this` DStream.
+ * Therefore, the values for each key in `this` DStream's RDDs are grouped into a
+ * single sequence to generate the RDDs of the new DStream. Hash partitioning is
+ * used to generate the RDDs with Spark's default number of partitions.
+ */
def groupByKey(): DStream[(K, Seq[V])] = {
groupByKey(defaultPartitioner())
}
+ /**
+ * Creates a new DStream by applying `groupByKey` on each RDD of `this` DStream.
+ * Therefore, the values for each key in `this` DStream's RDDs are grouped into a
+ * single sequence to generate the RDDs of the new DStream. Hash partitioning is
+ * used to generate the RDDs with `numPartitions` partitions.
+ */
def groupByKey(numPartitions: Int): DStream[(K, Seq[V])] = {
groupByKey(defaultPartitioner(numPartitions))
}
+ /**
+ * Creates a new DStream by applying `groupByKey` on each RDD of `this` DStream.
+ * Therefore, the values for each key in `this` DStream's RDDs are grouped into a
+ * single sequence to generate the RDDs of the new DStream. [[spark.Partitioner]]
+ * is used to control the partitioning of each RDD.
+ */
def groupByKey(partitioner: Partitioner): DStream[(K, Seq[V])] = {
val createCombiner = (v: V) => ArrayBuffer[V](v)
val mergeValue = (c: ArrayBuffer[V], v: V) => (c += v)
val mergeCombiner = (c1: ArrayBuffer[V], c2: ArrayBuffer[V]) => (c1 ++ c2)
- combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner).asInstanceOf[DStream[(K, Seq[V])]]
+ combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner)
+ .asInstanceOf[DStream[(K, Seq[V])]]
}
+ /**
+ * Creates a new DStream by applying `reduceByKey` on each RDD of `this` DStream.
+ * Therefore, the values for each key in `this` DStream's RDDs is merged using the
+ * associative reduce function to generate the RDDs of the new DStream.
+ * Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
+ */
def reduceByKey(reduceFunc: (V, V) => V): DStream[(K, V)] = {
reduceByKey(reduceFunc, defaultPartitioner())
}
+ /**
+ * Creates a new DStream by applying `reduceByKey` on each RDD of `this` DStream.
+ * Therefore, the values for each key in `this` DStream's RDDs is merged using the
+ * associative reduce function to generate the RDDs of the new DStream.
+ * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+ */
def reduceByKey(reduceFunc: (V, V) => V, numPartitions: Int): DStream[(K, V)] = {
reduceByKey(reduceFunc, defaultPartitioner(numPartitions))
}
+ /**
+ * Creates a new DStream by applying `reduceByKey` on each RDD of `this` DStream.
+ * Therefore, the values for each key in `this` DStream's RDDs is merged using the
+ * associative reduce function to generate the RDDs of the new DStream.
+ * [[spark.Partitioner]] is used to control the partitioning of each RDD.
+ */
def reduceByKey(reduceFunc: (V, V) => V, partitioner: Partitioner): DStream[(K, V)] = {
val cleanedReduceFunc = ssc.sc.clean(reduceFunc)
combineByKey((v: V) => v, cleanedReduceFunc, cleanedReduceFunc, partitioner)
}
+ /**
+ * Generic function to combine elements of each key in DStream's RDDs using custom function.
+ * This is similar to the combineByKey for RDDs. Please refer to combineByKey in
+ * [[spark.PairRDDFunctions]] for more information.
+ */
def combineByKey[C: ClassManifest](
createCombiner: V => C,
mergeValue: (C, V) => C,
@@ -61,14 +103,52 @@ extends Serializable {
new ShuffledDStream[K, V, C](self, createCombiner, mergeValue, mergeCombiner, partitioner)
}
+ /**
+ * Creates a new DStream by counting the number of values of each key in each RDD
+ * of `this` DStream. Hash partitioning is used to generate the RDDs with Spark's
+ * `numPartitions` partitions.
+ */
def countByKey(numPartitions: Int = self.ssc.sc.defaultParallelism): DStream[(K, Long)] = {
self.map(x => (x._1, 1L)).reduceByKey((x: Long, y: Long) => x + y, numPartitions)
}
+ /**
+ * Creates a new DStream by applying `groupByKey` over a sliding window on `this` DStream.
+ * This is similar to `DStream.groupByKey()` but applies it over a sliding window.
+ * The new DStream generates RDDs with the same interval as this DStream.
+ * Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
+ * @param windowDuration width of the window; must be a multiple of this DStream's
+ * batching interval
+ */
+ def groupByKeyAndWindow(windowDuration: Duration): DStream[(K, Seq[V])] = {
+ groupByKeyAndWindow(windowDuration, self.slideDuration, defaultPartitioner())
+ }
+
+ /**
+ * Creates a new DStream by applying `groupByKey` over a sliding window on `this` DStream.
+ * This is similar to `DStream.groupByKey()` but applies it over a sliding window.
+ * Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
+ * @param windowDuration width of the window; must be a multiple of this DStream's
+ * batching interval
+ * @param slideDuration sliding interval of the window (i.e., the interval after which
+ * the new DStream will generate RDDs); must be a multiple of this
+ * DStream's batching interval
+ */
def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration): DStream[(K, Seq[V])] = {
groupByKeyAndWindow(windowDuration, slideDuration, defaultPartitioner())
}
+ /**
+ * Creates a new DStream by applying `groupByKey` over a sliding window on `this` DStream.
+ * This is similar to `DStream.groupByKey()` but applies it over a sliding window.
+ * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+ * @param windowDuration width of the window; must be a multiple of this DStream's
+ * batching interval
+ * @param slideDuration sliding interval of the window (i.e., the interval after which
+ * the new DStream will generate RDDs); must be a multiple of this
+ * DStream's batching interval
+ * @param numPartitions Number of partitions of each RDD in the new DStream.
+ */
def groupByKeyAndWindow(
windowDuration: Duration,
slideDuration: Duration,
@@ -77,6 +157,16 @@ extends Serializable {
groupByKeyAndWindow(windowDuration, slideDuration, defaultPartitioner(numPartitions))
}
+ /**
+ * Creates a new DStream by applying `groupByKey` over a sliding window on `this` DStream.
+ * This is similar to `DStream.groupByKey()` but applies it over a sliding window.
+ * @param windowDuration width of the window; must be a multiple of this DStream's
+ * batching interval
+ * @param slideDuration sliding interval of the window (i.e., the interval after which
+ * the new DStream will generate RDDs); must be a multiple of this
+ * DStream's batching interval
+ * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+ */
def groupByKeyAndWindow(
windowDuration: Duration,
slideDuration: Duration,
@@ -85,6 +175,15 @@ extends Serializable {
self.window(windowDuration, slideDuration).groupByKey(partitioner)
}
+ /**
+ * Creates a new DStream by applying `reduceByKey` over a sliding window on `this` DStream.
+ * This is similar to `DStream.reduceByKey()` but applies it over a sliding window.
+ * The new DStream generates RDDs with the same interval as this DStream.
+ * Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
+ * @param reduceFunc associative reduce function
+ * @param windowDuration width of the window; must be a multiple of this DStream's
+ * batching interval
+ */
def reduceByKeyAndWindow(
reduceFunc: (V, V) => V,
windowDuration: Duration
@@ -92,6 +191,17 @@ extends Serializable {
reduceByKeyAndWindow(reduceFunc, windowDuration, self.slideDuration, defaultPartitioner())
}
+ /**
+ * Creates a new DStream by applying `reduceByKey` over a sliding window on `this` DStream.
+ * This is similar to `DStream.reduceByKey()` but applies it over a sliding window.
+ * Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
+ * @param reduceFunc associative reduce function
+ * @param windowDuration width of the window; must be a multiple of this DStream's
+ * batching interval
+ * @param slideDuration sliding interval of the window (i.e., the interval after which
+ * the new DStream will generate RDDs); must be a multiple of this
+ * DStream's batching interval
+ */
def reduceByKeyAndWindow(
reduceFunc: (V, V) => V,
windowDuration: Duration,
@@ -100,6 +210,18 @@ extends Serializable {
reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, defaultPartitioner())
}
+ /**
+ * Creates a new DStream by applying `reduceByKey` over a sliding window on `this` DStream.
+ * This is similar to `DStream.reduceByKey()` but applies it over a sliding window.
+ * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+ * @param reduceFunc associative reduce function
+ * @param windowDuration width of the window; must be a multiple of this DStream's
+ * batching interval
+ * @param slideDuration sliding interval of the window (i.e., the interval after which
+ * the new DStream will generate RDDs); must be a multiple of this
+ * DStream's batching interval
+ * @param numPartitions Number of partitions of each RDD in the new DStream.
+ */
def reduceByKeyAndWindow(
reduceFunc: (V, V) => V,
windowDuration: Duration,
@@ -109,6 +231,17 @@ extends Serializable {
reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, defaultPartitioner(numPartitions))
}
+ /**
+ * Creates a new DStream by applying `reduceByKey` over a sliding window on `this` DStream.
+ * This is similar to `DStream.reduceByKey()` but applies it over a sliding window.
+ * @param reduceFunc associative reduce function
+ * @param windowDuration width of the window; must be a multiple of this DStream's
+ * batching interval
+ * @param slideDuration sliding interval of the window (i.e., the interval after which
+ * the new DStream will generate RDDs); must be a multiple of this
+ * DStream's batching interval
+ * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+ */
def reduceByKeyAndWindow(
reduceFunc: (V, V) => V,
windowDuration: Duration,
@@ -121,12 +254,23 @@ extends Serializable {
.reduceByKey(cleanedReduceFunc, partitioner)
}
- // This method is the efficient sliding window reduce operation,
- // which requires the specification of an inverse reduce function,
- // so that new elements introduced in the window can be "added" using
- // reduceFunc to the previous window's result and old elements can be
- // "subtracted using invReduceFunc.
-
+ /**
+ * Creates a new DStream by reducing over a window in a smarter way.
+ * The reduced value of over a new window is calculated incrementally by using the
+ * old window's reduce value :
+ * 1. reduce the new values that entered the window (e.g., adding new counts)
+ * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
+ * This is more efficient that reduceByKeyAndWindow without "inverse reduce" function.
+ * However, it is applicable to only "invertible reduce functions".
+ * Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
+ * @param reduceFunc associative reduce function
+ * @param invReduceFunc inverse function
+ * @param windowDuration width of the window; must be a multiple of this DStream's
+ * batching interval
+ * @param slideDuration sliding interval of the window (i.e., the interval after which
+ * the new DStream will generate RDDs); must be a multiple of this
+ * DStream's batching interval
+ */
def reduceByKeyAndWindow(
reduceFunc: (V, V) => V,
invReduceFunc: (V, V) => V,
@@ -138,6 +282,24 @@ extends Serializable {
reduceFunc, invReduceFunc, windowDuration, slideDuration, defaultPartitioner())
}
+ /**
+ * Creates a new DStream by reducing over a window in a smarter way.
+ * The reduced value of over a new window is calculated incrementally by using the
+ * old window's reduce value :
+ * 1. reduce the new values that entered the window (e.g., adding new counts)
+ * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
+ * This is more efficient that reduceByKeyAndWindow without "inverse reduce" function.
+ * However, it is applicable to only "invertible reduce functions".
+ * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+ * @param reduceFunc associative reduce function
+ * @param invReduceFunc inverse function
+ * @param windowDuration width of the window; must be a multiple of this DStream's
+ * batching interval
+ * @param slideDuration sliding interval of the window (i.e., the interval after which
+ * the new DStream will generate RDDs); must be a multiple of this
+ * DStream's batching interval
+ * @param numPartitions Number of partitions of each RDD in the new DStream.
+ */
def reduceByKeyAndWindow(
reduceFunc: (V, V) => V,
invReduceFunc: (V, V) => V,
@@ -150,6 +312,23 @@ extends Serializable {
reduceFunc, invReduceFunc, windowDuration, slideDuration, defaultPartitioner(numPartitions))
}
+ /**
+ * Creates a new DStream by reducing over a window in a smarter way.
+ * The reduced value of over a new window is calculated incrementally by using the
+ * old window's reduce value :
+ * 1. reduce the new values that entered the window (e.g., adding new counts)
+ * 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
+ * This is more efficient that reduceByKeyAndWindow without "inverse reduce" function.
+ * However, it is applicable to only "invertible reduce functions".
+ * @param reduceFunc associative reduce function
+ * @param invReduceFunc inverse function
+ * @param windowDuration width of the window; must be a multiple of this DStream's
+ * batching interval
+ * @param slideDuration sliding interval of the window (i.e., the interval after which
+ * the new DStream will generate RDDs); must be a multiple of this
+ * DStream's batching interval
+ * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+ */
def reduceByKeyAndWindow(
reduceFunc: (V, V) => V,
invReduceFunc: (V, V) => V,
@@ -164,6 +343,16 @@ extends Serializable {
self, cleanedReduceFunc, cleanedInvReduceFunc, windowDuration, slideDuration, partitioner)
}
+ /**
+ * Creates a new DStream by counting the number of values for each key over a window.
+ * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+ * @param windowDuration width of the window; must be a multiple of this DStream's
+ * batching interval
+ * @param slideDuration sliding interval of the window (i.e., the interval after which
+ * the new DStream will generate RDDs); must be a multiple of this
+ * DStream's batching interval
+ * @param numPartitions Number of partitions of each RDD in the new DStream.
+ */
def countByKeyAndWindow(
windowDuration: Duration,
slideDuration: Duration,
@@ -179,17 +368,30 @@ extends Serializable {
)
}
- // TODO:
- //
- //
- //
- //
+ /**
+ * Creates a new "state" DStream where the state for each key is updated by applying
+ * the given function on the previous state of the key and the new values of the key from
+ * `this` DStream. Hash partitioning is used to generate the RDDs with Spark's default
+ * number of partitions.
+ * @param updateFunc State update function. If `this` function returns None, then
+ * corresponding state key-value pair will be eliminated.
+ * @tparam S State type
+ */
def updateStateByKey[S <: AnyRef : ClassManifest](
updateFunc: (Seq[V], Option[S]) => Option[S]
): DStream[(K, S)] = {
updateStateByKey(updateFunc, defaultPartitioner())
}
+ /**
+ * Creates a new "state" DStream where the state for each key is updated by applying
+ * the given function on the previous state of the key and the new values of the key from
+ * `this` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+ * @param updateFunc State update function. If `this` function returns None, then
+ * corresponding state key-value pair will be eliminated.
+ * @param numPartitions Number of partitions of each RDD in the new DStream.
+ * @tparam S State type
+ */
def updateStateByKey[S <: AnyRef : ClassManifest](
updateFunc: (Seq[V], Option[S]) => Option[S],
numPartitions: Int
@@ -197,6 +399,15 @@ extends Serializable {
updateStateByKey(updateFunc, defaultPartitioner(numPartitions))
}
+ /**
+ * Creates a new "state" DStream where the state for each key is updated by applying
+ * the given function on the previous state of the key and the new values of the key from
+ * `this` DStream. [[spark.Partitioner]] is used to control the partitioning of each RDD.
+ * @param updateFunc State update function. If `this` function returns None, then
+ * corresponding state key-value pair will be eliminated.
+ * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+ * @tparam S State type
+ */
def updateStateByKey[S <: AnyRef : ClassManifest](
updateFunc: (Seq[V], Option[S]) => Option[S],
partitioner: Partitioner
@@ -207,6 +418,19 @@ extends Serializable {
updateStateByKey(newUpdateFunc, partitioner, true)
}
+ /**
+ * Creates a new "state" DStream where the state for each key is updated by applying
+ * the given function on the previous state of the key and the new values of the key from
+ * `this` DStream. [[spark.Partitioner]] is used to control the partitioning of each RDD.
+ * @param updateFunc State update function. If `this` function returns None, then
+ * corresponding state key-value pair will be eliminated. Note, that
+ * this function may generate a different a tuple with a different key
+ * than the input key. It is up to the developer to decide whether to
+ * remember the partitioner despite the key being changed.
+ * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+ * @param rememberPartitioner Whether to remember the paritioner object in the generated RDDs.
+ * @tparam S State type
+ */
def updateStateByKey[S <: AnyRef : ClassManifest](
updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)],
partitioner: Partitioner,
@@ -226,10 +450,24 @@ extends Serializable {
new FlatMapValuedDStream[K, V, U](self, flatMapValuesFunc)
}
+ /**
+ * Cogroups `this` DStream with `other` DStream. Each RDD of the new DStream will
+ * be generated by cogrouping RDDs from`this`and `other` DStreams. Therefore, for
+ * each key k in corresponding RDDs of `this` or `other` DStreams, the generated RDD
+ * will contains a tuple with the list of values for that key in both RDDs.
+ * HashPartitioner is used to partition each generated RDD into default number of partitions.
+ */
def cogroup[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (Seq[V], Seq[W]))] = {
cogroup(other, defaultPartitioner())
}
+ /**
+ * Cogroups `this` DStream with `other` DStream. Each RDD of the new DStream will
+ * be generated by cogrouping RDDs from`this`and `other` DStreams. Therefore, for
+ * each key k in corresponding RDDs of `this` or `other` DStreams, the generated RDD
+ * will contains a tuple with the list of values for that key in both RDDs.
+ * Partitioner is used to partition each generated RDD.
+ */
def cogroup[W: ClassManifest](
other: DStream[(K, W)],
partitioner: Partitioner
@@ -249,11 +487,24 @@ extends Serializable {
}
}
+ /**
+ * Joins `this` DStream with `other` DStream. Each RDD of the new DStream will
+ * be generated by joining RDDs from `this` and `other` DStreams. HashPartitioner is used
+ * to partition each generated RDD into default number of partitions.
+ */
def join[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (V, W))] = {
join[W](other, defaultPartitioner())
}
- def join[W: ClassManifest](other: DStream[(K, W)], partitioner: Partitioner): DStream[(K, (V, W))] = {
+ /**
+ * Joins `this` DStream with `other` DStream, that is, each RDD of the new DStream will
+ * be generated by joining RDDs from `this` and other DStream. Uses the given
+ * Partitioner to partition each generated RDD.
+ */
+ def join[W: ClassManifest](
+ other: DStream[(K, W)],
+ partitioner: Partitioner
+ ): DStream[(K, (V, W))] = {
this.cogroup(other, partitioner)
.flatMapValues{
case (vs, ws) =>
@@ -261,6 +512,10 @@ extends Serializable {
}
}
+ /**
+ * Saves each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is generated
+ * based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix"
+ */
def saveAsHadoopFiles[F <: OutputFormat[K, V]](
prefix: String,
suffix: String
@@ -268,6 +523,10 @@ extends Serializable {
saveAsHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]])
}
+ /**
+ * Saves each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is generated
+ * based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix"
+ */
def saveAsHadoopFiles(
prefix: String,
suffix: String,
@@ -283,6 +542,10 @@ extends Serializable {
self.foreach(saveFunc)
}
+ /**
+ * Saves each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is generated
+ * based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+ */
def saveAsNewAPIHadoopFiles[F <: NewOutputFormat[K, V]](
prefix: String,
suffix: String
@@ -290,6 +553,10 @@ extends Serializable {
saveAsNewAPIHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]])
}
+ /**
+ * Saves each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is generated
+ * based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+ */
def saveAsNewAPIHadoopFiles(
prefix: String,
suffix: String,
diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala
index f31ae39a16..03749d4a94 100644
--- a/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala
+++ b/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala
@@ -81,7 +81,7 @@ object RawTextHelper {
* before real workload starts.
*/
def warmUp(sc: SparkContext) {
- for(i <- 0 to 4) {
+ for(i <- 0 to 1) {
sc.parallelize(1 to 200000, 1000)
.map(_ % 1331).map(_.toString)
.mapPartitions(splitAndCountPartitions).reduceByKey(_ + _, 10)