Skip to content

Commit cd3b68d

Browse files
committed
[SPARK-6128][Streaming][Documentation] Updates to Spark Streaming Programming Guide
Updates to the documentation are as follows: - Added information on Kafka Direct API and Kafka Python API - Added joins to the main streaming guide - Improved details on the fault-tolerance semantics Generated docs located here http://people.apache.org/~tdas/spark-1.3.0-temp-docs/streaming-programming-guide.html#fault-tolerance-semantics More things to add: - Configuration for Kafka receive rate - May be add concurrentJobs Author: Tathagata Das <[email protected]> Closes apache#4956 from tdas/streaming-guide-update-1.3 and squashes the following commits: 819408c [Tathagata Das] Minor fixes. debe484 [Tathagata Das] Added DataFrames and MLlib 380cf8d [Tathagata Das] Fix link 04167a6 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-guide-update-1.3 0b77486 [Tathagata Das] Updates based on Josh's comments. 86c4c2a [Tathagata Das] Updated streaming guides 82de92a [Tathagata Das] Add Kafka to Python api docs
1 parent 51a79a7 commit cd3b68d

File tree

4 files changed

+528
-109
lines changed

4 files changed

+528
-109
lines changed

docs/configuration.md

Lines changed: 12 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1345,9 +1345,9 @@ Apart from these, the following properties are also available, and may be useful
13451345
</tr>
13461346
<tr>
13471347
<td><code>spark.streaming.receiver.maxRate</code></td>
1348-
<td>infinite</td>
1348+
<td>not set</td>
13491349
<td>
1350-
Maximum number records per second at which each receiver will receive data.
1350+
Maximum rate (number of records per second) at which each receiver will receive data.
13511351
Effectively, each stream will consume at most this number of records per second.
13521352
Setting this configuration to 0 or a negative number will put no limit on the rate.
13531353
See the <a href="streaming-programming-guide.html#deploying-applications">deployment guide</a>
@@ -1375,6 +1375,16 @@ Apart from these, the following properties are also available, and may be useful
13751375
higher memory usage in Spark.
13761376
</td>
13771377
</tr>
1378+
<tr>
1379+
<td><code>spark.streaming.kafka.maxRatePerPartition</code></td>
1380+
<td>not set</td>
1381+
<td>
1382+
Maximum rate (number of records per second) at which data will be read from each Kafka
1383+
partition when using the new Kafka direct stream API. See the
1384+
<a href="streaming-kafka-integration.html">Kafka Integration guide</a>
1385+
for more details.
1386+
</td>
1387+
</tr>
13781388
</table>
13791389

13801390
#### Cluster Managers

docs/streaming-flume-integration.md

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -5,6 +5,8 @@ title: Spark Streaming + Flume Integration Guide
55

66
[Apache Flume](https://flume.apache.org/) is a distributed, reliable, and available service for efficiently collecting, aggregating, and moving large amounts of log data. Here we explain how to configure Flume and Spark Streaming to receive data from Flume. There are two approaches to this.
77

8+
<span class="badge" style="background-color: grey">Python API</span> Flume is not yet available in the Python API.
9+
810
## Approach 1: Flume-style Push-based Approach
911
Flume is designed to push data between Flume agents. In this approach, Spark Streaming essentially sets up a receiver that acts an Avro agent for Flume, to which Flume can push the data. Here are the configuration steps.
1012

docs/streaming-kafka-integration.md

Lines changed: 124 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -2,58 +2,155 @@
22
layout: global
33
title: Spark Streaming + Kafka Integration Guide
44
---
5-
[Apache Kafka](http://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Here we explain how to configure Spark Streaming to receive data from Kafka.
5+
[Apache Kafka](http://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Here we explain how to configure Spark Streaming to receive data from Kafka. There are two approaches to this - the old approach using Receivers and Kafka's high-level API, and a new experimental approach (introduced in Spark 1.3) without using Receivers. They have different programming models, performance characteristics, and semantics guarantees, so read on for more details.
66

7-
1. **Linking:** In your SBT/Maven project definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information).
7+
## Approach 1: Receiver-based Approach
8+
This approach uses a Receiver to receive the data. The Received is implemented using the Kafka high-level consumer API. As with all receivers, the data received from Kafka through a Receiver is stored in Spark executors, and then jobs launched by Spark Streaming processes the data.
9+
10+
However, under default configuration, this approach can lose data under failures (see [receiver reliability](streaming-programming-guide.html#receiver-reliability). To ensure zero-data loss, you have to additionally enable Write Ahead Logs in Spark Streaming. To ensure zero data loss, enable the Write Ahead Logs (introduced in Spark 1.2). This synchronously saves all the received Kafka data into write ahead logs on a distributed file system (e.g HDFS), so that all the data can be recovered on failure. See [Deploying section](streaming-programming-guide.html#deploying-applications) in the streaming programming guide for more details on Write Ahead Logs.
11+
12+
Next, we discuss how to use this approach in your streaming application.
13+
14+
1. **Linking:** For Scala/Java applications using SBT/Maven project definitions, link your streaming application with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information).
815

916
groupId = org.apache.spark
1017
artifactId = spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}
1118
version = {{site.SPARK_VERSION_SHORT}}
1219

13-
2. **Programming:** In the streaming application code, import `KafkaUtils` and create input DStream as follows.
20+
For Python applications, you will have to add this above library and its dependencies when deploying your application. See the *Deploying* subsection below.
21+
22+
2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows.
1423

1524
<div class="codetabs">
1625
<div data-lang="scala" markdown="1">
1726
import org.apache.spark.streaming.kafka._
1827

19-
val kafkaStream = KafkaUtils.createStream(
20-
streamingContext, [zookeeperQuorum], [group id of the consumer], [per-topic number of Kafka partitions to consume])
28+
val kafkaStream = KafkaUtils.createStream(streamingContext,
29+
[ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume])
2130

22-
See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$)
31+
You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$)
2332
and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala).
2433
</div>
2534
<div data-lang="java" markdown="1">
2635
import org.apache.spark.streaming.kafka.*;
2736

28-
JavaPairReceiverInputDStream<String, String> kafkaStream = KafkaUtils.createStream(
29-
streamingContext, [zookeeperQuorum], [group id of the consumer], [per-topic number of Kafka partitions to consume]);
37+
JavaPairReceiverInputDStream<String, String> kafkaStream =
38+
KafkaUtils.createStream(streamingContext,
39+
[ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]);
3040

31-
See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html)
41+
You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html)
3242
and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java).
43+
44+
</div>
45+
<div data-lang="python" markdown="1">
46+
from pyspark.streaming.kafka import KafkaUtils
47+
48+
kafkaStream = KafkaUtils.createStream(streamingContext, \
49+
[ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume])
50+
51+
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)
52+
and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/kafka_wordcount.py).
3353
</div>
3454
</div>
3555

36-
*Points to remember:*
56+
**Points to remember:**
3757

3858
- Topic partitions in Kafka does not correlate to partitions of RDDs generated in Spark Streaming. So increasing the number of topic-specific partitions in the `KafkaUtils.createStream()` only increases the number of threads using which topics that are consumed within a single receiver. It does not increase the parallelism of Spark in processing the data. Refer to the main document for more information on that.
3959

4060
- Multiple Kafka input DStreams can be created with different groups and topics for parallel receiving of data using multiple receivers.
4161

42-
3. **Deploying:** Package `spark-streaming-kafka_{{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).
43-
44-
Note that the Kafka receiver used by default is an
45-
[*unreliable* receiver](streaming-programming-guide.html#receiver-reliability) section in the
46-
programming guide). In Spark 1.2, we have added an experimental *reliable* Kafka receiver that
47-
provides stronger
48-
[fault-tolerance guarantees](streaming-programming-guide.html#fault-tolerance-semantics) of zero
49-
data loss on failures. This receiver is automatically used when the write ahead log
50-
(also introduced in Spark 1.2) is enabled
51-
(see [Deployment](#deploying-applications.html) section in the programming guide). This
52-
may reduce the receiving throughput of individual Kafka receivers compared to the unreliable
53-
receivers, but this can be corrected by running
54-
[more receivers in parallel](streaming-programming-guide.html#level-of-parallelism-in-data-receiving)
55-
to increase aggregate throughput. Additionally, it is recommended that the replication of the
56-
received data within Spark be disabled when the write ahead log is enabled as the log is already stored
57-
in a replicated storage system. This can be done by setting the storage level for the input
58-
stream to `StorageLevel.MEMORY_AND_DISK_SER` (that is, use
62+
- If you have enabled Write Ahead Logs with a replicated file system like HDFS, the received data is already being replicated in the log. Hence, the storage level in storage level for the input stream to `StorageLevel.MEMORY_AND_DISK_SER` (that is, use
5963
`KafkaUtils.createStream(..., StorageLevel.MEMORY_AND_DISK_SER)`).
64+
65+
3. **Deploying:** As with any Spark applications, `spark-submit` is used to launch your application. However, the details are slightly different for Scala/Java applications and Python applications.
66+
67+
For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. 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. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide).
68+
69+
For Python applications which lack SBT/Maven project management, `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies can be directly added to `spark-submit` using `--packages` (see [Application Submission Guide](submitting-applications.html)). That is,
70+
71+
./bin/spark-submit --packages org.apache.spark:spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ...
72+
73+
Alternatively, you can also download the JAR of the Maven artifact `spark-streaming-kafka-assembly` from the
74+
[Maven repository](http://search.maven.org/#search|ga|1|a%3A%22spark-streaming-kafka-assembly_2.10%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) and add it to `spark-submit` with `--jars`.
75+
76+
## Approach 2: Direct Approach (No Receivers)
77+
This is a new receiver-less "direct" approach has been introduced in Spark 1.3 to ensure stronger end-to-end guarantees. Instead of using receivers to receive data, this approach periodically queries Kafka for the latest offsets in each topic+partition, and accordingly defines the offset ranges to process in each batch. When the jobs to process the data are launched, Kafka's simple consumer API is used to read the defined ranges of offsets from Kafka (similar to read files from a file system). Note that this is an experimental feature in Spark 1.3 and is only available in the Scala and Java API.
78+
79+
This approach has the following advantages over the received-based approach (i.e. Approach 1).
80+
81+
- *Simplified Parallelism:* No need to create multiple input Kafka streams and union-ing them. With `directStream`, Spark Streaming will create as many RDD partitions as there is Kafka partitions to consume, which will all read data from Kafka in parallel. So there is one-to-one mapping between Kafka and RDD partitions, which is easier to understand and tune.
82+
83+
- *Efficiency:* Achieving zero-data loss in the first approach required the data to be stored in a Write Ahead Log, which further replicated the data. This is actually inefficient as the data effectively gets replicated twice - once by Kafka, and a second time by the Write Ahead Log. This second approach eliminate the problem as there is no receiver, and hence no need for Write Ahead Logs.
84+
85+
- *Exactly-once semantics:* The first approach uses Kafka's high level API to store consumed offsets in Zookeeper. This is traditionally the way to consume data from Kafka. While this approach (in combination with write ahead logs) can ensure zero data loss (i.e. at-least once semantics), there is a small chance some records may get consumed twice under some failures. This occurs because of inconsistencies between data reliably received by Spark Streaming and offsets tracked by Zookeeper. Hence, in this second approach, we use simple Kafka API that does not use Zookeeper and offsets tracked only by Spark Streaming within its checkpoints. This eliminates inconsistencies between Spark Streaming and Zookeeper/Kafka, and so each record is received by Spark Streaming effectively exactly once despite failures.
86+
87+
Note that one disadvantage of this approach is that it does not update offsets in Zookeeper, hence Zookeeper-based Kafka monitoring tools will not show progress. However, you can access the offsets processed by this approach in each batch and update Zookeeper yourself (see below).
88+
89+
Next, we discuss how to use this approach in your streaming application.
90+
91+
1. **Linking:** This approach is supported only in Scala/Java application. Link your SBT/Maven project with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information).
92+
93+
groupId = org.apache.spark
94+
artifactId = spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}
95+
version = {{site.SPARK_VERSION_SHORT}}
96+
97+
2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows.
98+
99+
<div class="codetabs">
100+
<div data-lang="scala" markdown="1">
101+
import org.apache.spark.streaming.kafka._
102+
103+
val directKafkaStream = KafkaUtils.createDirectStream[
104+
[key class], [value class], [key decoder class], [value decoder class] ](
105+
streamingContext, [map of Kafka parameters], [set of topics to consume])
106+
107+
See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$)
108+
and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala).
109+
</div>
110+
<div data-lang="java" markdown="1">
111+
import org.apache.spark.streaming.kafka.*;
112+
113+
JavaPairReceiverInputDStream<String, String> directKafkaStream =
114+
KafkaUtils.createDirectStream(streamingContext,
115+
[key class], [value class], [key decoder class], [value decoder class],
116+
[map of Kafka parameters], [set of topics to consume]);
117+
118+
See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html)
119+
and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java).
120+
121+
</div>
122+
</div>
123+
124+
In the Kafka parameters, you must specify either `metadata.broker.list` or `bootstrap.servers`.
125+
By default, it will start consuming from the latest offset of each Kafka partition. If you set configuration `auto.offset.reset` in Kafka parameters to `smallest`, then it will start consuming from the smallest offset.
126+
127+
You can also start consuming from any arbitrary offset using other variations of `KafkaUtils.createDirectStream`. Furthermore, if you want to access the Kafka offsets consumed in each batch, you can do the following.
128+
129+
<div class="codetabs">
130+
<div data-lang="scala" markdown="1">
131+
directKafkaStream.foreachRDD { rdd =>
132+
val offsetRanges = rdd.asInstanceOf[HasOffsetRanges]
133+
// offsetRanges.length = # of Kafka partitions being consumed
134+
...
135+
}
136+
</div>
137+
<div data-lang="java" markdown="1">
138+
directKafkaStream.foreachRDD(
139+
new Function<JavaPairRDD<String, String>, Void>() {
140+
@Override
141+
public Void call(JavaPairRDD<String, Integer> rdd) throws IOException {
142+
OffsetRange[] offsetRanges = ((HasOffsetRanges)rdd).offsetRanges
143+
// offsetRanges.length = # of Kafka partitions being consumed
144+
...
145+
return null;
146+
}
147+
}
148+
);
149+
</div>
150+
</div>
151+
152+
You can use this to update Zookeeper yourself if you want Zookeeper-based Kafka monitoring tools to show progress of the streaming application.
153+
154+
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+
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.

0 commit comments

Comments
 (0)