You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
- 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 <[email protected]>
Closesapache#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
By default, the Python API will decode Kafka data as UTF8 encoded strings. You can specify your custom decoding function to decode the byte arrays in Kafka records to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils)
127
+
and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/direct_kafka_wordcount.py).
121
128
</div>
122
129
</div>
123
130
@@ -147,10 +154,13 @@ Next, we discuss how to use this approach in your streaming application.
147
154
}
148
155
);
149
156
</div>
157
+
<div data-lang="python" markdown="1">
158
+
Not supported
150
159
</div>
160
+
</div>
151
161
152
162
You can use this to update Zookeeper yourself if you want Zookeeper-based Kafka monitoring tools to show progress of the streaming application.
153
163
154
164
Another thing to note is that since this approach does not use Receivers, the standard receiver-related (that is, [configurations](configuration.html) of the form `spark.streaming.receiver.*` ) will not apply to the input DStreams created by this approach (will apply to other input DStreams though). Instead, use the [configurations](configuration.html) `spark.streaming.kafka.*`. An important one is `spark.streaming.kafka.maxRatePerPartition` which is the maximum rate at which each Kafka partition will be read by this direct API.
155
165
156
-
3.**Deploying:**Similar to the first approach, you can package `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}`and its dependencies into the application JAR and the launch the application using `spark-submit`. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation.
166
+
3.**Deploying:**This is same as the first approach, for Scala, Java and Python.
See the [API docs](api/scala/index.html#org.apache.spark.streaming.kinesis.KinesisUtils$)
38
39
and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala). Refer to the Running the Example section for instructions on how to run the example.
@@ -44,7 +45,8 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m
See the [API docs](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisUtils.html)
50
52
and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java). Refer to the next subsection for instructions to run the example.
@@ -54,19 +56,23 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m
54
56
55
57
- `streamingContext`: StreamingContext containg an application name used by Kinesis to tie this Kinesis application to the Kinesis stream
56
58
57
-
- `[Kinesis stream name]`: The Kinesis stream that this streaming application receives from
58
-
- The application name used in the streaming context becomes the Kinesis application name
59
+
- `[Kineiss app name]`: The application name that will be used to checkpoint the Kinesis
60
+
sequence numbers in DynamoDB table.
59
61
- The application name must be unique for a given account and region.
60
-
- The Kinesis backend automatically associates the application name to the Kinesis stream using a DynamoDB table (always in the us-east-1 region) created during Kinesis Client Library initialization.
61
-
- Changing the application name or stream name can lead to Kinesis errors in some cases. If you see errors, you may need to manually delete the DynamoDB table.
62
+
- If the table exists but has incorrect checkpoint information (for a different stream, or
63
+
old expired sequenced numbers), then there may be temporary errors.
62
64
65
+
- `[Kinesis stream name]`: The Kinesis stream that this streaming application will pull data from.
63
66
64
67
- `[endpoint URL]`: Valid Kinesis endpoints URL can be found [here](http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region).
65
68
69
+
- `[region name]`: Valid Kinesis region names can be found [here](https://docs.aws.amazon.com/AWSEC2/latest/UserGuide/using-regions-availability-zones.html).
70
+
66
71
- `[checkpoint interval]`: The interval (e.g., Duration(2000) = 2 seconds) at which the Kinesis Client Library saves its position in the stream. For starters, set it to the same as the batch interval of the streaming application.
67
72
68
73
- `[initial position]`: Can be either `InitialPositionInStream.TRIM_HORIZON` or `InitialPositionInStream.LATEST` (see Kinesis Checkpointing section and Amazon Kinesis API documentation for more details).
69
74
75
+
In other versions of the API, you can also specify the AWS access key and secret key directly.
70
76
71
77
3.**Deploying:** Package `spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}}` and its dependencies (except `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` which are provided by `spark-submit`) into the application JAR. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide).
This will push 1000 lines per second of 10 random numbers per line to the Kinesis stream. This data should then be received and processed by the running example.
Copy file name to clipboardExpand all lines: docs/streaming-programming-guide.md
+24-46Lines changed: 24 additions & 46 deletions
Display the source diff
Display the rich diff
Original file line number
Diff line number
Diff line change
@@ -77,7 +77,7 @@ main entry point for all streaming functionality. We create a local StreamingCon
77
77
{% highlight scala %}
78
78
import org.apache.spark._
79
79
import org.apache.spark.streaming._
80
-
import org.apache.spark.streaming.StreamingContext._ // not necessary in Spark 1.3+
80
+
import org.apache.spark.streaming.StreamingContext._ // not necessary since Spark 1.3
81
81
82
82
// Create a local StreamingContext with two working thread and batch interval of 1 second.
83
83
// 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
109
109
`words` DStream. Next, we want to count these words.
110
110
111
111
{% highlight scala %}
112
-
import org.apache.spark.streaming.StreamingContext._ // not necessary in Spark 1.3+
112
+
import org.apache.spark.streaming.StreamingContext._ // not necessary since Spark 1.3
113
113
// Count each word in each batch
114
114
val pairs = words.map(word => (word, 1))
115
115
val wordCounts = pairs.reduceByKey(_ + _)
@@ -682,7 +682,7 @@ for Java, and [StreamingContext](api/python/pyspark.streaming.html#pyspark.strea
682
682
### Advanced Sources
683
683
{:.no_toc}
684
684
685
-
<spanclass="badge"style="background-color: grey">Python API</span> As of Spark 1.3,
685
+
<spanclass="badge"style="background-color: grey">Python API</span> As of Spark {{site.SPARK_VERSION_SHORT}},
686
686
out of these sources, *only* Kafka is available in the Python API. We will add more advanced sources in the Python API in future.
687
687
688
688
This category of sources require interfacing with external non-Spark libraries, some of them with
@@ -723,7 +723,7 @@ and it in the classpath.
723
723
724
724
Some of these advanced sources are as follows.
725
725
726
-
-**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.
726
+
-**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.
727
727
728
728
-**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.
@@ -2234,7 +2204,7 @@ The following table summarizes the semantics under failures:
2234
2204
2235
2205
### With Kafka Direct API
2236
2206
{:.no_toc}
2237
-
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).
2207
+
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).
2238
2208
2239
2209
## Semantics of output operations
2240
2210
{:.no_toc}
@@ -2248,8 +2218,16 @@ additional effort may be necessary to achieve exactly-once semantics. There are
2248
2218
2249
2219
-*Transactional updates*: All updates are made transactionally so that updates are made exactly once atomically. One way to do this would be the following.
2250
2220
2251
-
- 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.
2252
-
- 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.
2221
+
- 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.
2222
+
- 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.
2223
+
2224
+
dstream.foreachRDD { (rdd, time) =>
2225
+
rdd.foreachPartition { partitionIterator =>
2226
+
val partitionId = TaskContext.get.partitionId()
2227
+
val uniqueId = generateUniqueId(time.milliseconds, partitionId)
2228
+
// use this uniqueId to transactionally commit the data in partitionIterator
0 commit comments