Rename DStream.foreach to DStream.foreachRDD
`foreachRDD` makes it clear that the granularity of this operator is per-RDD. As it stands, `foreach` is inconsistent with with `map`, `filter`, and the other DStream operators which get pushed down to individual records within each RDD.
This commit is contained in:
parent
288a878999
commit
f4d77f8cb8
|
@ -175,7 +175,7 @@ When an output operator is called, it triggers the computation of a stream. Curr
|
|||
<table class="table">
|
||||
<tr><th style="width:30%">Operator</th><th>Meaning</th></tr>
|
||||
<tr>
|
||||
<td> <b>foreach</b>(<i>func</i>) </td>
|
||||
<td> <b>foreachRDD</b>(<i>func</i>) </td>
|
||||
<td> The fundamental output operator. Applies a function, <i>func</i>, to each RDD generated from the stream. This function should have side effects, such as printing output, saving the RDD to external files, or writing it over the network to an external system. </td>
|
||||
</tr>
|
||||
|
||||
|
@ -375,7 +375,7 @@ There are two failure behaviors based on which input sources are used.
|
|||
1. _Using HDFS files as input source_ - Since the data is reliably stored on HDFS, all data can re-computed and therefore no data will be lost due to any failure.
|
||||
1. _Using any input source that receives data through a network_ - For network-based data sources like Kafka and Flume, the received input data is replicated in memory between nodes of the cluster (default replication factor is 2). So if a worker node fails, then the system can recompute the lost from the the left over copy of the input data. However, if the worker node where a network receiver was running fails, then a tiny bit of data may be lost, that is, the data received by the system but not yet replicated to other node(s). The receiver will be started on a different node and it will continue to receive data.
|
||||
|
||||
Since all data is modeled as RDDs with their lineage of deterministic operations, any recomputation always leads to the same result. As a result, all DStream transformations are guaranteed to have _exactly-once_ semantics. That is, the final transformed result will be same even if there were was a worker node failure. However, output operations (like `foreach`) have _at-least once_ semantics, that is, the transformed data may get written to an external entity more than once in the event of a worker failure. While this is acceptable for saving to HDFS using the `saveAs*Files` operations (as the file will simply get over-written by the same data), additional transactions-like mechanisms may be necessary to achieve exactly-once semantics for output operations.
|
||||
Since all data is modeled as RDDs with their lineage of deterministic operations, any recomputation always leads to the same result. As a result, all DStream transformations are guaranteed to have _exactly-once_ semantics. That is, the final transformed result will be same even if there were was a worker node failure. However, output operations (like `foreachRDD`) have _at-least once_ semantics, that is, the transformed data may get written to an external entity more than once in the event of a worker failure. While this is acceptable for saving to HDFS using the `saveAs*Files` operations (as the file will simply get over-written by the same data), additional transactions-like mechanisms may be necessary to achieve exactly-once semantics for output operations.
|
||||
|
||||
## Failure of the Driver Node
|
||||
A system that is required to operate 24/7 needs to be able tolerate the failure of the driver node as well. Spark Streaming does this by saving the state of the DStream computation periodically to a HDFS file, that can be used to restart the streaming computation in the event of a failure of the driver node. This checkpointing is enabled by setting a HDFS directory for checkpointing using `ssc.checkpoint(<checkpoint directory>)` as described [earlier](#rdd-checkpointing-within-dstreams). To elaborate, the following state is periodically saved to a file.
|
||||
|
|
|
@ -58,7 +58,7 @@ object RawNetworkGrep {
|
|||
val rawStreams = (1 to numStreams).map(_ =>
|
||||
ssc.rawSocketStream[String](host, port, StorageLevel.MEMORY_ONLY_SER_2)).toArray
|
||||
val union = ssc.union(rawStreams)
|
||||
union.filter(_.contains("the")).count().foreach(r =>
|
||||
union.filter(_.contains("the")).count().foreachRDD(r =>
|
||||
println("Grep count: " + r.collect().mkString))
|
||||
ssc.start()
|
||||
}
|
||||
|
|
|
@ -81,7 +81,7 @@ object TwitterAlgebirdCMS {
|
|||
val exactTopUsers = users.map(id => (id, 1))
|
||||
.reduceByKey((a, b) => a + b)
|
||||
|
||||
approxTopUsers.foreach(rdd => {
|
||||
approxTopUsers.foreachRDD(rdd => {
|
||||
if (rdd.count() != 0) {
|
||||
val partial = rdd.first()
|
||||
val partialTopK = partial.heavyHitters.map(id =>
|
||||
|
@ -96,7 +96,7 @@ object TwitterAlgebirdCMS {
|
|||
}
|
||||
})
|
||||
|
||||
exactTopUsers.foreach(rdd => {
|
||||
exactTopUsers.foreachRDD(rdd => {
|
||||
if (rdd.count() != 0) {
|
||||
val partialMap = rdd.collect().toMap
|
||||
val partialTopK = rdd.map(
|
||||
|
|
|
@ -67,7 +67,7 @@ object TwitterAlgebirdHLL {
|
|||
|
||||
val exactUsers = users.map(id => Set(id)).reduce(_ ++ _)
|
||||
|
||||
approxUsers.foreach(rdd => {
|
||||
approxUsers.foreachRDD(rdd => {
|
||||
if (rdd.count() != 0) {
|
||||
val partial = rdd.first()
|
||||
globalHll += partial
|
||||
|
@ -76,7 +76,7 @@ object TwitterAlgebirdHLL {
|
|||
}
|
||||
})
|
||||
|
||||
exactUsers.foreach(rdd => {
|
||||
exactUsers.foreachRDD(rdd => {
|
||||
if (rdd.count() != 0) {
|
||||
val partial = rdd.first()
|
||||
userSet ++= partial
|
||||
|
|
|
@ -56,13 +56,13 @@ object TwitterPopularTags {
|
|||
|
||||
|
||||
// Print popular hashtags
|
||||
topCounts60.foreach(rdd => {
|
||||
topCounts60.foreachRDD(rdd => {
|
||||
val topList = rdd.take(5)
|
||||
println("\nPopular topics in last 60 seconds (%s total):".format(rdd.count()))
|
||||
topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))}
|
||||
})
|
||||
|
||||
topCounts10.foreach(rdd => {
|
||||
topCounts10.foreachRDD(rdd => {
|
||||
val topList = rdd.take(5)
|
||||
println("\nPopular topics in last 10 seconds (%s total):".format(rdd.count()))
|
||||
topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))}
|
||||
|
|
|
@ -91,7 +91,7 @@ object PageViewStream {
|
|||
case "popularUsersSeen" =>
|
||||
// Look for users in our existing dataset and print it out if we have a match
|
||||
pageViews.map(view => (view.userID, 1))
|
||||
.foreach((rdd, time) => rdd.join(userList)
|
||||
.foreachRDD((rdd, time) => rdd.join(userList)
|
||||
.map(_._2._2)
|
||||
.take(10)
|
||||
.foreach(u => println("Saw user %s at time %s".format(u, time))))
|
||||
|
|
|
@ -487,15 +487,15 @@ abstract class DStream[T: ClassTag] (
|
|||
* Apply a function to each RDD in this DStream. This is an output operator, so
|
||||
* 'this' DStream will be registered as an output stream and therefore materialized.
|
||||
*/
|
||||
def foreach(foreachFunc: RDD[T] => Unit) {
|
||||
this.foreach((r: RDD[T], t: Time) => foreachFunc(r))
|
||||
def foreachRDD(foreachFunc: RDD[T] => Unit) {
|
||||
this.foreachRDD((r: RDD[T], t: Time) => foreachFunc(r))
|
||||
}
|
||||
|
||||
/**
|
||||
* Apply a function to each RDD in this DStream. This is an output operator, so
|
||||
* 'this' DStream will be registered as an output stream and therefore materialized.
|
||||
*/
|
||||
def foreach(foreachFunc: (RDD[T], Time) => Unit) {
|
||||
def foreachRDD(foreachFunc: (RDD[T], Time) => Unit) {
|
||||
ssc.registerOutputStream(new ForEachDStream(this, context.sparkContext.clean(foreachFunc)))
|
||||
}
|
||||
|
||||
|
@ -719,7 +719,7 @@ abstract class DStream[T: ClassTag] (
|
|||
val file = rddToFileName(prefix, suffix, time)
|
||||
rdd.saveAsObjectFile(file)
|
||||
}
|
||||
this.foreach(saveFunc)
|
||||
this.foreachRDD(saveFunc)
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -732,7 +732,7 @@ abstract class DStream[T: ClassTag] (
|
|||
val file = rddToFileName(prefix, suffix, time)
|
||||
rdd.saveAsTextFile(file)
|
||||
}
|
||||
this.foreach(saveFunc)
|
||||
this.foreachRDD(saveFunc)
|
||||
}
|
||||
|
||||
def register() {
|
||||
|
|
|
@ -582,7 +582,7 @@ extends Serializable {
|
|||
val file = rddToFileName(prefix, suffix, time)
|
||||
rdd.saveAsHadoopFile(file, keyClass, valueClass, outputFormatClass, conf)
|
||||
}
|
||||
self.foreach(saveFunc)
|
||||
self.foreachRDD(saveFunc)
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -612,7 +612,7 @@ extends Serializable {
|
|||
val file = rddToFileName(prefix, suffix, time)
|
||||
rdd.saveAsNewAPIHadoopFile(file, keyClass, valueClass, outputFormatClass, conf)
|
||||
}
|
||||
self.foreach(saveFunc)
|
||||
self.foreachRDD(saveFunc)
|
||||
}
|
||||
|
||||
private def getKeyClass() = implicitly[ClassTag[K]].runtimeClass
|
||||
|
|
|
@ -244,16 +244,16 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
|
|||
* Apply a function to each RDD in this DStream. This is an output operator, so
|
||||
* 'this' DStream will be registered as an output stream and therefore materialized.
|
||||
*/
|
||||
def foreach(foreachFunc: JFunction[R, Void]) {
|
||||
dstream.foreach(rdd => foreachFunc.call(wrapRDD(rdd)))
|
||||
def foreachRDD(foreachFunc: JFunction[R, Void]) {
|
||||
dstream.foreachRDD(rdd => foreachFunc.call(wrapRDD(rdd)))
|
||||
}
|
||||
|
||||
/**
|
||||
* Apply a function to each RDD in this DStream. This is an output operator, so
|
||||
* 'this' DStream will be registered as an output stream and therefore materialized.
|
||||
*/
|
||||
def foreach(foreachFunc: JFunction2[R, Time, Void]) {
|
||||
dstream.foreach((rdd, time) => foreachFunc.call(wrapRDD(rdd), time))
|
||||
def foreachRDD(foreachFunc: JFunction2[R, Time, Void]) {
|
||||
dstream.foreachRDD((rdd, time) => foreachFunc.call(wrapRDD(rdd), time))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -383,7 +383,7 @@ class BasicOperationsSuite extends TestSuiteBase {
|
|||
val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4))
|
||||
val stream = new TestInputStream[Int](ssc, input, 2)
|
||||
ssc.registerInputStream(stream)
|
||||
stream.foreach(_ => {}) // Dummy output stream
|
||||
stream.foreachRDD(_ => {}) // Dummy output stream
|
||||
ssc.start()
|
||||
Thread.sleep(2000)
|
||||
def getInputFromSlice(fromMillis: Long, toMillis: Long) = {
|
||||
|
|
Loading…
Reference in a new issue