aboutsummaryrefslogtreecommitdiff
path: root/docs/streaming-programming-guide.md
diff options
context:
space:
mode:
authorTathagata Das <tathagata.das1565@gmail.com>2015-06-12 15:22:59 -0700
committerTathagata Das <tathagata.das1565@gmail.com>2015-06-12 15:22:59 -0700
commite9471d3414d327c7d0853e18f1844ab1bd09c8ed (patch)
treec71dabecf459ff18b1c974c40f6b8f2f8924a9ce /docs/streaming-programming-guide.md
parent88604051511c788d7abb41a49e3eb3a8330c09a9 (diff)
downloadspark-e9471d3414d327c7d0853e18f1844ab1bd09c8ed.tar.gz
spark-e9471d3414d327c7d0853e18f1844ab1bd09c8ed.tar.bz2
spark-e9471d3414d327c7d0853e18f1844ab1bd09c8ed.zip
[SPARK-7284] [STREAMING] Updated streaming documentation
- Kinesis API updated - Kafka version updated, and Python API for Direct Kafka added - Added SQLContext.getOrCreate() - Added information on how to get partitionId in foreachRDD Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #6781 from tdas/SPARK-7284 and squashes the following commits: aac7be0 [Tathagata Das] Added information on how to get partition id a66ec22 [Tathagata Das] Complete the line incomplete line, a92ca39 [Tathagata Das] Updated streaming documentation
Diffstat (limited to 'docs/streaming-programming-guide.md')
-rw-r--r--docs/streaming-programming-guide.md70
1 files changed, 24 insertions, 46 deletions
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
index 42b3394787..836f047359 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -77,7 +77,7 @@ main entry point for all streaming functionality. We create a local StreamingCon
{% highlight scala %}
import org.apache.spark._
import org.apache.spark.streaming._
-import org.apache.spark.streaming.StreamingContext._ // not necessary in Spark 1.3+
+import org.apache.spark.streaming.StreamingContext._ // not necessary since Spark 1.3
// Create a local StreamingContext with two working thread and batch interval of 1 second.
// The master requires 2 cores to prevent from a starvation scenario.
@@ -109,7 +109,7 @@ each line will be split into multiple words and the stream of words is represent
`words` DStream. Next, we want to count these words.
{% highlight scala %}
-import org.apache.spark.streaming.StreamingContext._ // not necessary in Spark 1.3+
+import org.apache.spark.streaming.StreamingContext._ // not necessary since Spark 1.3
// Count each word in each batch
val pairs = words.map(word => (word, 1))
val wordCounts = pairs.reduceByKey(_ + _)
@@ -682,7 +682,7 @@ for Java, and [StreamingContext](api/python/pyspark.streaming.html#pyspark.strea
### Advanced Sources
{:.no_toc}
-<span class="badge" style="background-color: grey">Python API</span> As of Spark 1.3,
+<span class="badge" style="background-color: grey">Python API</span> As of Spark {{site.SPARK_VERSION_SHORT}},
out of these sources, *only* Kafka is available in the Python API. We will add more advanced sources in the Python API in future.
This category of sources require interfacing with external non-Spark libraries, some of them with
@@ -723,7 +723,7 @@ and it in the classpath.
Some of these advanced sources are as follows.
-- **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kafka 0.8.1.1. See the [Kafka Integration Guide](streaming-kafka-integration.html) for more details.
+- **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kafka 0.8.2.1. See the [Kafka Integration Guide](streaming-kafka-integration.html) for more details.
- **Flume:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Flume 1.4.0. See the [Flume Integration Guide](streaming-flume-integration.html) for more details.
@@ -991,8 +991,9 @@ cleanedDStream = wordCounts.transform(lambda rdd: rdd.join(spamInfoRDD).filter(.
</div>
</div>
-In fact, you can also use [machine learning](mllib-guide.html) and
-[graph computation](graphx-programming-guide.html) algorithms in the `transform` method.
+Note that the supplied function gets called in every batch interval. This allows you to do
+time-varying RDD operations, that is, RDD operations, number of partitions, broadcast variables,
+etc. can be changed between batches.
#### Window Operations
{:.no_toc}
@@ -1427,26 +1428,6 @@ You can easily use [DataFrames and SQL](sql-programming-guide.html) operations o
<div data-lang="scala" markdown="1">
{% highlight scala %}
-/** Lazily instantiated singleton instance of SQLContext */
-object SQLContextSingleton {
- @transient private var instance: SQLContext = null
-
- // Instantiate SQLContext on demand
- def getInstance(sparkContext: SparkContext): SQLContext = synchronized {
- if (instance == null) {
- instance = new SQLContext(sparkContext)
- }
- instance
- }
-}
-
-...
-
-/** Case class for converting RDD to DataFrame */
-case class Row(word: String)
-
-...
-
/** DataFrame operations inside your streaming program */
val words: DStream[String] = ...
@@ -1454,11 +1435,11 @@ val words: DStream[String] = ...
words.foreachRDD { rdd =>
// Get the singleton instance of SQLContext
- val sqlContext = SQLContextSingleton.getInstance(rdd.sparkContext)
+ val sqlContext = SQLContext.getOrCreate(rdd.sparkContext)
import sqlContext.implicits._
- // Convert RDD[String] to RDD[case class] to DataFrame
- val wordsDataFrame = rdd.map(w => Row(w)).toDF()
+ // Convert RDD[String] to DataFrame
+ val wordsDataFrame = rdd.toDF("word")
// Register as table
wordsDataFrame.registerTempTable("words")
@@ -1476,19 +1457,6 @@ See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/ma
<div data-lang="java" markdown="1">
{% highlight java %}
-/** Lazily instantiated singleton instance of SQLContext */
-class JavaSQLContextSingleton {
- static private transient SQLContext instance = null;
- static public SQLContext getInstance(SparkContext sparkContext) {
- if (instance == null) {
- instance = new SQLContext(sparkContext);
- }
- return instance;
- }
-}
-
-...
-
/** Java Bean class for converting RDD to DataFrame */
public class JavaRow implements java.io.Serializable {
private String word;
@@ -1512,7 +1480,9 @@ words.foreachRDD(
new Function2<JavaRDD<String>, Time, Void>() {
@Override
public Void call(JavaRDD<String> rdd, Time time) {
- SQLContext sqlContext = JavaSQLContextSingleton.getInstance(rdd.context());
+
+ // Get the singleton instance of SQLContext
+ SQLContext sqlContext = SQLContext.getOrCreate(rdd.context());
// Convert RDD[String] to RDD[case class] to DataFrame
JavaRDD<JavaRow> rowRDD = rdd.map(new Function<String, JavaRow>() {
@@ -2234,7 +2204,7 @@ The following table summarizes the semantics under failures:
### With Kafka Direct API
{:.no_toc}
-In Spark 1.3, we have introduced a new Kafka Direct API, which can ensure that all the Kafka data is received by Spark Streaming exactly once. Along with this, if you implement exactly-once output operation, you can achieve end-to-end exactly-once guarantees. This approach (experimental as of Spark 1.3) is further discussed in the [Kafka Integration Guide](streaming-kafka-integration.html).
+In Spark 1.3, we have introduced a new Kafka Direct API, which can ensure that all the Kafka data is received by Spark Streaming exactly once. Along with this, if you implement exactly-once output operation, you can achieve end-to-end exactly-once guarantees. This approach (experimental as of Spark {{site.SPARK_VERSION_SHORT}}) is further discussed in the [Kafka Integration Guide](streaming-kafka-integration.html).
## Semantics of output operations
{:.no_toc}
@@ -2248,8 +2218,16 @@ additional effort may be necessary to achieve exactly-once semantics. There are
- *Transactional updates*: All updates are made transactionally so that updates are made exactly once atomically. One way to do this would be the following.
- - Use the batch time (available in `foreachRDD`) and the partition index of the transformed RDD to create an identifier. This identifier uniquely identifies a blob data in the streaming application.
- - Update external system with this blob transactionally (that is, exactly once, atomically) using the identifier. That is, if the identifier is not already committed, commit the partition data and the identifier atomically. Else if this was already committed, skip the update.
+ - Use the batch time (available in `foreachRDD`) and the partition index of the RDD to create an identifier. This identifier uniquely identifies a blob data in the streaming application.
+ - Update external system with this blob transactionally (that is, exactly once, atomically) using the identifier. That is, if the identifier is not already committed, commit the partition data and the identifier atomically. Else if this was already committed, skip the update.
+
+ dstream.foreachRDD { (rdd, time) =>
+ rdd.foreachPartition { partitionIterator =>
+ val partitionId = TaskContext.get.partitionId()
+ val uniqueId = generateUniqueId(time.milliseconds, partitionId)
+ // use this uniqueId to transactionally commit the data in partitionIterator
+ }
+ }
***************************************************************************************************