Skip to content

Commit 9293734

Browse files
zsxwingtdas
authored andcommitted
[SPARK-17346][SQL] Add Kafka source for Structured Streaming
## What changes were proposed in this pull request? This PR adds a new project ` external/kafka-0-10-sql` for Structured Streaming Kafka source. It's based on the design doc: https://docs.google.com/document/d/19t2rWe51x7tq2e5AOfrsM9qb8_m7BRuv9fel9i0PqR8/edit?usp=sharing tdas did most of work and part of them was inspired by koeninger's work. ### Introduction The Kafka source is a structured streaming data source to poll data from Kafka. The schema of reading data is as follows: Column | Type ---- | ---- key | binary value | binary topic | string partition | int offset | long timestamp | long timestampType | int The source can deal with deleting topics. However, the user should make sure there is no Spark job processing the data when deleting a topic. ### Configuration The user can use `DataStreamReader.option` to set the following configurations. Kafka Source's options | value | default | meaning ------ | ------- | ------ | ----- startingOffset | ["earliest", "latest"] | "latest" | The start point when a query is started, either "earliest" which is from the earliest offset, or "latest" which is just from the latest offset. Note: This only applies when a new Streaming query is started, and that resuming will always pick up from where the query left off. failOnDataLost | [true, false] | true | Whether to fail the query when it's possible that data is lost (e.g., topics are deleted, or offsets are out of range). This may be a false alarm. You can disable it when it doesn't work as you expected. subscribe | A comma-separated list of topics | (none) | The topic list to subscribe. Only one of "subscribe" and "subscribeParttern" options can be specified for Kafka source. subscribePattern | Java regex string | (none) | The pattern used to subscribe the topic. Only one of "subscribe" and "subscribeParttern" options can be specified for Kafka source. kafka.consumer.poll.timeoutMs | long | 512 | The timeout in milliseconds to poll data from Kafka in executors fetchOffset.numRetries | int | 3 | Number of times to retry before giving up fatch Kafka latest offsets. fetchOffset.retryIntervalMs | long | 10 | milliseconds to wait before retrying to fetch Kafka offsets Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, `stream.option("kafka.bootstrap.servers", "host:port")` ### Usage * Subscribe to 1 topic ```Scala spark .readStream .format("kafka") .option("kafka.bootstrap.servers", "host:port") .option("subscribe", "topic1") .load() ``` * Subscribe to multiple topics ```Scala spark .readStream .format("kafka") .option("kafka.bootstrap.servers", "host:port") .option("subscribe", "topic1,topic2") .load() ``` * Subscribe to a pattern ```Scala spark .readStream .format("kafka") .option("kafka.bootstrap.servers", "host:port") .option("subscribePattern", "topic.*") .load() ``` ## How was this patch tested? The new unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Shixiong Zhu <zsxwing@gmail.com> Author: cody koeninger <cody@koeninger.org> Closes #15102 from zsxwing/kafka-source.
1 parent 5fd54b9 commit 9293734

File tree

21 files changed

+2268
-23
lines changed

21 files changed

+2268
-23
lines changed

core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala

Lines changed: 0 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -89,13 +89,6 @@ private[spark] class UninterruptibleThread(name: String) extends Thread(name) {
8989
}
9090
}
9191

92-
/**
93-
* Tests whether `interrupt()` has been called.
94-
*/
95-
override def isInterrupted: Boolean = {
96-
super.isInterrupted || uninterruptibleLock.synchronized { shouldInterruptThread }
97-
}
98-
9992
/**
10093
* Interrupt `this` thread if possible. If `this` is in the uninterruptible status, it won't be
10194
* interrupted until it enters into the interruptible status.

dev/run-tests.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -110,7 +110,7 @@ def determine_modules_to_test(changed_modules):
110110
['graphx', 'examples']
111111
>>> x = [x.name for x in determine_modules_to_test([modules.sql])]
112112
>>> x # doctest: +NORMALIZE_WHITESPACE
113-
['sql', 'hive', 'mllib', 'examples', 'hive-thriftserver',
113+
['sql', 'hive', 'mllib', 'sql-kafka-0-10', 'examples', 'hive-thriftserver',
114114
'pyspark-sql', 'sparkr', 'pyspark-mllib', 'pyspark-ml']
115115
"""
116116
modules_to_test = set()

dev/sparktestsupport/modules.py

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -158,6 +158,18 @@ def __hash__(self):
158158
)
159159

160160

161+
sql_kafka = Module(
162+
name="sql-kafka-0-10",
163+
dependencies=[sql],
164+
source_file_regexes=[
165+
"external/kafka-0-10-sql",
166+
],
167+
sbt_test_goals=[
168+
"sql-kafka-0-10/test",
169+
]
170+
)
171+
172+
161173
sketch = Module(
162174
name="sketch",
163175
dependencies=[tags],
Lines changed: 239 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,239 @@
1+
---
2+
layout: global
3+
title: Structured Streaming + Kafka Integration Guide (Kafka broker version 0.10.0 or higher)
4+
---
5+
6+
Structured Streaming integration for Kafka 0.10 to poll data from Kafka.
7+
8+
### Linking
9+
For Scala/Java applications using SBT/Maven project definitions, link your application with the following artifact:
10+
11+
groupId = org.apache.spark
12+
artifactId = spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}
13+
version = {{site.SPARK_VERSION_SHORT}}
14+
15+
For Python applications, you need to add this above library and its dependencies when deploying your
16+
application. See the [Deploying](#deploying) subsection below.
17+
18+
### Creating a Kafka Source Stream
19+
20+
<div class="codetabs">
21+
<div data-lang="scala" markdown="1">
22+
23+
// Subscribe to 1 topic
24+
val ds1 = spark
25+
.readStream
26+
.format("kafka")
27+
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
28+
.option("subscribe", "topic1")
29+
.load()
30+
ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
31+
.as[(String, String)]
32+
33+
// Subscribe to multiple topics
34+
val ds2 = spark
35+
.readStream
36+
.format("kafka")
37+
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
38+
.option("subscribe", "topic1,topic2")
39+
.load()
40+
ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
41+
.as[(String, String)]
42+
43+
// Subscribe to a pattern
44+
val ds3 = spark
45+
.readStream
46+
.format("kafka")
47+
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
48+
.option("subscribePattern", "topic.*")
49+
.load()
50+
ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
51+
.as[(String, String)]
52+
53+
</div>
54+
<div data-lang="java" markdown="1">
55+
56+
// Subscribe to 1 topic
57+
Dataset<Row> ds1 = spark
58+
.readStream()
59+
.format("kafka")
60+
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
61+
.option("subscribe", "topic1")
62+
.load()
63+
ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
64+
65+
// Subscribe to multiple topics
66+
Dataset<Row> ds2 = spark
67+
.readStream()
68+
.format("kafka")
69+
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
70+
.option("subscribe", "topic1,topic2")
71+
.load()
72+
ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
73+
74+
// Subscribe to a pattern
75+
Dataset<Row> ds3 = spark
76+
.readStream()
77+
.format("kafka")
78+
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
79+
.option("subscribePattern", "topic.*")
80+
.load()
81+
ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
82+
83+
</div>
84+
<div data-lang="python" markdown="1">
85+
86+
# Subscribe to 1 topic
87+
ds1 = spark
88+
.readStream()
89+
.format("kafka")
90+
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
91+
.option("subscribe", "topic1")
92+
.load()
93+
ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
94+
95+
# Subscribe to multiple topics
96+
ds2 = spark
97+
.readStream
98+
.format("kafka")
99+
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
100+
.option("subscribe", "topic1,topic2")
101+
.load()
102+
ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
103+
104+
# Subscribe to a pattern
105+
ds3 = spark
106+
.readStream()
107+
.format("kafka")
108+
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
109+
.option("subscribePattern", "topic.*")
110+
.load()
111+
ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
112+
113+
</div>
114+
</div>
115+
116+
Each row in the source has the following schema:
117+
<table class="table">
118+
<tr><th>Column</th><th>Type</th></tr>
119+
<tr>
120+
<td>key</td>
121+
<td>binary</td>
122+
</tr>
123+
<tr>
124+
<td>value</td>
125+
<td>binary</td>
126+
</tr>
127+
<tr>
128+
<td>topic</td>
129+
<td>string</td>
130+
</tr>
131+
<tr>
132+
<td>partition</td>
133+
<td>int</td>
134+
</tr>
135+
<tr>
136+
<td>offset</td>
137+
<td>long</td>
138+
</tr>
139+
<tr>
140+
<td>timestamp</td>
141+
<td>long</td>
142+
</tr>
143+
<tr>
144+
<td>timestampType</td>
145+
<td>int</td>
146+
</tr>
147+
</table>
148+
149+
The following options must be set for the Kafka source.
150+
151+
<table class="table">
152+
<tr><th>Option</th><th>value</th><th>meaning</th></tr>
153+
<tr>
154+
<td>subscribe</td>
155+
<td>A comma-separated list of topics</td>
156+
<td>The topic list to subscribe. Only one of "subscribe" and "subscribePattern" options can be
157+
specified for Kafka source.</td>
158+
</tr>
159+
<tr>
160+
<td>subscribePattern</td>
161+
<td>Java regex string</td>
162+
<td>The pattern used to subscribe the topic. Only one of "subscribe" and "subscribePattern"
163+
options can be specified for Kafka source.</td>
164+
</tr>
165+
<tr>
166+
<td>kafka.bootstrap.servers</td>
167+
<td>A comma-separated list of host:port</td>
168+
<td>The Kafka "bootstrap.servers" configuration.</td>
169+
</tr>
170+
</table>
171+
172+
The following configurations are optional:
173+
174+
<table class="table">
175+
<tr><th>Option</th><th>value</th><th>default</th><th>meaning</th></tr>
176+
<tr>
177+
<td>startingOffset</td>
178+
<td>["earliest", "latest"]</td>
179+
<td>"latest"</td>
180+
<td>The start point when a query is started, either "earliest" which is from the earliest offset,
181+
or "latest" which is just from the latest offset. Note: This only applies when a new Streaming q
182+
uery is started, and that resuming will always pick up from where the query left off.</td>
183+
</tr>
184+
<tr>
185+
<td>failOnDataLoss</td>
186+
<td>[true, false]</td>
187+
<td>true</td>
188+
<td>Whether to fail the query when it's possible that data is lost (e.g., topics are deleted, or
189+
offsets are out of range). This may be a false alarm. You can disable it when it doesn't work
190+
as you expected.</td>
191+
</tr>
192+
<tr>
193+
<td>kafkaConsumer.pollTimeoutMs</td>
194+
<td>long</td>
195+
<td>512</td>
196+
<td>The timeout in milliseconds to poll data from Kafka in executors.</td>
197+
</tr>
198+
<tr>
199+
<td>fetchOffset.numRetries</td>
200+
<td>int</td>
201+
<td>3</td>
202+
<td>Number of times to retry before giving up fatch Kafka latest offsets.</td>
203+
</tr>
204+
<tr>
205+
<td>fetchOffset.retryIntervalMs</td>
206+
<td>long</td>
207+
<td>10</td>
208+
<td>milliseconds to wait before retrying to fetch Kafka offsets</td>
209+
</tr>
210+
</table>
211+
212+
Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g,
213+
`stream.option("kafka.bootstrap.servers", "host:port")`. For possible kafkaParams, see
214+
[Kafka consumer config docs](http://kafka.apache.org/documentation.html#newconsumerconfigs).
215+
216+
Note that the following Kafka params cannot be set and the Kafka source will throw an exception:
217+
- **group.id**: Kafka source will create a unique group id for each query automatically.
218+
- **auto.offset.reset**: Set the source option `startingOffset` to `earliest` or `latest` to specify
219+
where to start instead. Structured Streaming manages which offsets are consumed internally, rather
220+
than rely on the kafka Consumer to do it. This will ensure that no data is missed when when new
221+
topics/partitions are dynamically subscribed. Note that `startingOffset` only applies when a new
222+
Streaming query is started, and that resuming will always pick up from where the query left off.
223+
- **key.deserializer**: Keys are always deserialized as byte arrays with ByteArrayDeserializer. Use
224+
DataFrame operations to explicitly deserialize the keys.
225+
- **value.deserializer**: Values are always deserialized as byte arrays with ByteArrayDeserializer.
226+
Use DataFrame operations to explicitly deserialize the values.
227+
- **enable.auto.commit**: Kafka source doesn't commit any offset.
228+
- **interceptor.classes**: Kafka source always read keys and values as byte arrays. It's not safe to
229+
use ConsumerInterceptor as it may break the query.
230+
231+
### Deploying
232+
233+
As with any Spark applications, `spark-submit` is used to launch your application. `spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}`
234+
and its dependencies can be directly added to `spark-submit` using `--packages`, such as,
235+
236+
./bin/spark-submit --packages org.apache.spark:spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ...
237+
238+
See [Application Submission Guide](submitting-applications.html) for more details about submitting
239+
applications with external dependencies.

docs/structured-streaming-programming-guide.md

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -418,10 +418,15 @@ Since Spark 2.0, DataFrames and Datasets can represent static, bounded data, as
418418
Streaming DataFrames can be created through the `DataStreamReader` interface
419419
([Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamReader)/
420420
[Java](api/java/org/apache/spark/sql/streaming/DataStreamReader.html)/
421-
[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader) docs) returned by `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source – data format, schema, options, etc. In Spark 2.0, there are a few built-in sources.
421+
[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader) docs) returned by `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source – data format, schema, options, etc.
422+
423+
#### Data Sources
424+
In Spark 2.0, there are a few built-in sources.
422425

423426
- **File source** - Reads files written in a directory as a stream of data. Supported file formats are text, csv, json, parquet. See the docs of the DataStreamReader interface for a more up-to-date list, and supported options for each file format. Note that the files must be atomically placed in the given directory, which in most file systems, can be achieved by file move operations.
424427

428+
- **Kafka source** - Poll data from Kafka. It's compatible with Kafka broker versions 0.10.0 or higher. See the [Kafka Integration Guide](structured-streaming-kafka-integration.html) for more details.
429+
425430
- **Socket source (for testing)** - Reads UTF8 text data from a socket connection. The listening server socket is at the driver. Note that this should be used only for testing as this does not provide end-to-end fault-tolerance guarantees.
426431

427432
Here are some examples.

external/kafka-0-10-sql/pom.xml

Lines changed: 82 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,82 @@
1+
<?xml version="1.0" encoding="UTF-8"?>
2+
<!--
3+
~ Licensed to the Apache Software Foundation (ASF) under one or more
4+
~ contributor license agreements. See the NOTICE file distributed with
5+
~ this work for additional information regarding copyright ownership.
6+
~ The ASF licenses this file to You under the Apache License, Version 2.0
7+
~ (the "License"); you may not use this file except in compliance with
8+
~ the License. You may obtain a copy of the License at
9+
~
10+
~ http://www.apache.org/licenses/LICENSE-2.0
11+
~
12+
~ Unless required by applicable law or agreed to in writing, software
13+
~ distributed under the License is distributed on an "AS IS" BASIS,
14+
~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
~ See the License for the specific language governing permissions and
16+
~ limitations under the License.
17+
-->
18+
19+
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
20+
<modelVersion>4.0.0</modelVersion>
21+
<parent>
22+
<groupId>org.apache.spark</groupId>
23+
<artifactId>spark-parent_2.11</artifactId>
24+
<version>2.1.0-SNAPSHOT</version>
25+
<relativePath>../../pom.xml</relativePath>
26+
</parent>
27+
28+
<groupId>org.apache.spark</groupId>
29+
<artifactId>spark-sql-kafka-0-10_2.11</artifactId>
30+
<properties>
31+
<sbt.project.name>sql-kafka-0-10</sbt.project.name>
32+
</properties>
33+
<packaging>jar</packaging>
34+
<name>Kafka 0.10 Source for Structured Streaming</name>
35+
<url>http://spark.apache.org/</url>
36+
37+
<dependencies>
38+
<dependency>
39+
<groupId>org.apache.spark</groupId>
40+
<artifactId>spark-sql_${scala.binary.version}</artifactId>
41+
<version>${project.version}</version>
42+
<scope>provided</scope>
43+
</dependency>
44+
<dependency>
45+
<groupId>org.apache.spark</groupId>
46+
<artifactId>spark-sql_${scala.binary.version}</artifactId>
47+
<version>${project.version}</version>
48+
<type>test-jar</type>
49+
<scope>test</scope>
50+
</dependency>
51+
<dependency>
52+
<groupId>org.apache.kafka</groupId>
53+
<artifactId>kafka-clients</artifactId>
54+
<version>0.10.0.1</version>
55+
</dependency>
56+
<dependency>
57+
<groupId>org.apache.kafka</groupId>
58+
<artifactId>kafka_${scala.binary.version}</artifactId>
59+
<version>0.10.0.1</version>
60+
<scope>test</scope>
61+
</dependency>
62+
<dependency>
63+
<groupId>net.sf.jopt-simple</groupId>
64+
<artifactId>jopt-simple</artifactId>
65+
<version>3.2</version>
66+
<scope>test</scope>
67+
</dependency>
68+
<dependency>
69+
<groupId>org.scalacheck</groupId>
70+
<artifactId>scalacheck_${scala.binary.version}</artifactId>
71+
<scope>test</scope>
72+
</dependency>
73+
<dependency>
74+
<groupId>org.apache.spark</groupId>
75+
<artifactId>spark-tags_${scala.binary.version}</artifactId>
76+
</dependency>
77+
</dependencies>
78+
<build>
79+
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
80+
<testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
81+
</build>
82+
</project>
Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1 @@
1+
org.apache.spark.sql.kafka010.KafkaSourceProvider

0 commit comments

Comments
 (0)