-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-17346][SQL] Add Kafka source for Structured Streaming #15102
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
985cfa9
c4db857
d3c0754
f5c57f5
d005ba5
b64d104
a3d0a2b
6bc2994
881b206
8e86f98
786af2f
5f33eb4
4fe16c8
e167152
608b8c3
3ee8305
ce3f38f
755ceaa
852f607
59a93a5
9d95d52
e883062
d154532
77208d1
ccadd81
7ff1059
a6c4970
9e9fef3
d50a05e
4316906
d9d848c
7d658f1
4754125
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,239 @@ | ||
| --- | ||
| layout: global | ||
| title: Structured Streaming + Kafka Integration Guide (Kafka broker version 0.10.0 or higher) | ||
| --- | ||
|
|
||
| Structured Streaming integration for Kafka 0.10 to poll data from Kafka. | ||
|
|
||
| ### Linking | ||
| For Scala/Java applications using SBT/Maven project definitions, link your application with the following artifact: | ||
|
|
||
| groupId = org.apache.spark | ||
| artifactId = spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}} | ||
| version = {{site.SPARK_VERSION_SHORT}} | ||
|
|
||
| For Python applications, you need to add this above library and its dependencies when deploying your | ||
| application. See the [Deploying](#deploying) subsection below. | ||
|
|
||
| ### Creating a Kafka Source Stream | ||
|
|
||
| <div class="codetabs"> | ||
| <div data-lang="scala" markdown="1"> | ||
|
|
||
| // Subscribe to 1 topic | ||
| val ds1 = spark | ||
| .readStream | ||
| .format("kafka") | ||
| .option("kafka.bootstrap.servers", "host1:port1,host2:port2") | ||
| .option("subscribe", "topic1") | ||
| .load() | ||
| ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") | ||
| .as[(String, String)] | ||
|
|
||
| // Subscribe to multiple topics | ||
| val ds2 = spark | ||
| .readStream | ||
| .format("kafka") | ||
| .option("kafka.bootstrap.servers", "host1:port1,host2:port2") | ||
| .option("subscribe", "topic1,topic2") | ||
| .load() | ||
| ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") | ||
| .as[(String, String)] | ||
|
|
||
| // Subscribe to a pattern | ||
| val ds3 = spark | ||
| .readStream | ||
| .format("kafka") | ||
| .option("kafka.bootstrap.servers", "host1:port1,host2:port2") | ||
| .option("subscribePattern", "topic.*") | ||
| .load() | ||
| ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") | ||
| .as[(String, String)] | ||
|
|
||
| </div> | ||
| <div data-lang="java" markdown="1"> | ||
|
|
||
| // Subscribe to 1 topic | ||
| Dataset<Row> ds1 = spark | ||
| .readStream() | ||
| .format("kafka") | ||
| .option("kafka.bootstrap.servers", "host1:port1,host2:port2") | ||
| .option("subscribe", "topic1") | ||
| .load() | ||
| ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") | ||
|
|
||
| // Subscribe to multiple topics | ||
| Dataset<Row> ds2 = spark | ||
| .readStream() | ||
| .format("kafka") | ||
| .option("kafka.bootstrap.servers", "host1:port1,host2:port2") | ||
| .option("subscribe", "topic1,topic2") | ||
| .load() | ||
| ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") | ||
|
|
||
| // Subscribe to a pattern | ||
| Dataset<Row> ds3 = spark | ||
| .readStream() | ||
| .format("kafka") | ||
| .option("kafka.bootstrap.servers", "host1:port1,host2:port2") | ||
| .option("subscribePattern", "topic.*") | ||
| .load() | ||
| ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") | ||
|
|
||
| </div> | ||
| <div data-lang="python" markdown="1"> | ||
|
|
||
| # Subscribe to 1 topic | ||
| ds1 = spark | ||
| .readStream() | ||
| .format("kafka") | ||
| .option("kafka.bootstrap.servers", "host1:port1,host2:port2") | ||
| .option("subscribe", "topic1") | ||
| .load() | ||
| ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") | ||
|
|
||
| # Subscribe to multiple topics | ||
| ds2 = spark | ||
| .readStream | ||
| .format("kafka") | ||
| .option("kafka.bootstrap.servers", "host1:port1,host2:port2") | ||
| .option("subscribe", "topic1,topic2") | ||
| .load() | ||
| ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") | ||
|
|
||
| # Subscribe to a pattern | ||
| ds3 = spark | ||
| .readStream() | ||
| .format("kafka") | ||
| .option("kafka.bootstrap.servers", "host1:port1,host2:port2") | ||
| .option("subscribePattern", "topic.*") | ||
| .load() | ||
| ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") | ||
|
|
||
| </div> | ||
| </div> | ||
|
|
||
| Each row in the source has the following schema: | ||
| <table class="table"> | ||
| <tr><th>Column</th><th>Type</th></tr> | ||
| <tr> | ||
| <td>key</td> | ||
| <td>binary</td> | ||
| </tr> | ||
| <tr> | ||
| <td>value</td> | ||
| <td>binary</td> | ||
| </tr> | ||
| <tr> | ||
| <td>topic</td> | ||
| <td>string</td> | ||
| </tr> | ||
| <tr> | ||
| <td>partition</td> | ||
| <td>int</td> | ||
| </tr> | ||
| <tr> | ||
| <td>offset</td> | ||
| <td>long</td> | ||
| </tr> | ||
| <tr> | ||
| <td>timestamp</td> | ||
| <td>long</td> | ||
| </tr> | ||
| <tr> | ||
| <td>timestampType</td> | ||
| <td>int</td> | ||
| </tr> | ||
| </table> | ||
|
|
||
| The following options must be set for the Kafka source. | ||
|
|
||
| <table class="table"> | ||
| <tr><th>Option</th><th>value</th><th>meaning</th></tr> | ||
| <tr> | ||
| <td>subscribe</td> | ||
| <td>A comma-separated list of topics</td> | ||
| <td>The topic list to subscribe. Only one of "subscribe" and "subscribePattern" options can be | ||
| specified for Kafka source.</td> | ||
| </tr> | ||
| <tr> | ||
| <td>subscribePattern</td> | ||
| <td>Java regex string</td> | ||
| <td>The pattern used to subscribe the topic. Only one of "subscribe" and "subscribePattern" | ||
| options can be specified for Kafka source.</td> | ||
| </tr> | ||
| <tr> | ||
| <td>kafka.bootstrap.servers</td> | ||
| <td>A comma-separated list of host:port</td> | ||
| <td>The Kafka "bootstrap.servers" configuration.</td> | ||
| </tr> | ||
| </table> | ||
|
|
||
| The following configurations are optional: | ||
|
|
||
| <table class="table"> | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I would split this into two tables, one that must be specified (subscribe/subscribePattern, and kafka.bootstrap...) and then optional params.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
done |
||
| <tr><th>Option</th><th>value</th><th>default</th><th>meaning</th></tr> | ||
| <tr> | ||
| <td>startingOffset</td> | ||
| <td>["earliest", "latest"]</td> | ||
| <td>"latest"</td> | ||
| <td>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 q | ||
| uery is started, and that resuming will always pick up from where the query left off.</td> | ||
| </tr> | ||
| <tr> | ||
| <td>failOnDataLoss</td> | ||
| <td>[true, false]</td> | ||
| <td>true</td> | ||
| <td>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.</td> | ||
| </tr> | ||
| <tr> | ||
| <td>kafkaConsumer.pollTimeoutMs</td> | ||
| <td>long</td> | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: can keep this is
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Since this is a milliseconds, I try to make the type consistent with other configurations in Spark. |
||
| <td>512</td> | ||
| <td>The timeout in milliseconds to poll data from Kafka in executors.</td> | ||
| </tr> | ||
| <tr> | ||
| <td>fetchOffset.numRetries</td> | ||
| <td>int</td> | ||
| <td>3</td> | ||
| <td>Number of times to retry before giving up fatch Kafka latest offsets.</td> | ||
| </tr> | ||
| <tr> | ||
| <td>fetchOffset.retryIntervalMs</td> | ||
| <td>long</td> | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: can keep this as int.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Since this is a milliseconds, I try to make the type consistent with other configurations in Spark. |
||
| <td>10</td> | ||
| <td>milliseconds to wait before retrying to fetch Kafka offsets</td> | ||
| </tr> | ||
| </table> | ||
|
|
||
| Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, | ||
| `stream.option("kafka.bootstrap.servers", "host:port")`. For possible kafkaParams, see | ||
| [Kafka consumer config docs](http://kafka.apache.org/documentation.html#newconsumerconfigs). | ||
|
|
||
| Note that the following Kafka params cannot be set and the Kafka source will throw an exception: | ||
| - **group.id**: Kafka source will create a unique group id for each query automatically. | ||
| - **auto.offset.reset**: Set the source option `startingOffset` to `earliest` or `latest` to specify | ||
| where to start instead. Structured Streaming manages which offsets are consumed internally, rather | ||
| than rely on the kafka Consumer to do it. This will ensure that no data is missed when when new | ||
| topics/partitions are dynamically subscribed. Note that `startingOffset` only applies when a new | ||
| Streaming query is started, and that resuming will always pick up from where the query left off. | ||
| - **key.deserializer**: Keys are always deserialized as byte arrays with ByteArrayDeserializer. Use | ||
| DataFrame operations to explicitly deserialize the keys. | ||
| - **value.deserializer**: Values are always deserialized as byte arrays with ByteArrayDeserializer. | ||
| Use DataFrame operations to explicitly deserialize the values. | ||
| - **enable.auto.commit**: Kafka source doesn't commit any offset. | ||
| - **interceptor.classes**: Kafka source always read keys and values as byte arrays. It's not safe to | ||
| use ConsumerInterceptor as it may break the query. | ||
|
|
||
| ### Deploying | ||
|
|
||
| As with any Spark applications, `spark-submit` is used to launch your application. `spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}` | ||
| and its dependencies can be directly added to `spark-submit` using `--packages`, such as, | ||
|
|
||
| ./bin/spark-submit --packages org.apache.spark:spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... | ||
|
|
||
| See [Application Submission Guide](submitting-applications.html) for more details about submitting | ||
| applications with external dependencies. | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,82 @@ | ||
| <?xml version="1.0" encoding="UTF-8"?> | ||
| <!-- | ||
| ~ Licensed to the Apache Software Foundation (ASF) under one or more | ||
| ~ contributor license agreements. See the NOTICE file distributed with | ||
| ~ this work for additional information regarding copyright ownership. | ||
| ~ The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| ~ (the "License"); you may not use this file except in compliance with | ||
| ~ the License. You may obtain a copy of the License at | ||
| ~ | ||
| ~ http://www.apache.org/licenses/LICENSE-2.0 | ||
| ~ | ||
| ~ Unless required by applicable law or agreed to in writing, software | ||
| ~ distributed under the License is distributed on an "AS IS" BASIS, | ||
| ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| ~ See the License for the specific language governing permissions and | ||
| ~ limitations under the License. | ||
| --> | ||
|
|
||
| <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"> | ||
| <modelVersion>4.0.0</modelVersion> | ||
| <parent> | ||
| <groupId>org.apache.spark</groupId> | ||
| <artifactId>spark-parent_2.11</artifactId> | ||
| <version>2.1.0-SNAPSHOT</version> | ||
| <relativePath>../../pom.xml</relativePath> | ||
| </parent> | ||
|
|
||
| <groupId>org.apache.spark</groupId> | ||
| <artifactId>spark-sql-kafka-0-10_2.11</artifactId> | ||
| <properties> | ||
| <sbt.project.name>sql-kafka-0-10</sbt.project.name> | ||
| </properties> | ||
| <packaging>jar</packaging> | ||
| <name>Kafka 0.10 Source for Structured Streaming</name> | ||
| <url>http://spark.apache.org/</url> | ||
|
|
||
| <dependencies> | ||
| <dependency> | ||
| <groupId>org.apache.spark</groupId> | ||
| <artifactId>spark-sql_${scala.binary.version}</artifactId> | ||
| <version>${project.version}</version> | ||
| <scope>provided</scope> | ||
| </dependency> | ||
| <dependency> | ||
| <groupId>org.apache.spark</groupId> | ||
| <artifactId>spark-sql_${scala.binary.version}</artifactId> | ||
| <version>${project.version}</version> | ||
| <type>test-jar</type> | ||
| <scope>test</scope> | ||
| </dependency> | ||
| <dependency> | ||
| <groupId>org.apache.kafka</groupId> | ||
| <artifactId>kafka-clients</artifactId> | ||
| <version>0.10.0.1</version> | ||
| </dependency> | ||
| <dependency> | ||
| <groupId>org.apache.kafka</groupId> | ||
| <artifactId>kafka_${scala.binary.version}</artifactId> | ||
| <version>0.10.0.1</version> | ||
| <scope>test</scope> | ||
| </dependency> | ||
| <dependency> | ||
| <groupId>net.sf.jopt-simple</groupId> | ||
| <artifactId>jopt-simple</artifactId> | ||
| <version>3.2</version> | ||
| <scope>test</scope> | ||
| </dependency> | ||
| <dependency> | ||
| <groupId>org.scalacheck</groupId> | ||
| <artifactId>scalacheck_${scala.binary.version}</artifactId> | ||
| <scope>test</scope> | ||
| </dependency> | ||
| <dependency> | ||
| <groupId>org.apache.spark</groupId> | ||
| <artifactId>spark-tags_${scala.binary.version}</artifactId> | ||
| </dependency> | ||
| </dependencies> | ||
| <build> | ||
| <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory> | ||
| <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory> | ||
| </build> | ||
| </project> |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1 @@ | ||
| org.apache.spark.sql.kafka010.KafkaSourceProvider |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This should probably include an example of how to select key / value in the desired format.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Added an example to convert key/value to strings.