spark-instrumented-optimizer/docs/structured-streaming-kafka-integration.md

1144 lines
46 KiB
Markdown
Raw Normal View History

[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.
2016-10-05 19:45:45 -04:00
---
layout: global
title: Structured Streaming + Kafka Integration Guide (Kafka broker version 0.10.0 or higher)
license: |
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.
[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.
2016-10-05 19:45:45 -04:00
---
Structured Streaming integration for Kafka 0.10 to read data from and write data to Kafka.
[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.
2016-10-05 19:45:45 -04:00
## Linking
[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.
2016-10-05 19:45:45 -04:00
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}}
Please note that to use the headers functionality, your Kafka client version should be version 0.11.0.0 or up.
[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.
2016-10-05 19:45:45 -04:00
For Python applications, you need to add this above library and its dependencies when deploying your
application. See the [Deploying](#deploying) subsection below.
For experimenting on `spark-shell`, you need to add this above library and its dependencies too when invoking `spark-shell`. Also, see the [Deploying](#deploying) subsection below.
## Reading Data from Kafka
### Creating a Kafka Source for Streaming Queries
[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.
2016-10-05 19:45:45 -04:00
<div class="codetabs">
<div data-lang="scala" markdown="1">
{% highlight scala %}
// Subscribe to 1 topic
val df = spark
.readStream
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("subscribe", "topic1")
.load()
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
.as[(String, String)]
// Subscribe to 1 topic, with headers
val df = spark
.readStream
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("subscribe", "topic1")
.option("includeHeaders", "true")
.load()
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)", "headers")
.as[(String, String, Array[(String, Array[Byte])])]
// Subscribe to multiple topics
val df = spark
.readStream
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("subscribe", "topic1,topic2")
.load()
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
.as[(String, String)]
// Subscribe to a pattern
val df = spark
.readStream
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("subscribePattern", "topic.*")
.load()
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
.as[(String, String)]
{% endhighlight %}
[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.
2016-10-05 19:45:45 -04:00
</div>
<div data-lang="java" markdown="1">
{% highlight java %}
// Subscribe to 1 topic
Dataset<Row> df = spark
.readStream()
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("subscribe", "topic1")
.load();
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)");
// Subscribe to 1 topic, with headers
Dataset<Row> df = spark
.readStream()
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("subscribe", "topic1")
.option("includeHeaders", "true")
.load()
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)", "headers");
// Subscribe to multiple topics
Dataset<Row> df = spark
.readStream()
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("subscribe", "topic1,topic2")
.load();
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)");
// Subscribe to a pattern
Dataset<Row> df = spark
.readStream()
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("subscribePattern", "topic.*")
.load();
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)");
{% endhighlight %}
[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.
2016-10-05 19:45:45 -04:00
</div>
<div data-lang="python" markdown="1">
{% highlight python %}
# Subscribe to 1 topic
df = spark \
.readStream \
.format("kafka") \
.option("kafka.bootstrap.servers", "host1:port1,host2:port2") \
.option("subscribe", "topic1") \
.load()
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
# Subscribe to 1 topic, with headers
df = spark \
.readStream \
.format("kafka") \
.option("kafka.bootstrap.servers", "host1:port1,host2:port2") \
.option("subscribe", "topic1") \
.option("includeHeaders", "true") \
.load()
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)", "headers")
# Subscribe to multiple topics
df = spark \
.readStream \
.format("kafka") \
.option("kafka.bootstrap.servers", "host1:port1,host2:port2") \
.option("subscribe", "topic1,topic2") \
.load()
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
# Subscribe to a pattern
df = spark \
.readStream \
.format("kafka") \
.option("kafka.bootstrap.servers", "host1:port1,host2:port2") \
.option("subscribePattern", "topic.*") \
.load()
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
{% endhighlight %}
[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.
2016-10-05 19:45:45 -04:00
</div>
</div>
### Creating a Kafka Source for Batch Queries
If you have a use case that is better suited to batch processing,
you can create a Dataset/DataFrame for a defined range of offsets.
<div class="codetabs">
<div data-lang="scala" markdown="1">
{% highlight scala %}
// Subscribe to 1 topic defaults to the earliest and latest offsets
val df = spark
.read
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("subscribe", "topic1")
.load()
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
.as[(String, String)]
// Subscribe to multiple topics, specifying explicit Kafka offsets
val df = spark
.read
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("subscribe", "topic1,topic2")
.option("startingOffsets", """{"topic1":{"0":23,"1":-2},"topic2":{"0":-2}}""")
.option("endingOffsets", """{"topic1":{"0":50,"1":-1},"topic2":{"0":-1}}""")
.load()
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
.as[(String, String)]
// Subscribe to a pattern, at the earliest and latest offsets
val df = spark
.read
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("subscribePattern", "topic.*")
.option("startingOffsets", "earliest")
.option("endingOffsets", "latest")
.load()
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
.as[(String, String)]
{% endhighlight %}
</div>
<div data-lang="java" markdown="1">
{% highlight java %}
// Subscribe to 1 topic defaults to the earliest and latest offsets
Dataset<Row> df = spark
.read()
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("subscribe", "topic1")
.load();
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)");
// Subscribe to multiple topics, specifying explicit Kafka offsets
Dataset<Row> df = spark
.read()
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("subscribe", "topic1,topic2")
.option("startingOffsets", "{\"topic1\":{\"0\":23,\"1\":-2},\"topic2\":{\"0\":-2}}")
.option("endingOffsets", "{\"topic1\":{\"0\":50,\"1\":-1},\"topic2\":{\"0\":-1}}")
.load();
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)");
// Subscribe to a pattern, at the earliest and latest offsets
Dataset<Row> df = spark
.read()
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("subscribePattern", "topic.*")
.option("startingOffsets", "earliest")
.option("endingOffsets", "latest")
.load();
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)");
{% endhighlight %}
</div>
<div data-lang="python" markdown="1">
{% highlight python %}
# Subscribe to 1 topic defaults to the earliest and latest offsets
df = spark \
.read \
.format("kafka") \
.option("kafka.bootstrap.servers", "host1:port1,host2:port2") \
.option("subscribe", "topic1") \
.load()
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
# Subscribe to multiple topics, specifying explicit Kafka offsets
df = spark \
.read \
.format("kafka") \
.option("kafka.bootstrap.servers", "host1:port1,host2:port2") \
.option("subscribe", "topic1,topic2") \
.option("startingOffsets", """{"topic1":{"0":23,"1":-2},"topic2":{"0":-2}}""") \
.option("endingOffsets", """{"topic1":{"0":50,"1":-1},"topic2":{"0":-1}}""") \
.load()
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
# Subscribe to a pattern, at the earliest and latest offsets
df = spark \
.read \
.format("kafka") \
.option("kafka.bootstrap.servers", "host1:port1,host2:port2") \
.option("subscribePattern", "topic.*") \
.option("startingOffsets", "earliest") \
.option("endingOffsets", "latest") \
.load()
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
{% endhighlight %}
</div>
</div>
[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.
2016-10-05 19:45:45 -04:00
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>timestamp</td>
[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.
2016-10-05 19:45:45 -04:00
</tr>
<tr>
<td>timestampType</td>
<td>int</td>
</tr>
<tr>
<td>headers (optional)</td>
<td>array</td>
</tr>
[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.
2016-10-05 19:45:45 -04:00
</table>
The following options must be set for the Kafka source
for both batch and streaming queries.
[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.
2016-10-05 19:45:45 -04:00
<table class="table">
<tr><th>Option</th><th>value</th><th>meaning</th></tr>
<tr>
<td>assign</td>
<td>json string {"topicA":[0,1],"topicB":[2,4]}</td>
<td>Specific TopicPartitions to consume.
Only one of "assign", "subscribe" or "subscribePattern"
options can be specified for Kafka source.</td>
</tr>
[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.
2016-10-05 19:45:45 -04:00
<tr>
<td>subscribe</td>
<td>A comma-separated list of topics</td>
<td>The topic list to subscribe.
Only one of "assign", "subscribe" or "subscribePattern"
options can be specified for Kafka source.</td>
[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.
2016-10-05 19:45:45 -04:00
</tr>
<tr>
<td>subscribePattern</td>
<td>Java regex string</td>
<td>The pattern used to subscribe to topic(s).
Only one of "assign, "subscribe" or "subscribePattern"
[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.
2016-10-05 19:45:45 -04:00
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">
<tr><th>Option</th><th>value</th><th>default</th><th>query type</th><th>meaning</th></tr>
[SPARK-29223][SQL][SS] New option to specify timestamp on all subscribing topic-partitions in Kafka source ### What changes were proposed in this pull request? This patch is a follow-up of SPARK-26848 (#23747). In SPARK-26848, we decided to open possibility to let end users set individual timestamp per partition. But in many cases, specifying timestamp represents the intention that we would want to go back to specific timestamp and reprocess records, which should be applied to all topics and partitions. This patch proposes to provide a way to set a global timestamp across topic-partitions which the source is subscribing to, so that end users can set all offsets by specific timestamp easily. To provide the way to config the timestamp easier, the new options only receive "a" timestamp for start/end timestamp. New options introduced in this PR: * startingTimestamp * endingTimestamp All two options receive timestamp as string. There're priorities for options regarding starting/ending offset as we will have three options for start offsets and another three options for end offsets. Priorities are following: * starting offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets * ending offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets ### Why are the changes needed? Existing option to specify timestamp as offset is quite verbose if there're a lot of partitions across topics. Suppose there're 100s of partitions in a topic, the json should contain 100s of times of the same timestamp. Also, the number of partitions can also change, which requires either: * fixing the code if the json is statically created * introducing the dependencies on Kafka client and deal with Kafka API on crafting json programmatically Both approaches are even not "acceptable" if we're dealing with ad-hoc query; anyone doesn't want to write the code more complicated than the query itself. Flink [provides the option](https://ci.apache.org/projects/flink/flink-docs-release-1.13/docs/connectors/datastream/kafka/#kafka-consumers-start-position-configuration) to specify a timestamp for all topic-partitions like this PR, and even doesn't provide the option to specify the timestamp per topic-partition. With this PR, end users are only required to provide a single timestamp value. No more complicated JSON format end users need to know about the structure. ### Does this PR introduce _any_ user-facing change? Yes, this PR introduces two new options, described in above section. Doc changes are following: ![스크린샷 2021-05-21 오후 12 01 02](https://user-images.githubusercontent.com/1317309/119076244-3034e680-ba2d-11eb-8323-0e227932d2e5.png) ![스크린샷 2021-05-21 오후 12 01 12](https://user-images.githubusercontent.com/1317309/119076255-35923100-ba2d-11eb-9d79-538a7f9ee738.png) ![스크린샷 2021-05-21 오후 12 01 24](https://user-images.githubusercontent.com/1317309/119076264-39be4e80-ba2d-11eb-8265-ac158f55c360.png) ![스크린샷 2021-05-21 오후 12 06 01](https://user-images.githubusercontent.com/1317309/119076271-3d51d580-ba2d-11eb-98ea-35fd72b1bbfc.png) ### How was this patch tested? New UTs covering new functionalities. Also manually tested via simple batch & streaming queries. Closes #32609 from HeartSaVioR/SPARK-29223-v2. Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com> Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-05-25 08:43:49 -04:00
<tr>
<td>startingTimestamp</td>
<td>timestamp string e.g. "1000"</td>
<td>none (next preference is <code>startingOffsetsByTimestamp</code>)</td>
<td>streaming and batch</td>
<td>The start point of timestamp when a query is started, a string specifying a starting timestamp for
all partitions in topics being subscribed. Please refer the details on timestamp offset options below. If Kafka doesn't return the matched offset,
the query will fail immediately to prevent unintended read from such partition. (This is a kind of limitation as of now, and will be addressed in near future.)<p/>
<p/>
Note1: <code>startingTimestamp</code> takes precedence over <code>startingOffsetsByTimestamp</code> and <code>startingOffsets</code>.<p/>
Note2: For streaming queries, this only applies when a new query is started, and that resuming will
always pick up from where the query left off. Newly discovered partitions during a query will start at
earliest.</td>
</tr>
[SPARK-26848][SQL][SS] Introduce new option to Kafka source: offset by timestamp (starting/ending) ## What changes were proposed in this pull request? This patch introduces new options "startingOffsetsByTimestamp" and "endingOffsetsByTimestamp" to set specific timestamp per topic (since we're unlikely to set the different value per partition) to let source starts reading from offsets which have equal of greater timestamp, and ends reading until offsets which have equal of greater timestamp. The new option would be optional of course, and take preference over existing offset options. ## How was this patch tested? New unit tests added. Also manually tested basic functionality with Kafka 2.0.0 server. Running query below ``` val df = spark.read.format("kafka") .option("kafka.bootstrap.servers", "localhost:9092") .option("subscribe", "spark_26848_test_v1,spark_26848_test_2_v1") .option("startingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669142193, "spark_26848_test_2_v1": 1549669240965}""") .option("endingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669265676, "spark_26848_test_2_v1": 1549699265676}""") .load().selectExpr("CAST(value AS STRING)") df.show() ``` with below records (one string which number part remarks when they're put after such timestamp) in topic `spark_26848_test_v1` ``` hello1 1549669142193 world1 1549669142193 hellow1 1549669240965 world1 1549669240965 hello1 1549669265676 world1 1549669265676 ``` topic `spark_26848_test_2_v1` ``` hello2 1549669142193 world2 1549669142193 hello2 1549669240965 world2 1549669240965 hello2 1549669265676 world2 1549669265676 ``` the result of `df.show()` follows: ``` +--------------------+ | value| +--------------------+ |world1 1549669240965| |world1 1549669142193| |world2 1549669240965| |hello2 1549669240965| |hellow1 154966924...| |hello2 1549669265676| |hello1 1549669142193| |world2 1549669265676| +--------------------+ ``` Note that endingOffsets (as well as endingOffsetsByTimestamp) are exclusive. Closes #23747 from HeartSaVioR/SPARK-26848. Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-23 20:25:36 -04:00
<tr>
<td>startingOffsetsByTimestamp</td>
<td>json string
""" {"topicA":{"0": 1000, "1": 1000}, "topicB": {"0": 2000, "1": 2000}} """
</td>
[SPARK-29223][SQL][SS] New option to specify timestamp on all subscribing topic-partitions in Kafka source ### What changes were proposed in this pull request? This patch is a follow-up of SPARK-26848 (#23747). In SPARK-26848, we decided to open possibility to let end users set individual timestamp per partition. But in many cases, specifying timestamp represents the intention that we would want to go back to specific timestamp and reprocess records, which should be applied to all topics and partitions. This patch proposes to provide a way to set a global timestamp across topic-partitions which the source is subscribing to, so that end users can set all offsets by specific timestamp easily. To provide the way to config the timestamp easier, the new options only receive "a" timestamp for start/end timestamp. New options introduced in this PR: * startingTimestamp * endingTimestamp All two options receive timestamp as string. There're priorities for options regarding starting/ending offset as we will have three options for start offsets and another three options for end offsets. Priorities are following: * starting offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets * ending offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets ### Why are the changes needed? Existing option to specify timestamp as offset is quite verbose if there're a lot of partitions across topics. Suppose there're 100s of partitions in a topic, the json should contain 100s of times of the same timestamp. Also, the number of partitions can also change, which requires either: * fixing the code if the json is statically created * introducing the dependencies on Kafka client and deal with Kafka API on crafting json programmatically Both approaches are even not "acceptable" if we're dealing with ad-hoc query; anyone doesn't want to write the code more complicated than the query itself. Flink [provides the option](https://ci.apache.org/projects/flink/flink-docs-release-1.13/docs/connectors/datastream/kafka/#kafka-consumers-start-position-configuration) to specify a timestamp for all topic-partitions like this PR, and even doesn't provide the option to specify the timestamp per topic-partition. With this PR, end users are only required to provide a single timestamp value. No more complicated JSON format end users need to know about the structure. ### Does this PR introduce _any_ user-facing change? Yes, this PR introduces two new options, described in above section. Doc changes are following: ![스크린샷 2021-05-21 오후 12 01 02](https://user-images.githubusercontent.com/1317309/119076244-3034e680-ba2d-11eb-8323-0e227932d2e5.png) ![스크린샷 2021-05-21 오후 12 01 12](https://user-images.githubusercontent.com/1317309/119076255-35923100-ba2d-11eb-9d79-538a7f9ee738.png) ![스크린샷 2021-05-21 오후 12 01 24](https://user-images.githubusercontent.com/1317309/119076264-39be4e80-ba2d-11eb-8265-ac158f55c360.png) ![스크린샷 2021-05-21 오후 12 06 01](https://user-images.githubusercontent.com/1317309/119076271-3d51d580-ba2d-11eb-98ea-35fd72b1bbfc.png) ### How was this patch tested? New UTs covering new functionalities. Also manually tested via simple batch & streaming queries. Closes #32609 from HeartSaVioR/SPARK-29223-v2. Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com> Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-05-25 08:43:49 -04:00
<td>none (next preference is <code>startingOffsets</code>)</td>
[SPARK-26848][SQL][SS] Introduce new option to Kafka source: offset by timestamp (starting/ending) ## What changes were proposed in this pull request? This patch introduces new options "startingOffsetsByTimestamp" and "endingOffsetsByTimestamp" to set specific timestamp per topic (since we're unlikely to set the different value per partition) to let source starts reading from offsets which have equal of greater timestamp, and ends reading until offsets which have equal of greater timestamp. The new option would be optional of course, and take preference over existing offset options. ## How was this patch tested? New unit tests added. Also manually tested basic functionality with Kafka 2.0.0 server. Running query below ``` val df = spark.read.format("kafka") .option("kafka.bootstrap.servers", "localhost:9092") .option("subscribe", "spark_26848_test_v1,spark_26848_test_2_v1") .option("startingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669142193, "spark_26848_test_2_v1": 1549669240965}""") .option("endingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669265676, "spark_26848_test_2_v1": 1549699265676}""") .load().selectExpr("CAST(value AS STRING)") df.show() ``` with below records (one string which number part remarks when they're put after such timestamp) in topic `spark_26848_test_v1` ``` hello1 1549669142193 world1 1549669142193 hellow1 1549669240965 world1 1549669240965 hello1 1549669265676 world1 1549669265676 ``` topic `spark_26848_test_2_v1` ``` hello2 1549669142193 world2 1549669142193 hello2 1549669240965 world2 1549669240965 hello2 1549669265676 world2 1549669265676 ``` the result of `df.show()` follows: ``` +--------------------+ | value| +--------------------+ |world1 1549669240965| |world1 1549669142193| |world2 1549669240965| |hello2 1549669240965| |hellow1 154966924...| |hello2 1549669265676| |hello1 1549669142193| |world2 1549669265676| +--------------------+ ``` Note that endingOffsets (as well as endingOffsetsByTimestamp) are exclusive. Closes #23747 from HeartSaVioR/SPARK-26848. Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-23 20:25:36 -04:00
<td>streaming and batch</td>
<td>The start point of timestamp when a query is started, a json string specifying a starting timestamp for
[SPARK-29223][SQL][SS] New option to specify timestamp on all subscribing topic-partitions in Kafka source ### What changes were proposed in this pull request? This patch is a follow-up of SPARK-26848 (#23747). In SPARK-26848, we decided to open possibility to let end users set individual timestamp per partition. But in many cases, specifying timestamp represents the intention that we would want to go back to specific timestamp and reprocess records, which should be applied to all topics and partitions. This patch proposes to provide a way to set a global timestamp across topic-partitions which the source is subscribing to, so that end users can set all offsets by specific timestamp easily. To provide the way to config the timestamp easier, the new options only receive "a" timestamp for start/end timestamp. New options introduced in this PR: * startingTimestamp * endingTimestamp All two options receive timestamp as string. There're priorities for options regarding starting/ending offset as we will have three options for start offsets and another three options for end offsets. Priorities are following: * starting offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets * ending offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets ### Why are the changes needed? Existing option to specify timestamp as offset is quite verbose if there're a lot of partitions across topics. Suppose there're 100s of partitions in a topic, the json should contain 100s of times of the same timestamp. Also, the number of partitions can also change, which requires either: * fixing the code if the json is statically created * introducing the dependencies on Kafka client and deal with Kafka API on crafting json programmatically Both approaches are even not "acceptable" if we're dealing with ad-hoc query; anyone doesn't want to write the code more complicated than the query itself. Flink [provides the option](https://ci.apache.org/projects/flink/flink-docs-release-1.13/docs/connectors/datastream/kafka/#kafka-consumers-start-position-configuration) to specify a timestamp for all topic-partitions like this PR, and even doesn't provide the option to specify the timestamp per topic-partition. With this PR, end users are only required to provide a single timestamp value. No more complicated JSON format end users need to know about the structure. ### Does this PR introduce _any_ user-facing change? Yes, this PR introduces two new options, described in above section. Doc changes are following: ![스크린샷 2021-05-21 오후 12 01 02](https://user-images.githubusercontent.com/1317309/119076244-3034e680-ba2d-11eb-8323-0e227932d2e5.png) ![스크린샷 2021-05-21 오후 12 01 12](https://user-images.githubusercontent.com/1317309/119076255-35923100-ba2d-11eb-9d79-538a7f9ee738.png) ![스크린샷 2021-05-21 오후 12 01 24](https://user-images.githubusercontent.com/1317309/119076264-39be4e80-ba2d-11eb-8265-ac158f55c360.png) ![스크린샷 2021-05-21 오후 12 06 01](https://user-images.githubusercontent.com/1317309/119076271-3d51d580-ba2d-11eb-98ea-35fd72b1bbfc.png) ### How was this patch tested? New UTs covering new functionalities. Also manually tested via simple batch & streaming queries. Closes #32609 from HeartSaVioR/SPARK-29223-v2. Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com> Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-05-25 08:43:49 -04:00
each TopicPartition. Please refer the details on timestamp offset options below. If Kafka doesn't return the matched offset,
[SPARK-26848][SQL][SS] Introduce new option to Kafka source: offset by timestamp (starting/ending) ## What changes were proposed in this pull request? This patch introduces new options "startingOffsetsByTimestamp" and "endingOffsetsByTimestamp" to set specific timestamp per topic (since we're unlikely to set the different value per partition) to let source starts reading from offsets which have equal of greater timestamp, and ends reading until offsets which have equal of greater timestamp. The new option would be optional of course, and take preference over existing offset options. ## How was this patch tested? New unit tests added. Also manually tested basic functionality with Kafka 2.0.0 server. Running query below ``` val df = spark.read.format("kafka") .option("kafka.bootstrap.servers", "localhost:9092") .option("subscribe", "spark_26848_test_v1,spark_26848_test_2_v1") .option("startingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669142193, "spark_26848_test_2_v1": 1549669240965}""") .option("endingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669265676, "spark_26848_test_2_v1": 1549699265676}""") .load().selectExpr("CAST(value AS STRING)") df.show() ``` with below records (one string which number part remarks when they're put after such timestamp) in topic `spark_26848_test_v1` ``` hello1 1549669142193 world1 1549669142193 hellow1 1549669240965 world1 1549669240965 hello1 1549669265676 world1 1549669265676 ``` topic `spark_26848_test_2_v1` ``` hello2 1549669142193 world2 1549669142193 hello2 1549669240965 world2 1549669240965 hello2 1549669265676 world2 1549669265676 ``` the result of `df.show()` follows: ``` +--------------------+ | value| +--------------------+ |world1 1549669240965| |world1 1549669142193| |world2 1549669240965| |hello2 1549669240965| |hellow1 154966924...| |hello2 1549669265676| |hello1 1549669142193| |world2 1549669265676| +--------------------+ ``` Note that endingOffsets (as well as endingOffsetsByTimestamp) are exclusive. Closes #23747 from HeartSaVioR/SPARK-26848. Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-23 20:25:36 -04:00
the query will fail immediately to prevent unintended read from such partition. (This is a kind of limitation as of now, and will be addressed in near future.)<p/>
<p/>
[SPARK-29223][SQL][SS] New option to specify timestamp on all subscribing topic-partitions in Kafka source ### What changes were proposed in this pull request? This patch is a follow-up of SPARK-26848 (#23747). In SPARK-26848, we decided to open possibility to let end users set individual timestamp per partition. But in many cases, specifying timestamp represents the intention that we would want to go back to specific timestamp and reprocess records, which should be applied to all topics and partitions. This patch proposes to provide a way to set a global timestamp across topic-partitions which the source is subscribing to, so that end users can set all offsets by specific timestamp easily. To provide the way to config the timestamp easier, the new options only receive "a" timestamp for start/end timestamp. New options introduced in this PR: * startingTimestamp * endingTimestamp All two options receive timestamp as string. There're priorities for options regarding starting/ending offset as we will have three options for start offsets and another three options for end offsets. Priorities are following: * starting offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets * ending offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets ### Why are the changes needed? Existing option to specify timestamp as offset is quite verbose if there're a lot of partitions across topics. Suppose there're 100s of partitions in a topic, the json should contain 100s of times of the same timestamp. Also, the number of partitions can also change, which requires either: * fixing the code if the json is statically created * introducing the dependencies on Kafka client and deal with Kafka API on crafting json programmatically Both approaches are even not "acceptable" if we're dealing with ad-hoc query; anyone doesn't want to write the code more complicated than the query itself. Flink [provides the option](https://ci.apache.org/projects/flink/flink-docs-release-1.13/docs/connectors/datastream/kafka/#kafka-consumers-start-position-configuration) to specify a timestamp for all topic-partitions like this PR, and even doesn't provide the option to specify the timestamp per topic-partition. With this PR, end users are only required to provide a single timestamp value. No more complicated JSON format end users need to know about the structure. ### Does this PR introduce _any_ user-facing change? Yes, this PR introduces two new options, described in above section. Doc changes are following: ![스크린샷 2021-05-21 오후 12 01 02](https://user-images.githubusercontent.com/1317309/119076244-3034e680-ba2d-11eb-8323-0e227932d2e5.png) ![스크린샷 2021-05-21 오후 12 01 12](https://user-images.githubusercontent.com/1317309/119076255-35923100-ba2d-11eb-9d79-538a7f9ee738.png) ![스크린샷 2021-05-21 오후 12 01 24](https://user-images.githubusercontent.com/1317309/119076264-39be4e80-ba2d-11eb-8265-ac158f55c360.png) ![스크린샷 2021-05-21 오후 12 06 01](https://user-images.githubusercontent.com/1317309/119076271-3d51d580-ba2d-11eb-98ea-35fd72b1bbfc.png) ### How was this patch tested? New UTs covering new functionalities. Also manually tested via simple batch & streaming queries. Closes #32609 from HeartSaVioR/SPARK-29223-v2. Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com> Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-05-25 08:43:49 -04:00
Note1: <code>startingOffsetsByTimestamp</code> takes precedence over <code>startingOffsets</code>.<p/>
Note2: For streaming queries, this only applies when a new query is started, and that resuming will
[SPARK-26848][SQL][SS] Introduce new option to Kafka source: offset by timestamp (starting/ending) ## What changes were proposed in this pull request? This patch introduces new options "startingOffsetsByTimestamp" and "endingOffsetsByTimestamp" to set specific timestamp per topic (since we're unlikely to set the different value per partition) to let source starts reading from offsets which have equal of greater timestamp, and ends reading until offsets which have equal of greater timestamp. The new option would be optional of course, and take preference over existing offset options. ## How was this patch tested? New unit tests added. Also manually tested basic functionality with Kafka 2.0.0 server. Running query below ``` val df = spark.read.format("kafka") .option("kafka.bootstrap.servers", "localhost:9092") .option("subscribe", "spark_26848_test_v1,spark_26848_test_2_v1") .option("startingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669142193, "spark_26848_test_2_v1": 1549669240965}""") .option("endingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669265676, "spark_26848_test_2_v1": 1549699265676}""") .load().selectExpr("CAST(value AS STRING)") df.show() ``` with below records (one string which number part remarks when they're put after such timestamp) in topic `spark_26848_test_v1` ``` hello1 1549669142193 world1 1549669142193 hellow1 1549669240965 world1 1549669240965 hello1 1549669265676 world1 1549669265676 ``` topic `spark_26848_test_2_v1` ``` hello2 1549669142193 world2 1549669142193 hello2 1549669240965 world2 1549669240965 hello2 1549669265676 world2 1549669265676 ``` the result of `df.show()` follows: ``` +--------------------+ | value| +--------------------+ |world1 1549669240965| |world1 1549669142193| |world2 1549669240965| |hello2 1549669240965| |hellow1 154966924...| |hello2 1549669265676| |hello1 1549669142193| |world2 1549669265676| +--------------------+ ``` Note that endingOffsets (as well as endingOffsetsByTimestamp) are exclusive. Closes #23747 from HeartSaVioR/SPARK-26848. Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-23 20:25:36 -04:00
always pick up from where the query left off. Newly discovered partitions during a query will start at
earliest.</td>
</tr>
[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.
2016-10-05 19:45:45 -04:00
<tr>
<td>startingOffsets</td>
<td>"earliest", "latest" (streaming only), or json string
""" {"topicA":{"0":23,"1":-1},"topicB":{"0":-2}} """
</td>
<td>"latest" for streaming, "earliest" for batch</td>
<td>streaming and batch</td>
<td>The start point when a query is started, either "earliest" which is from the earliest offsets,
"latest" which is just from the latest offsets, or a json string specifying a starting offset for
each TopicPartition. In the json, -2 as an offset can be used to refer to earliest, -1 to latest.
Note: For batch queries, latest (either implicitly or by using -1 in json) is not allowed.
For streaming queries, this only applies when a new query is started, and that resuming will
always pick up from where the query left off. Newly discovered partitions during a query will start at
earliest.</td>
[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.
2016-10-05 19:45:45 -04:00
</tr>
[SPARK-29223][SQL][SS] New option to specify timestamp on all subscribing topic-partitions in Kafka source ### What changes were proposed in this pull request? This patch is a follow-up of SPARK-26848 (#23747). In SPARK-26848, we decided to open possibility to let end users set individual timestamp per partition. But in many cases, specifying timestamp represents the intention that we would want to go back to specific timestamp and reprocess records, which should be applied to all topics and partitions. This patch proposes to provide a way to set a global timestamp across topic-partitions which the source is subscribing to, so that end users can set all offsets by specific timestamp easily. To provide the way to config the timestamp easier, the new options only receive "a" timestamp for start/end timestamp. New options introduced in this PR: * startingTimestamp * endingTimestamp All two options receive timestamp as string. There're priorities for options regarding starting/ending offset as we will have three options for start offsets and another three options for end offsets. Priorities are following: * starting offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets * ending offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets ### Why are the changes needed? Existing option to specify timestamp as offset is quite verbose if there're a lot of partitions across topics. Suppose there're 100s of partitions in a topic, the json should contain 100s of times of the same timestamp. Also, the number of partitions can also change, which requires either: * fixing the code if the json is statically created * introducing the dependencies on Kafka client and deal with Kafka API on crafting json programmatically Both approaches are even not "acceptable" if we're dealing with ad-hoc query; anyone doesn't want to write the code more complicated than the query itself. Flink [provides the option](https://ci.apache.org/projects/flink/flink-docs-release-1.13/docs/connectors/datastream/kafka/#kafka-consumers-start-position-configuration) to specify a timestamp for all topic-partitions like this PR, and even doesn't provide the option to specify the timestamp per topic-partition. With this PR, end users are only required to provide a single timestamp value. No more complicated JSON format end users need to know about the structure. ### Does this PR introduce _any_ user-facing change? Yes, this PR introduces two new options, described in above section. Doc changes are following: ![스크린샷 2021-05-21 오후 12 01 02](https://user-images.githubusercontent.com/1317309/119076244-3034e680-ba2d-11eb-8323-0e227932d2e5.png) ![스크린샷 2021-05-21 오후 12 01 12](https://user-images.githubusercontent.com/1317309/119076255-35923100-ba2d-11eb-9d79-538a7f9ee738.png) ![스크린샷 2021-05-21 오후 12 01 24](https://user-images.githubusercontent.com/1317309/119076264-39be4e80-ba2d-11eb-8265-ac158f55c360.png) ![스크린샷 2021-05-21 오후 12 06 01](https://user-images.githubusercontent.com/1317309/119076271-3d51d580-ba2d-11eb-98ea-35fd72b1bbfc.png) ### How was this patch tested? New UTs covering new functionalities. Also manually tested via simple batch & streaming queries. Closes #32609 from HeartSaVioR/SPARK-29223-v2. Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com> Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-05-25 08:43:49 -04:00
<tr>
<td>endingTimestamp</td>
<td>timestamp string e.g. "1000"</td>
<td>none (next preference is <code>endingOffsetsByTimestamp</code>)</td>
<td>batch query</td>
<td>The end point when a batch query is ended, a json string specifying an ending timestamp for
all partitions in topics being subscribed. Please refer the details on timestamp offset options below.
If Kafka doesn't return the matched offset, the offset will be set to latest.<p/>
Note: <code>endingTimestamp</code> takes precedence over <code>endingOffsetsByTimestamp</code> and <code>endingOffsets</code>.<p/>
</td>
</tr>
[SPARK-26848][SQL][SS] Introduce new option to Kafka source: offset by timestamp (starting/ending) ## What changes were proposed in this pull request? This patch introduces new options "startingOffsetsByTimestamp" and "endingOffsetsByTimestamp" to set specific timestamp per topic (since we're unlikely to set the different value per partition) to let source starts reading from offsets which have equal of greater timestamp, and ends reading until offsets which have equal of greater timestamp. The new option would be optional of course, and take preference over existing offset options. ## How was this patch tested? New unit tests added. Also manually tested basic functionality with Kafka 2.0.0 server. Running query below ``` val df = spark.read.format("kafka") .option("kafka.bootstrap.servers", "localhost:9092") .option("subscribe", "spark_26848_test_v1,spark_26848_test_2_v1") .option("startingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669142193, "spark_26848_test_2_v1": 1549669240965}""") .option("endingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669265676, "spark_26848_test_2_v1": 1549699265676}""") .load().selectExpr("CAST(value AS STRING)") df.show() ``` with below records (one string which number part remarks when they're put after such timestamp) in topic `spark_26848_test_v1` ``` hello1 1549669142193 world1 1549669142193 hellow1 1549669240965 world1 1549669240965 hello1 1549669265676 world1 1549669265676 ``` topic `spark_26848_test_2_v1` ``` hello2 1549669142193 world2 1549669142193 hello2 1549669240965 world2 1549669240965 hello2 1549669265676 world2 1549669265676 ``` the result of `df.show()` follows: ``` +--------------------+ | value| +--------------------+ |world1 1549669240965| |world1 1549669142193| |world2 1549669240965| |hello2 1549669240965| |hellow1 154966924...| |hello2 1549669265676| |hello1 1549669142193| |world2 1549669265676| +--------------------+ ``` Note that endingOffsets (as well as endingOffsetsByTimestamp) are exclusive. Closes #23747 from HeartSaVioR/SPARK-26848. Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-23 20:25:36 -04:00
<tr>
<td>endingOffsetsByTimestamp</td>
<td>json string
""" {"topicA":{"0": 1000, "1": 1000}, "topicB": {"0": 2000, "1": 2000}} """
</td>
[SPARK-29223][SQL][SS] New option to specify timestamp on all subscribing topic-partitions in Kafka source ### What changes were proposed in this pull request? This patch is a follow-up of SPARK-26848 (#23747). In SPARK-26848, we decided to open possibility to let end users set individual timestamp per partition. But in many cases, specifying timestamp represents the intention that we would want to go back to specific timestamp and reprocess records, which should be applied to all topics and partitions. This patch proposes to provide a way to set a global timestamp across topic-partitions which the source is subscribing to, so that end users can set all offsets by specific timestamp easily. To provide the way to config the timestamp easier, the new options only receive "a" timestamp for start/end timestamp. New options introduced in this PR: * startingTimestamp * endingTimestamp All two options receive timestamp as string. There're priorities for options regarding starting/ending offset as we will have three options for start offsets and another three options for end offsets. Priorities are following: * starting offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets * ending offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets ### Why are the changes needed? Existing option to specify timestamp as offset is quite verbose if there're a lot of partitions across topics. Suppose there're 100s of partitions in a topic, the json should contain 100s of times of the same timestamp. Also, the number of partitions can also change, which requires either: * fixing the code if the json is statically created * introducing the dependencies on Kafka client and deal with Kafka API on crafting json programmatically Both approaches are even not "acceptable" if we're dealing with ad-hoc query; anyone doesn't want to write the code more complicated than the query itself. Flink [provides the option](https://ci.apache.org/projects/flink/flink-docs-release-1.13/docs/connectors/datastream/kafka/#kafka-consumers-start-position-configuration) to specify a timestamp for all topic-partitions like this PR, and even doesn't provide the option to specify the timestamp per topic-partition. With this PR, end users are only required to provide a single timestamp value. No more complicated JSON format end users need to know about the structure. ### Does this PR introduce _any_ user-facing change? Yes, this PR introduces two new options, described in above section. Doc changes are following: ![스크린샷 2021-05-21 오후 12 01 02](https://user-images.githubusercontent.com/1317309/119076244-3034e680-ba2d-11eb-8323-0e227932d2e5.png) ![스크린샷 2021-05-21 오후 12 01 12](https://user-images.githubusercontent.com/1317309/119076255-35923100-ba2d-11eb-9d79-538a7f9ee738.png) ![스크린샷 2021-05-21 오후 12 01 24](https://user-images.githubusercontent.com/1317309/119076264-39be4e80-ba2d-11eb-8265-ac158f55c360.png) ![스크린샷 2021-05-21 오후 12 06 01](https://user-images.githubusercontent.com/1317309/119076271-3d51d580-ba2d-11eb-98ea-35fd72b1bbfc.png) ### How was this patch tested? New UTs covering new functionalities. Also manually tested via simple batch & streaming queries. Closes #32609 from HeartSaVioR/SPARK-29223-v2. Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com> Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-05-25 08:43:49 -04:00
<td>none (next preference is <code>endingOffsets</code>)</td>
[SPARK-26848][SQL][SS] Introduce new option to Kafka source: offset by timestamp (starting/ending) ## What changes were proposed in this pull request? This patch introduces new options "startingOffsetsByTimestamp" and "endingOffsetsByTimestamp" to set specific timestamp per topic (since we're unlikely to set the different value per partition) to let source starts reading from offsets which have equal of greater timestamp, and ends reading until offsets which have equal of greater timestamp. The new option would be optional of course, and take preference over existing offset options. ## How was this patch tested? New unit tests added. Also manually tested basic functionality with Kafka 2.0.0 server. Running query below ``` val df = spark.read.format("kafka") .option("kafka.bootstrap.servers", "localhost:9092") .option("subscribe", "spark_26848_test_v1,spark_26848_test_2_v1") .option("startingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669142193, "spark_26848_test_2_v1": 1549669240965}""") .option("endingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669265676, "spark_26848_test_2_v1": 1549699265676}""") .load().selectExpr("CAST(value AS STRING)") df.show() ``` with below records (one string which number part remarks when they're put after such timestamp) in topic `spark_26848_test_v1` ``` hello1 1549669142193 world1 1549669142193 hellow1 1549669240965 world1 1549669240965 hello1 1549669265676 world1 1549669265676 ``` topic `spark_26848_test_2_v1` ``` hello2 1549669142193 world2 1549669142193 hello2 1549669240965 world2 1549669240965 hello2 1549669265676 world2 1549669265676 ``` the result of `df.show()` follows: ``` +--------------------+ | value| +--------------------+ |world1 1549669240965| |world1 1549669142193| |world2 1549669240965| |hello2 1549669240965| |hellow1 154966924...| |hello2 1549669265676| |hello1 1549669142193| |world2 1549669265676| +--------------------+ ``` Note that endingOffsets (as well as endingOffsetsByTimestamp) are exclusive. Closes #23747 from HeartSaVioR/SPARK-26848. Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-23 20:25:36 -04:00
<td>batch query</td>
<td>The end point when a batch query is ended, a json string specifying an ending timestamp for each TopicPartition.
[SPARK-29223][SQL][SS] New option to specify timestamp on all subscribing topic-partitions in Kafka source ### What changes were proposed in this pull request? This patch is a follow-up of SPARK-26848 (#23747). In SPARK-26848, we decided to open possibility to let end users set individual timestamp per partition. But in many cases, specifying timestamp represents the intention that we would want to go back to specific timestamp and reprocess records, which should be applied to all topics and partitions. This patch proposes to provide a way to set a global timestamp across topic-partitions which the source is subscribing to, so that end users can set all offsets by specific timestamp easily. To provide the way to config the timestamp easier, the new options only receive "a" timestamp for start/end timestamp. New options introduced in this PR: * startingTimestamp * endingTimestamp All two options receive timestamp as string. There're priorities for options regarding starting/ending offset as we will have three options for start offsets and another three options for end offsets. Priorities are following: * starting offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets * ending offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets ### Why are the changes needed? Existing option to specify timestamp as offset is quite verbose if there're a lot of partitions across topics. Suppose there're 100s of partitions in a topic, the json should contain 100s of times of the same timestamp. Also, the number of partitions can also change, which requires either: * fixing the code if the json is statically created * introducing the dependencies on Kafka client and deal with Kafka API on crafting json programmatically Both approaches are even not "acceptable" if we're dealing with ad-hoc query; anyone doesn't want to write the code more complicated than the query itself. Flink [provides the option](https://ci.apache.org/projects/flink/flink-docs-release-1.13/docs/connectors/datastream/kafka/#kafka-consumers-start-position-configuration) to specify a timestamp for all topic-partitions like this PR, and even doesn't provide the option to specify the timestamp per topic-partition. With this PR, end users are only required to provide a single timestamp value. No more complicated JSON format end users need to know about the structure. ### Does this PR introduce _any_ user-facing change? Yes, this PR introduces two new options, described in above section. Doc changes are following: ![스크린샷 2021-05-21 오후 12 01 02](https://user-images.githubusercontent.com/1317309/119076244-3034e680-ba2d-11eb-8323-0e227932d2e5.png) ![스크린샷 2021-05-21 오후 12 01 12](https://user-images.githubusercontent.com/1317309/119076255-35923100-ba2d-11eb-9d79-538a7f9ee738.png) ![스크린샷 2021-05-21 오후 12 01 24](https://user-images.githubusercontent.com/1317309/119076264-39be4e80-ba2d-11eb-8265-ac158f55c360.png) ![스크린샷 2021-05-21 오후 12 06 01](https://user-images.githubusercontent.com/1317309/119076271-3d51d580-ba2d-11eb-98ea-35fd72b1bbfc.png) ### How was this patch tested? New UTs covering new functionalities. Also manually tested via simple batch & streaming queries. Closes #32609 from HeartSaVioR/SPARK-29223-v2. Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com> Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-05-25 08:43:49 -04:00
Please refer the details on timestamp offset options below. If Kafka doesn't return the matched offset,
the offset will be set to latest.<p/>
Note: <code>endingOffsetsByTimestamp</code> takes precedence over <code>endingOffsets</code>.
[SPARK-26848][SQL][SS] Introduce new option to Kafka source: offset by timestamp (starting/ending) ## What changes were proposed in this pull request? This patch introduces new options "startingOffsetsByTimestamp" and "endingOffsetsByTimestamp" to set specific timestamp per topic (since we're unlikely to set the different value per partition) to let source starts reading from offsets which have equal of greater timestamp, and ends reading until offsets which have equal of greater timestamp. The new option would be optional of course, and take preference over existing offset options. ## How was this patch tested? New unit tests added. Also manually tested basic functionality with Kafka 2.0.0 server. Running query below ``` val df = spark.read.format("kafka") .option("kafka.bootstrap.servers", "localhost:9092") .option("subscribe", "spark_26848_test_v1,spark_26848_test_2_v1") .option("startingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669142193, "spark_26848_test_2_v1": 1549669240965}""") .option("endingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669265676, "spark_26848_test_2_v1": 1549699265676}""") .load().selectExpr("CAST(value AS STRING)") df.show() ``` with below records (one string which number part remarks when they're put after such timestamp) in topic `spark_26848_test_v1` ``` hello1 1549669142193 world1 1549669142193 hellow1 1549669240965 world1 1549669240965 hello1 1549669265676 world1 1549669265676 ``` topic `spark_26848_test_2_v1` ``` hello2 1549669142193 world2 1549669142193 hello2 1549669240965 world2 1549669240965 hello2 1549669265676 world2 1549669265676 ``` the result of `df.show()` follows: ``` +--------------------+ | value| +--------------------+ |world1 1549669240965| |world1 1549669142193| |world2 1549669240965| |hello2 1549669240965| |hellow1 154966924...| |hello2 1549669265676| |hello1 1549669142193| |world2 1549669265676| +--------------------+ ``` Note that endingOffsets (as well as endingOffsetsByTimestamp) are exclusive. Closes #23747 from HeartSaVioR/SPARK-26848. Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-23 20:25:36 -04:00
</td>
</tr>
<tr>
<td>endingOffsets</td>
<td>latest or json string
{"topicA":{"0":23,"1":-1},"topicB":{"0":-1}}
</td>
<td>latest</td>
<td>batch query</td>
<td>The end point when a batch query is ended, either "latest" which is just referred to the
latest, or a json string specifying an ending offset for each TopicPartition. In the json, -1
as an offset can be used to refer to latest, and -2 (earliest) as an offset is not allowed.</td>
</tr>
[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.
2016-10-05 19:45:45 -04:00
<tr>
<td>failOnDataLoss</td>
<td>true or false</td>
[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.
2016-10-05 19:45:45 -04:00
<td>true</td>
<td>streaming and batch</td>
<td>Whether to fail the query when it's possible that data is lost (e.g., topics are deleted, or
[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.
2016-10-05 19:45:45 -04:00
offsets are out of range). This may be a false alarm. You can disable it when it doesn't work
as you expected.</td>
[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.
2016-10-05 19:45:45 -04:00
</tr>
<tr>
<td>kafkaConsumer.pollTimeoutMs</td>
<td>long</td>
<td>120000</td>
<td>streaming and batch</td>
<td>The timeout in milliseconds to poll data from Kafka in executors. When not defined it falls
back to <code>spark.network.timeout</code>.</td>
[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.
2016-10-05 19:45:45 -04:00
</tr>
<tr>
<td>fetchOffset.numRetries</td>
<td>int</td>
<td>3</td>
<td>streaming and batch</td>
<td>Number of times to retry before giving up fetching Kafka offsets.</td>
[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.
2016-10-05 19:45:45 -04:00
</tr>
<tr>
<td>fetchOffset.retryIntervalMs</td>
<td>long</td>
<td>10</td>
<td>streaming and batch</td>
[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.
2016-10-05 19:45:45 -04:00
<td>milliseconds to wait before retrying to fetch Kafka offsets</td>
</tr>
<tr>
<td>maxOffsetsPerTrigger</td>
<td>long</td>
<td>none</td>
<td>streaming and batch</td>
<td>Rate limit on maximum number of offsets processed per trigger interval. The specified total number of offsets will be proportionally split across topicPartitions of different volume.</td>
</tr>
[SPARK-35312][SS] Introduce new Option in Kafka source to specify minimum number of records to read per trigger ### What changes were proposed in this pull request? This patch introduces a new option to specify the minimum number of offsets to read per trigger i.e. minOffsetsPerTrigger and maxTriggerDelay to avoid the infinite wait for the trigger. This new option will allow skipping trigger/batch when the number of records available in Kafka is low. This is a very useful feature in cases where we have a sudden burst of data at certain intervals in a day and data volume is low for the rest of the day. 'maxTriggerDelay' option will help to avoid cases of infinite delay in scheduling trigger and the trigger will happen irrespective of records available if the maxTriggerDelay time exceeds the last trigger. It would be an optional parameter with a default value of 15 mins. This option will be only applicable if minOffsetsPerTrigger is set. minOffsetsPerTrigger option would be optional of course, but once specified it would take precedence over maxOffestsPerTrigger which will be honored only after minOffsetsPerTrigger is satisfied. ### Why are the changes needed? There are many scenarios where there is a sudden burst of data at certain intervals in a day and data volume is low for the rest of the day. Tunning such jobs is difficult as decreasing trigger processing time increasing the number of batches and hence cluster resource usage and adds to small file issues. Increasing trigger processing time adds consumer lag. This patch tries to address this issue. ### How was this patch tested? This patch was tested by adding test cases as well as manually on a cluster where the job was running for a full one day with a data burst happening once a day. Here is the picture of databurst and hence consumer lag: <img width="1198" alt="Screenshot 2021-04-29 at 11 39 35 PM" src="https://user-images.githubusercontent.com/1044003/116997587-9b2ab180-acfa-11eb-91fd-524802ce3316.png"> This is how the job behaved at burst time running every 4.5 mins (which is the specified trigger time): <img width="1154" alt="Burst Time" src="https://user-images.githubusercontent.com/1044003/116997919-12f8dc00-acfb-11eb-9b0a-98387fc67560.png"> This is job behavior during the non-burst time where it is skipping 2 to 3 triggers and running once every 9 to 13.5 mins <img width="1154" alt="Non Burst Time" src="https://user-images.githubusercontent.com/1044003/116998244-8b5f9d00-acfb-11eb-8340-33d47149ef81.png"> Here are some more stats from the two-run i.e. one normal run and the other with minOffsetsperTrigger set: | Run | Data Size | Number of Batch Runs | Number of Files | | ------------- | ------------- |------------- |------------- | | Normal Run | 54.2 GB | 320 | 21968 | | Run with minOffsetsperTrigger | 54.2 GB | 120 | 12104 | Closes #32653 from satishgopalani/SPARK-35312. Authored-by: Satish Gopalani <satish.gopalani@pubmatic.com> Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-06-08 10:48:09 -04:00
<tr>
<td>minOffsetsPerTrigger</td>
<td>long</td>
<td>none</td>
<td>streaming and batch</td>
<td>Minimum number of offsets to be processed per trigger interval. The specified total number of offsets will
be proportionally split across topicPartitions of different volume.</td>
</tr>
<tr>
<td>maxTriggerDelay</td>
<td>time with units</td>
<td>15m</td>
<td>streaming and batch</td>
<td>Maximum amount of time for which trigger can be delayed between two triggers provided some data is available from the source.</td>
</tr>
<tr>
<td>minPartitions</td>
<td>int</td>
<td>none</td>
<td>streaming and batch</td>
<td>Desired minimum number of partitions to read from Kafka.
By default, Spark has a 1-1 mapping of topicPartitions to Spark partitions consuming from Kafka.
If you set this option to a value greater than your topicPartitions, Spark will divvy up large
Kafka partitions to smaller pieces. Please note that this configuration is like a <code>hint</code>: the
number of Spark tasks will be <strong>approximately</strong> <code>minPartitions</code>. It can be less or more depending on
rounding errors or Kafka partitions that didn't receive any new data.</td>
</tr>
<tr>
<td>groupIdPrefix</td>
<td>string</td>
<td>spark-kafka-source</td>
<td>streaming and batch</td>
<td>Prefix of consumer group identifiers (<code>group.id</code>) that are generated by structured streaming
queries. If "kafka.group.id" is set, this option will be ignored.</td>
</tr>
<tr>
<td>kafka.group.id</td>
<td>string</td>
<td>none</td>
<td>streaming and batch</td>
<td>The Kafka group id to use in Kafka consumer while reading from Kafka. Use this with caution.
By default, each query generates a unique group id for reading data. This ensures that each Kafka
source has its own consumer group that does not face interference from any other consumer, and
therefore can read all of the partitions of its subscribed topics. In some scenarios (for example,
Kafka group-based authorization), you may want to use a specific authorized group id to read data.
You can optionally set the group id. However, do this with extreme caution as it can cause
unexpected behavior. Concurrently running queries (both, batch and streaming) or sources with the
same group id are likely interfere with each other causing each query to read only part of the
data. This may also occur when queries are started/restarted in quick succession. To minimize such
issues, set the Kafka consumer session timeout (by setting option "kafka.session.timeout.ms") to
be very small. When this is set, option "groupIdPrefix" will be ignored.</td>
</tr>
<tr>
<td>includeHeaders</td>
<td>boolean</td>
<td>false</td>
<td>streaming and batch</td>
<td>Whether to include the Kafka headers in the row.</td>
</tr>
[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.
2016-10-05 19:45:45 -04:00
</table>
[SPARK-29223][SQL][SS] New option to specify timestamp on all subscribing topic-partitions in Kafka source ### What changes were proposed in this pull request? This patch is a follow-up of SPARK-26848 (#23747). In SPARK-26848, we decided to open possibility to let end users set individual timestamp per partition. But in many cases, specifying timestamp represents the intention that we would want to go back to specific timestamp and reprocess records, which should be applied to all topics and partitions. This patch proposes to provide a way to set a global timestamp across topic-partitions which the source is subscribing to, so that end users can set all offsets by specific timestamp easily. To provide the way to config the timestamp easier, the new options only receive "a" timestamp for start/end timestamp. New options introduced in this PR: * startingTimestamp * endingTimestamp All two options receive timestamp as string. There're priorities for options regarding starting/ending offset as we will have three options for start offsets and another three options for end offsets. Priorities are following: * starting offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets * ending offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets ### Why are the changes needed? Existing option to specify timestamp as offset is quite verbose if there're a lot of partitions across topics. Suppose there're 100s of partitions in a topic, the json should contain 100s of times of the same timestamp. Also, the number of partitions can also change, which requires either: * fixing the code if the json is statically created * introducing the dependencies on Kafka client and deal with Kafka API on crafting json programmatically Both approaches are even not "acceptable" if we're dealing with ad-hoc query; anyone doesn't want to write the code more complicated than the query itself. Flink [provides the option](https://ci.apache.org/projects/flink/flink-docs-release-1.13/docs/connectors/datastream/kafka/#kafka-consumers-start-position-configuration) to specify a timestamp for all topic-partitions like this PR, and even doesn't provide the option to specify the timestamp per topic-partition. With this PR, end users are only required to provide a single timestamp value. No more complicated JSON format end users need to know about the structure. ### Does this PR introduce _any_ user-facing change? Yes, this PR introduces two new options, described in above section. Doc changes are following: ![스크린샷 2021-05-21 오후 12 01 02](https://user-images.githubusercontent.com/1317309/119076244-3034e680-ba2d-11eb-8323-0e227932d2e5.png) ![스크린샷 2021-05-21 오후 12 01 12](https://user-images.githubusercontent.com/1317309/119076255-35923100-ba2d-11eb-9d79-538a7f9ee738.png) ![스크린샷 2021-05-21 오후 12 01 24](https://user-images.githubusercontent.com/1317309/119076264-39be4e80-ba2d-11eb-8265-ac158f55c360.png) ![스크린샷 2021-05-21 오후 12 06 01](https://user-images.githubusercontent.com/1317309/119076271-3d51d580-ba2d-11eb-98ea-35fd72b1bbfc.png) ### How was this patch tested? New UTs covering new functionalities. Also manually tested via simple batch & streaming queries. Closes #32609 from HeartSaVioR/SPARK-29223-v2. Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com> Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-05-25 08:43:49 -04:00
### Details on timestamp offset options
The returned offset for each partition is the earliest offset whose timestamp is greater than or equal to the given timestamp in the corresponding partition.
The behavior varies across options if Kafka doesn't return the matched offset - check the description of each option.
Spark simply passes the timestamp information to <code>KafkaConsumer.offsetsForTimes</code>, and doesn't interpret or reason about the value.
For more details on <code>KafkaConsumer.offsetsForTimes</code>, please refer <a href="http://kafka.apache.org/0101/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#offsetsForTimes(java.util.Map)">javadoc</a> for details.
Also, the meaning of <code>timestamp</code> here can be vary according to Kafka configuration (<code>log.message.timestamp.type</code>): please refer <a href="https://kafka.apache.org/documentation/">Kafka documentation</a> for further details.
Timestamp offset options require Kafka 0.10.1.0 or higher.
[SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API ### What changes were proposed in this pull request? Deprecated `KafkaConsumer.poll(long)` API calls may cause infinite wait in the driver. In this PR I've added a new `AdminClient` based offset fetching which is turned off by default. There is a new flag named `spark.sql.streaming.kafka.useDeprecatedOffsetFetching` (default: `true`) which can be set to `false` to reach the newly added functionality. The Structured Streaming migration guide contains more information what migration consideration must be done. Please see the following [doc](https://docs.google.com/document/d/1gAh0pKgZUgyqO2Re3sAy-fdYpe_SxpJ6DkeXE8R1P7E/edit?usp=sharing) for further details. The PR contains the following changes: * Added `AdminClient` based offset fetching * GroupId prefix feature removed from driver but only in `AdminClient` based approach (`AdminClient` doesn't need any GroupId) * GroupId override feature removed from driver but only in `AdminClient` based approach (`AdminClient` doesn't need any GroupId) * Additional unit tests * Code comment changes * Minor bugfixes here and there * Removed Kafka auto topic creation feature but only in `AdminClient` based approach (please see doc for rationale). In short, it's super hidden, not sure anybody ever used in production + error prone. * Added documentation to `ss-migration-guide` and `structured-streaming-kafka-integration` ### Why are the changes needed? Driver may hang forever. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing + additional unit tests. Cluster test with simple Kafka topic to another topic query. Documentation: ``` cd docs/ SKIP_API=1 jekyll build ``` Manual webpage check. Closes #29729 from gaborgsomogyi/SPARK-32032. Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com> Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2020-12-01 06:34:00 -05:00
### Offset fetching
In Spark 3.0 and before Spark uses <code>KafkaConsumer</code> for offset fetching which could cause infinite wait in the driver.
In Spark 3.1 a new configuration option added <code>spark.sql.streaming.kafka.useDeprecatedOffsetFetching</code> (default: <code>true</code>)
which could be set to `false` allowing Spark to use new offset fetching mechanism using <code>AdminClient</code>.
When the new mechanism used the following applies.
First of all the new approach supports Kafka brokers `0.11.0.0+`.
In Spark 3.0 and below, secure Kafka processing needed the following ACLs from driver perspective:
* Topic resource describe operation
* Topic resource read operation
* Group resource read operation
Since Spark 3.1, offsets can be obtained with <code>AdminClient</code> instead of <code>KafkaConsumer</code> and for that the following ACLs needed from driver perspective:
* Topic resource describe operation
Since <code>AdminClient</code> in driver is not connecting to consumer group, <code>group.id</code> based authorization will not work anymore (executors never done group based authorization).
Worth to mention executor side is behaving the exact same way like before (group prefix and override works).
### Consumer Caching
It's time-consuming to initialize Kafka consumers, especially in streaming scenarios where processing time is a key factor.
[SPARK-25151][SS] Apply Apache Commons Pool to KafkaDataConsumer ## What changes were proposed in this pull request? This patch does pooling for both kafka consumers as well as fetched data. The overall benefits of the patch are following: * Both pools support eviction on idle objects, which will help closing invalid idle objects which topic or partition are no longer be assigned to any tasks. * It also enables applying different policies on pool, which helps optimization of pooling for each pool. * We concerned about multiple tasks pointing same topic partition as well as same group id, and existing code can't handle this hence excess seek and fetch could happen. This patch properly handles the case. * It also makes the code always safe to leverage cache, hence no need to maintain reuseCache parameter. Moreover, pooling kafka consumers is implemented based on Apache Commons Pool, which also gives couple of benefits: * We can get rid of synchronization of KafkaDataConsumer object while acquiring and returning InternalKafkaConsumer. * We can extract the feature of object pool to outside of the class, so that the behaviors of the pool can be tested easily. * We can get various statistics for the object pool, and also be able to enable JMX for the pool. FetchedData instances are pooled by custom implementation of pool instead of leveraging Apache Commons Pool, because they have CacheKey as first key and "desired offset" as second key which "desired offset" is changing - I haven't found any general pool implementations supporting this. This patch brings additional dependency, Apache Commons Pool 2.6.0 into `spark-sql-kafka-0-10` module. ## How was this patch tested? Existing unit tests as well as new tests for object pool. Also did some experiment regarding proving concurrent access of consumers for same topic partition. * Made change on both sides (master and patch) to log when creating Kafka consumer or fetching records from Kafka is happening. * branches * master: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-master-ref-debugging * patch: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-debugging * Test query (doing self-join) * https://gist.github.com/HeartSaVioR/d831974c3f25c02846f4b15b8d232cc2 * Ran query from spark-shell, with using `local[*]` to maximize the chance to have concurrent access * Collected the count of fetch requests on Kafka via command: `grep "creating new Kafka consumer" logfile | wc -l` * Collected the count of creating Kafka consumers via command: `grep "fetching data from Kafka consumer" logfile | wc -l` Topic and data distribution is follow: ``` truck_speed_events_stream_spark_25151_v1:0:99440 truck_speed_events_stream_spark_25151_v1:1:99489 truck_speed_events_stream_spark_25151_v1:2:397759 truck_speed_events_stream_spark_25151_v1:3:198917 truck_speed_events_stream_spark_25151_v1:4:99484 truck_speed_events_stream_spark_25151_v1:5:497320 truck_speed_events_stream_spark_25151_v1:6:99430 truck_speed_events_stream_spark_25151_v1:7:397887 truck_speed_events_stream_spark_25151_v1:8:397813 truck_speed_events_stream_spark_25151_v1:9:0 ``` The experiment only used smallest 4 partitions (0, 1, 4, 6) from these partitions to finish the query earlier. The result of experiment is below: branch | create Kafka consumer | fetch request -- | -- | -- master | 1986 | 2837 patch | 8 | 1706 Closes #22138 from HeartSaVioR/SPARK-25151. Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Co-authored-by: Jungtaek Lim <kabhwan@gmail.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-09-04 13:17:38 -04:00
Because of this, Spark pools Kafka consumers on executors, by leveraging Apache Commons Pool.
The caching key is built up from the following information:
* Topic name
* Topic partition
* Group ID
[SPARK-25151][SS] Apply Apache Commons Pool to KafkaDataConsumer ## What changes were proposed in this pull request? This patch does pooling for both kafka consumers as well as fetched data. The overall benefits of the patch are following: * Both pools support eviction on idle objects, which will help closing invalid idle objects which topic or partition are no longer be assigned to any tasks. * It also enables applying different policies on pool, which helps optimization of pooling for each pool. * We concerned about multiple tasks pointing same topic partition as well as same group id, and existing code can't handle this hence excess seek and fetch could happen. This patch properly handles the case. * It also makes the code always safe to leverage cache, hence no need to maintain reuseCache parameter. Moreover, pooling kafka consumers is implemented based on Apache Commons Pool, which also gives couple of benefits: * We can get rid of synchronization of KafkaDataConsumer object while acquiring and returning InternalKafkaConsumer. * We can extract the feature of object pool to outside of the class, so that the behaviors of the pool can be tested easily. * We can get various statistics for the object pool, and also be able to enable JMX for the pool. FetchedData instances are pooled by custom implementation of pool instead of leveraging Apache Commons Pool, because they have CacheKey as first key and "desired offset" as second key which "desired offset" is changing - I haven't found any general pool implementations supporting this. This patch brings additional dependency, Apache Commons Pool 2.6.0 into `spark-sql-kafka-0-10` module. ## How was this patch tested? Existing unit tests as well as new tests for object pool. Also did some experiment regarding proving concurrent access of consumers for same topic partition. * Made change on both sides (master and patch) to log when creating Kafka consumer or fetching records from Kafka is happening. * branches * master: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-master-ref-debugging * patch: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-debugging * Test query (doing self-join) * https://gist.github.com/HeartSaVioR/d831974c3f25c02846f4b15b8d232cc2 * Ran query from spark-shell, with using `local[*]` to maximize the chance to have concurrent access * Collected the count of fetch requests on Kafka via command: `grep "creating new Kafka consumer" logfile | wc -l` * Collected the count of creating Kafka consumers via command: `grep "fetching data from Kafka consumer" logfile | wc -l` Topic and data distribution is follow: ``` truck_speed_events_stream_spark_25151_v1:0:99440 truck_speed_events_stream_spark_25151_v1:1:99489 truck_speed_events_stream_spark_25151_v1:2:397759 truck_speed_events_stream_spark_25151_v1:3:198917 truck_speed_events_stream_spark_25151_v1:4:99484 truck_speed_events_stream_spark_25151_v1:5:497320 truck_speed_events_stream_spark_25151_v1:6:99430 truck_speed_events_stream_spark_25151_v1:7:397887 truck_speed_events_stream_spark_25151_v1:8:397813 truck_speed_events_stream_spark_25151_v1:9:0 ``` The experiment only used smallest 4 partitions (0, 1, 4, 6) from these partitions to finish the query earlier. The result of experiment is below: branch | create Kafka consumer | fetch request -- | -- | -- master | 1986 | 2837 patch | 8 | 1706 Closes #22138 from HeartSaVioR/SPARK-25151. Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Co-authored-by: Jungtaek Lim <kabhwan@gmail.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-09-04 13:17:38 -04:00
The following properties are available to configure the consumer pool:
<table class="table">
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
[SPARK-25151][SS] Apply Apache Commons Pool to KafkaDataConsumer ## What changes were proposed in this pull request? This patch does pooling for both kafka consumers as well as fetched data. The overall benefits of the patch are following: * Both pools support eviction on idle objects, which will help closing invalid idle objects which topic or partition are no longer be assigned to any tasks. * It also enables applying different policies on pool, which helps optimization of pooling for each pool. * We concerned about multiple tasks pointing same topic partition as well as same group id, and existing code can't handle this hence excess seek and fetch could happen. This patch properly handles the case. * It also makes the code always safe to leverage cache, hence no need to maintain reuseCache parameter. Moreover, pooling kafka consumers is implemented based on Apache Commons Pool, which also gives couple of benefits: * We can get rid of synchronization of KafkaDataConsumer object while acquiring and returning InternalKafkaConsumer. * We can extract the feature of object pool to outside of the class, so that the behaviors of the pool can be tested easily. * We can get various statistics for the object pool, and also be able to enable JMX for the pool. FetchedData instances are pooled by custom implementation of pool instead of leveraging Apache Commons Pool, because they have CacheKey as first key and "desired offset" as second key which "desired offset" is changing - I haven't found any general pool implementations supporting this. This patch brings additional dependency, Apache Commons Pool 2.6.0 into `spark-sql-kafka-0-10` module. ## How was this patch tested? Existing unit tests as well as new tests for object pool. Also did some experiment regarding proving concurrent access of consumers for same topic partition. * Made change on both sides (master and patch) to log when creating Kafka consumer or fetching records from Kafka is happening. * branches * master: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-master-ref-debugging * patch: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-debugging * Test query (doing self-join) * https://gist.github.com/HeartSaVioR/d831974c3f25c02846f4b15b8d232cc2 * Ran query from spark-shell, with using `local[*]` to maximize the chance to have concurrent access * Collected the count of fetch requests on Kafka via command: `grep "creating new Kafka consumer" logfile | wc -l` * Collected the count of creating Kafka consumers via command: `grep "fetching data from Kafka consumer" logfile | wc -l` Topic and data distribution is follow: ``` truck_speed_events_stream_spark_25151_v1:0:99440 truck_speed_events_stream_spark_25151_v1:1:99489 truck_speed_events_stream_spark_25151_v1:2:397759 truck_speed_events_stream_spark_25151_v1:3:198917 truck_speed_events_stream_spark_25151_v1:4:99484 truck_speed_events_stream_spark_25151_v1:5:497320 truck_speed_events_stream_spark_25151_v1:6:99430 truck_speed_events_stream_spark_25151_v1:7:397887 truck_speed_events_stream_spark_25151_v1:8:397813 truck_speed_events_stream_spark_25151_v1:9:0 ``` The experiment only used smallest 4 partitions (0, 1, 4, 6) from these partitions to finish the query earlier. The result of experiment is below: branch | create Kafka consumer | fetch request -- | -- | -- master | 1986 | 2837 patch | 8 | 1706 Closes #22138 from HeartSaVioR/SPARK-25151. Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Co-authored-by: Jungtaek Lim <kabhwan@gmail.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-09-04 13:17:38 -04:00
<tr>
<td>spark.kafka.consumer.cache.capacity</td>
<td>64</td>
<td>The maximum number of consumers cached. Please note that it's a soft limit.</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>3.0.0</td>
[SPARK-25151][SS] Apply Apache Commons Pool to KafkaDataConsumer ## What changes were proposed in this pull request? This patch does pooling for both kafka consumers as well as fetched data. The overall benefits of the patch are following: * Both pools support eviction on idle objects, which will help closing invalid idle objects which topic or partition are no longer be assigned to any tasks. * It also enables applying different policies on pool, which helps optimization of pooling for each pool. * We concerned about multiple tasks pointing same topic partition as well as same group id, and existing code can't handle this hence excess seek and fetch could happen. This patch properly handles the case. * It also makes the code always safe to leverage cache, hence no need to maintain reuseCache parameter. Moreover, pooling kafka consumers is implemented based on Apache Commons Pool, which also gives couple of benefits: * We can get rid of synchronization of KafkaDataConsumer object while acquiring and returning InternalKafkaConsumer. * We can extract the feature of object pool to outside of the class, so that the behaviors of the pool can be tested easily. * We can get various statistics for the object pool, and also be able to enable JMX for the pool. FetchedData instances are pooled by custom implementation of pool instead of leveraging Apache Commons Pool, because they have CacheKey as first key and "desired offset" as second key which "desired offset" is changing - I haven't found any general pool implementations supporting this. This patch brings additional dependency, Apache Commons Pool 2.6.0 into `spark-sql-kafka-0-10` module. ## How was this patch tested? Existing unit tests as well as new tests for object pool. Also did some experiment regarding proving concurrent access of consumers for same topic partition. * Made change on both sides (master and patch) to log when creating Kafka consumer or fetching records from Kafka is happening. * branches * master: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-master-ref-debugging * patch: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-debugging * Test query (doing self-join) * https://gist.github.com/HeartSaVioR/d831974c3f25c02846f4b15b8d232cc2 * Ran query from spark-shell, with using `local[*]` to maximize the chance to have concurrent access * Collected the count of fetch requests on Kafka via command: `grep "creating new Kafka consumer" logfile | wc -l` * Collected the count of creating Kafka consumers via command: `grep "fetching data from Kafka consumer" logfile | wc -l` Topic and data distribution is follow: ``` truck_speed_events_stream_spark_25151_v1:0:99440 truck_speed_events_stream_spark_25151_v1:1:99489 truck_speed_events_stream_spark_25151_v1:2:397759 truck_speed_events_stream_spark_25151_v1:3:198917 truck_speed_events_stream_spark_25151_v1:4:99484 truck_speed_events_stream_spark_25151_v1:5:497320 truck_speed_events_stream_spark_25151_v1:6:99430 truck_speed_events_stream_spark_25151_v1:7:397887 truck_speed_events_stream_spark_25151_v1:8:397813 truck_speed_events_stream_spark_25151_v1:9:0 ``` The experiment only used smallest 4 partitions (0, 1, 4, 6) from these partitions to finish the query earlier. The result of experiment is below: branch | create Kafka consumer | fetch request -- | -- | -- master | 1986 | 2837 patch | 8 | 1706 Closes #22138 from HeartSaVioR/SPARK-25151. Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Co-authored-by: Jungtaek Lim <kabhwan@gmail.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-09-04 13:17:38 -04:00
</tr>
<tr>
<td>spark.kafka.consumer.cache.timeout</td>
<td>5m (5 minutes)</td>
<td>The minimum amount of time a consumer may sit idle in the pool before it is eligible for eviction by the evictor.</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>3.0.0</td>
[SPARK-25151][SS] Apply Apache Commons Pool to KafkaDataConsumer ## What changes were proposed in this pull request? This patch does pooling for both kafka consumers as well as fetched data. The overall benefits of the patch are following: * Both pools support eviction on idle objects, which will help closing invalid idle objects which topic or partition are no longer be assigned to any tasks. * It also enables applying different policies on pool, which helps optimization of pooling for each pool. * We concerned about multiple tasks pointing same topic partition as well as same group id, and existing code can't handle this hence excess seek and fetch could happen. This patch properly handles the case. * It also makes the code always safe to leverage cache, hence no need to maintain reuseCache parameter. Moreover, pooling kafka consumers is implemented based on Apache Commons Pool, which also gives couple of benefits: * We can get rid of synchronization of KafkaDataConsumer object while acquiring and returning InternalKafkaConsumer. * We can extract the feature of object pool to outside of the class, so that the behaviors of the pool can be tested easily. * We can get various statistics for the object pool, and also be able to enable JMX for the pool. FetchedData instances are pooled by custom implementation of pool instead of leveraging Apache Commons Pool, because they have CacheKey as first key and "desired offset" as second key which "desired offset" is changing - I haven't found any general pool implementations supporting this. This patch brings additional dependency, Apache Commons Pool 2.6.0 into `spark-sql-kafka-0-10` module. ## How was this patch tested? Existing unit tests as well as new tests for object pool. Also did some experiment regarding proving concurrent access of consumers for same topic partition. * Made change on both sides (master and patch) to log when creating Kafka consumer or fetching records from Kafka is happening. * branches * master: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-master-ref-debugging * patch: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-debugging * Test query (doing self-join) * https://gist.github.com/HeartSaVioR/d831974c3f25c02846f4b15b8d232cc2 * Ran query from spark-shell, with using `local[*]` to maximize the chance to have concurrent access * Collected the count of fetch requests on Kafka via command: `grep "creating new Kafka consumer" logfile | wc -l` * Collected the count of creating Kafka consumers via command: `grep "fetching data from Kafka consumer" logfile | wc -l` Topic and data distribution is follow: ``` truck_speed_events_stream_spark_25151_v1:0:99440 truck_speed_events_stream_spark_25151_v1:1:99489 truck_speed_events_stream_spark_25151_v1:2:397759 truck_speed_events_stream_spark_25151_v1:3:198917 truck_speed_events_stream_spark_25151_v1:4:99484 truck_speed_events_stream_spark_25151_v1:5:497320 truck_speed_events_stream_spark_25151_v1:6:99430 truck_speed_events_stream_spark_25151_v1:7:397887 truck_speed_events_stream_spark_25151_v1:8:397813 truck_speed_events_stream_spark_25151_v1:9:0 ``` The experiment only used smallest 4 partitions (0, 1, 4, 6) from these partitions to finish the query earlier. The result of experiment is below: branch | create Kafka consumer | fetch request -- | -- | -- master | 1986 | 2837 patch | 8 | 1706 Closes #22138 from HeartSaVioR/SPARK-25151. Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Co-authored-by: Jungtaek Lim <kabhwan@gmail.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-09-04 13:17:38 -04:00
</tr>
<tr>
<td>spark.kafka.consumer.cache.evictorThreadRunInterval</td>
<td>1m (1 minute)</td>
<td>The interval of time between runs of the idle evictor thread for consumer pool. When non-positive, no idle evictor thread will be run.</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>3.0.0</td>
[SPARK-25151][SS] Apply Apache Commons Pool to KafkaDataConsumer ## What changes were proposed in this pull request? This patch does pooling for both kafka consumers as well as fetched data. The overall benefits of the patch are following: * Both pools support eviction on idle objects, which will help closing invalid idle objects which topic or partition are no longer be assigned to any tasks. * It also enables applying different policies on pool, which helps optimization of pooling for each pool. * We concerned about multiple tasks pointing same topic partition as well as same group id, and existing code can't handle this hence excess seek and fetch could happen. This patch properly handles the case. * It also makes the code always safe to leverage cache, hence no need to maintain reuseCache parameter. Moreover, pooling kafka consumers is implemented based on Apache Commons Pool, which also gives couple of benefits: * We can get rid of synchronization of KafkaDataConsumer object while acquiring and returning InternalKafkaConsumer. * We can extract the feature of object pool to outside of the class, so that the behaviors of the pool can be tested easily. * We can get various statistics for the object pool, and also be able to enable JMX for the pool. FetchedData instances are pooled by custom implementation of pool instead of leveraging Apache Commons Pool, because they have CacheKey as first key and "desired offset" as second key which "desired offset" is changing - I haven't found any general pool implementations supporting this. This patch brings additional dependency, Apache Commons Pool 2.6.0 into `spark-sql-kafka-0-10` module. ## How was this patch tested? Existing unit tests as well as new tests for object pool. Also did some experiment regarding proving concurrent access of consumers for same topic partition. * Made change on both sides (master and patch) to log when creating Kafka consumer or fetching records from Kafka is happening. * branches * master: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-master-ref-debugging * patch: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-debugging * Test query (doing self-join) * https://gist.github.com/HeartSaVioR/d831974c3f25c02846f4b15b8d232cc2 * Ran query from spark-shell, with using `local[*]` to maximize the chance to have concurrent access * Collected the count of fetch requests on Kafka via command: `grep "creating new Kafka consumer" logfile | wc -l` * Collected the count of creating Kafka consumers via command: `grep "fetching data from Kafka consumer" logfile | wc -l` Topic and data distribution is follow: ``` truck_speed_events_stream_spark_25151_v1:0:99440 truck_speed_events_stream_spark_25151_v1:1:99489 truck_speed_events_stream_spark_25151_v1:2:397759 truck_speed_events_stream_spark_25151_v1:3:198917 truck_speed_events_stream_spark_25151_v1:4:99484 truck_speed_events_stream_spark_25151_v1:5:497320 truck_speed_events_stream_spark_25151_v1:6:99430 truck_speed_events_stream_spark_25151_v1:7:397887 truck_speed_events_stream_spark_25151_v1:8:397813 truck_speed_events_stream_spark_25151_v1:9:0 ``` The experiment only used smallest 4 partitions (0, 1, 4, 6) from these partitions to finish the query earlier. The result of experiment is below: branch | create Kafka consumer | fetch request -- | -- | -- master | 1986 | 2837 patch | 8 | 1706 Closes #22138 from HeartSaVioR/SPARK-25151. Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Co-authored-by: Jungtaek Lim <kabhwan@gmail.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-09-04 13:17:38 -04:00
</tr>
<tr>
<td>spark.kafka.consumer.cache.jmx.enable</td>
<td>false</td>
[SPARK-25151][SS] Apply Apache Commons Pool to KafkaDataConsumer ## What changes were proposed in this pull request? This patch does pooling for both kafka consumers as well as fetched data. The overall benefits of the patch are following: * Both pools support eviction on idle objects, which will help closing invalid idle objects which topic or partition are no longer be assigned to any tasks. * It also enables applying different policies on pool, which helps optimization of pooling for each pool. * We concerned about multiple tasks pointing same topic partition as well as same group id, and existing code can't handle this hence excess seek and fetch could happen. This patch properly handles the case. * It also makes the code always safe to leverage cache, hence no need to maintain reuseCache parameter. Moreover, pooling kafka consumers is implemented based on Apache Commons Pool, which also gives couple of benefits: * We can get rid of synchronization of KafkaDataConsumer object while acquiring and returning InternalKafkaConsumer. * We can extract the feature of object pool to outside of the class, so that the behaviors of the pool can be tested easily. * We can get various statistics for the object pool, and also be able to enable JMX for the pool. FetchedData instances are pooled by custom implementation of pool instead of leveraging Apache Commons Pool, because they have CacheKey as first key and "desired offset" as second key which "desired offset" is changing - I haven't found any general pool implementations supporting this. This patch brings additional dependency, Apache Commons Pool 2.6.0 into `spark-sql-kafka-0-10` module. ## How was this patch tested? Existing unit tests as well as new tests for object pool. Also did some experiment regarding proving concurrent access of consumers for same topic partition. * Made change on both sides (master and patch) to log when creating Kafka consumer or fetching records from Kafka is happening. * branches * master: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-master-ref-debugging * patch: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-debugging * Test query (doing self-join) * https://gist.github.com/HeartSaVioR/d831974c3f25c02846f4b15b8d232cc2 * Ran query from spark-shell, with using `local[*]` to maximize the chance to have concurrent access * Collected the count of fetch requests on Kafka via command: `grep "creating new Kafka consumer" logfile | wc -l` * Collected the count of creating Kafka consumers via command: `grep "fetching data from Kafka consumer" logfile | wc -l` Topic and data distribution is follow: ``` truck_speed_events_stream_spark_25151_v1:0:99440 truck_speed_events_stream_spark_25151_v1:1:99489 truck_speed_events_stream_spark_25151_v1:2:397759 truck_speed_events_stream_spark_25151_v1:3:198917 truck_speed_events_stream_spark_25151_v1:4:99484 truck_speed_events_stream_spark_25151_v1:5:497320 truck_speed_events_stream_spark_25151_v1:6:99430 truck_speed_events_stream_spark_25151_v1:7:397887 truck_speed_events_stream_spark_25151_v1:8:397813 truck_speed_events_stream_spark_25151_v1:9:0 ``` The experiment only used smallest 4 partitions (0, 1, 4, 6) from these partitions to finish the query earlier. The result of experiment is below: branch | create Kafka consumer | fetch request -- | -- | -- master | 1986 | 2837 patch | 8 | 1706 Closes #22138 from HeartSaVioR/SPARK-25151. Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Co-authored-by: Jungtaek Lim <kabhwan@gmail.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-09-04 13:17:38 -04:00
<td>Enable or disable JMX for pools created with this configuration instance. Statistics of the pool are available via JMX instance.
The prefix of JMX name is set to "kafka010-cached-simple-kafka-consumer-pool".
</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>3.0.0</td>
[SPARK-25151][SS] Apply Apache Commons Pool to KafkaDataConsumer ## What changes were proposed in this pull request? This patch does pooling for both kafka consumers as well as fetched data. The overall benefits of the patch are following: * Both pools support eviction on idle objects, which will help closing invalid idle objects which topic or partition are no longer be assigned to any tasks. * It also enables applying different policies on pool, which helps optimization of pooling for each pool. * We concerned about multiple tasks pointing same topic partition as well as same group id, and existing code can't handle this hence excess seek and fetch could happen. This patch properly handles the case. * It also makes the code always safe to leverage cache, hence no need to maintain reuseCache parameter. Moreover, pooling kafka consumers is implemented based on Apache Commons Pool, which also gives couple of benefits: * We can get rid of synchronization of KafkaDataConsumer object while acquiring and returning InternalKafkaConsumer. * We can extract the feature of object pool to outside of the class, so that the behaviors of the pool can be tested easily. * We can get various statistics for the object pool, and also be able to enable JMX for the pool. FetchedData instances are pooled by custom implementation of pool instead of leveraging Apache Commons Pool, because they have CacheKey as first key and "desired offset" as second key which "desired offset" is changing - I haven't found any general pool implementations supporting this. This patch brings additional dependency, Apache Commons Pool 2.6.0 into `spark-sql-kafka-0-10` module. ## How was this patch tested? Existing unit tests as well as new tests for object pool. Also did some experiment regarding proving concurrent access of consumers for same topic partition. * Made change on both sides (master and patch) to log when creating Kafka consumer or fetching records from Kafka is happening. * branches * master: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-master-ref-debugging * patch: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-debugging * Test query (doing self-join) * https://gist.github.com/HeartSaVioR/d831974c3f25c02846f4b15b8d232cc2 * Ran query from spark-shell, with using `local[*]` to maximize the chance to have concurrent access * Collected the count of fetch requests on Kafka via command: `grep "creating new Kafka consumer" logfile | wc -l` * Collected the count of creating Kafka consumers via command: `grep "fetching data from Kafka consumer" logfile | wc -l` Topic and data distribution is follow: ``` truck_speed_events_stream_spark_25151_v1:0:99440 truck_speed_events_stream_spark_25151_v1:1:99489 truck_speed_events_stream_spark_25151_v1:2:397759 truck_speed_events_stream_spark_25151_v1:3:198917 truck_speed_events_stream_spark_25151_v1:4:99484 truck_speed_events_stream_spark_25151_v1:5:497320 truck_speed_events_stream_spark_25151_v1:6:99430 truck_speed_events_stream_spark_25151_v1:7:397887 truck_speed_events_stream_spark_25151_v1:8:397813 truck_speed_events_stream_spark_25151_v1:9:0 ``` The experiment only used smallest 4 partitions (0, 1, 4, 6) from these partitions to finish the query earlier. The result of experiment is below: branch | create Kafka consumer | fetch request -- | -- | -- master | 1986 | 2837 patch | 8 | 1706 Closes #22138 from HeartSaVioR/SPARK-25151. Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Co-authored-by: Jungtaek Lim <kabhwan@gmail.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-09-04 13:17:38 -04:00
</tr>
</table>
The size of the pool is limited by <code>spark.kafka.consumer.cache.capacity</code>,
but it works as "soft-limit" to not block Spark tasks.
Idle eviction thread periodically removes consumers which are not used longer than given timeout.
If this threshold is reached when borrowing, it tries to remove the least-used entry that is currently not in use.
If it cannot be removed, then the pool will keep growing. In the worst case, the pool will grow to
the max number of concurrent tasks that can run in the executor (that is, number of task slots).
If a task fails for any reason, the new task is executed with a newly created Kafka consumer for safety reasons.
At the same time, we invalidate all consumers in pool which have same caching key, to remove consumer which was used
in failed execution. Consumers which any other tasks are using will not be closed, but will be invalidated as well
when they are returned into pool.
Along with consumers, Spark pools the records fetched from Kafka separately, to let Kafka consumers stateless in point
of Spark's view, and maximize the efficiency of pooling. It leverages same cache key with Kafka consumers pool.
Note that it doesn't leverage Apache Commons Pool due to the difference of characteristics.
The following properties are available to configure the fetched data pool:
[SPARK-25151][SS] Apply Apache Commons Pool to KafkaDataConsumer ## What changes were proposed in this pull request? This patch does pooling for both kafka consumers as well as fetched data. The overall benefits of the patch are following: * Both pools support eviction on idle objects, which will help closing invalid idle objects which topic or partition are no longer be assigned to any tasks. * It also enables applying different policies on pool, which helps optimization of pooling for each pool. * We concerned about multiple tasks pointing same topic partition as well as same group id, and existing code can't handle this hence excess seek and fetch could happen. This patch properly handles the case. * It also makes the code always safe to leverage cache, hence no need to maintain reuseCache parameter. Moreover, pooling kafka consumers is implemented based on Apache Commons Pool, which also gives couple of benefits: * We can get rid of synchronization of KafkaDataConsumer object while acquiring and returning InternalKafkaConsumer. * We can extract the feature of object pool to outside of the class, so that the behaviors of the pool can be tested easily. * We can get various statistics for the object pool, and also be able to enable JMX for the pool. FetchedData instances are pooled by custom implementation of pool instead of leveraging Apache Commons Pool, because they have CacheKey as first key and "desired offset" as second key which "desired offset" is changing - I haven't found any general pool implementations supporting this. This patch brings additional dependency, Apache Commons Pool 2.6.0 into `spark-sql-kafka-0-10` module. ## How was this patch tested? Existing unit tests as well as new tests for object pool. Also did some experiment regarding proving concurrent access of consumers for same topic partition. * Made change on both sides (master and patch) to log when creating Kafka consumer or fetching records from Kafka is happening. * branches * master: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-master-ref-debugging * patch: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-debugging * Test query (doing self-join) * https://gist.github.com/HeartSaVioR/d831974c3f25c02846f4b15b8d232cc2 * Ran query from spark-shell, with using `local[*]` to maximize the chance to have concurrent access * Collected the count of fetch requests on Kafka via command: `grep "creating new Kafka consumer" logfile | wc -l` * Collected the count of creating Kafka consumers via command: `grep "fetching data from Kafka consumer" logfile | wc -l` Topic and data distribution is follow: ``` truck_speed_events_stream_spark_25151_v1:0:99440 truck_speed_events_stream_spark_25151_v1:1:99489 truck_speed_events_stream_spark_25151_v1:2:397759 truck_speed_events_stream_spark_25151_v1:3:198917 truck_speed_events_stream_spark_25151_v1:4:99484 truck_speed_events_stream_spark_25151_v1:5:497320 truck_speed_events_stream_spark_25151_v1:6:99430 truck_speed_events_stream_spark_25151_v1:7:397887 truck_speed_events_stream_spark_25151_v1:8:397813 truck_speed_events_stream_spark_25151_v1:9:0 ``` The experiment only used smallest 4 partitions (0, 1, 4, 6) from these partitions to finish the query earlier. The result of experiment is below: branch | create Kafka consumer | fetch request -- | -- | -- master | 1986 | 2837 patch | 8 | 1706 Closes #22138 from HeartSaVioR/SPARK-25151. Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Co-authored-by: Jungtaek Lim <kabhwan@gmail.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-09-04 13:17:38 -04:00
<table class="table">
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
[SPARK-25151][SS] Apply Apache Commons Pool to KafkaDataConsumer ## What changes were proposed in this pull request? This patch does pooling for both kafka consumers as well as fetched data. The overall benefits of the patch are following: * Both pools support eviction on idle objects, which will help closing invalid idle objects which topic or partition are no longer be assigned to any tasks. * It also enables applying different policies on pool, which helps optimization of pooling for each pool. * We concerned about multiple tasks pointing same topic partition as well as same group id, and existing code can't handle this hence excess seek and fetch could happen. This patch properly handles the case. * It also makes the code always safe to leverage cache, hence no need to maintain reuseCache parameter. Moreover, pooling kafka consumers is implemented based on Apache Commons Pool, which also gives couple of benefits: * We can get rid of synchronization of KafkaDataConsumer object while acquiring and returning InternalKafkaConsumer. * We can extract the feature of object pool to outside of the class, so that the behaviors of the pool can be tested easily. * We can get various statistics for the object pool, and also be able to enable JMX for the pool. FetchedData instances are pooled by custom implementation of pool instead of leveraging Apache Commons Pool, because they have CacheKey as first key and "desired offset" as second key which "desired offset" is changing - I haven't found any general pool implementations supporting this. This patch brings additional dependency, Apache Commons Pool 2.6.0 into `spark-sql-kafka-0-10` module. ## How was this patch tested? Existing unit tests as well as new tests for object pool. Also did some experiment regarding proving concurrent access of consumers for same topic partition. * Made change on both sides (master and patch) to log when creating Kafka consumer or fetching records from Kafka is happening. * branches * master: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-master-ref-debugging * patch: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-debugging * Test query (doing self-join) * https://gist.github.com/HeartSaVioR/d831974c3f25c02846f4b15b8d232cc2 * Ran query from spark-shell, with using `local[*]` to maximize the chance to have concurrent access * Collected the count of fetch requests on Kafka via command: `grep "creating new Kafka consumer" logfile | wc -l` * Collected the count of creating Kafka consumers via command: `grep "fetching data from Kafka consumer" logfile | wc -l` Topic and data distribution is follow: ``` truck_speed_events_stream_spark_25151_v1:0:99440 truck_speed_events_stream_spark_25151_v1:1:99489 truck_speed_events_stream_spark_25151_v1:2:397759 truck_speed_events_stream_spark_25151_v1:3:198917 truck_speed_events_stream_spark_25151_v1:4:99484 truck_speed_events_stream_spark_25151_v1:5:497320 truck_speed_events_stream_spark_25151_v1:6:99430 truck_speed_events_stream_spark_25151_v1:7:397887 truck_speed_events_stream_spark_25151_v1:8:397813 truck_speed_events_stream_spark_25151_v1:9:0 ``` The experiment only used smallest 4 partitions (0, 1, 4, 6) from these partitions to finish the query earlier. The result of experiment is below: branch | create Kafka consumer | fetch request -- | -- | -- master | 1986 | 2837 patch | 8 | 1706 Closes #22138 from HeartSaVioR/SPARK-25151. Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Co-authored-by: Jungtaek Lim <kabhwan@gmail.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-09-04 13:17:38 -04:00
<tr>
<td>spark.kafka.consumer.fetchedData.cache.timeout</td>
<td>5m (5 minutes)</td>
<td>The minimum amount of time a fetched data may sit idle in the pool before it is eligible for eviction by the evictor.</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>3.0.0</td>
[SPARK-25151][SS] Apply Apache Commons Pool to KafkaDataConsumer ## What changes were proposed in this pull request? This patch does pooling for both kafka consumers as well as fetched data. The overall benefits of the patch are following: * Both pools support eviction on idle objects, which will help closing invalid idle objects which topic or partition are no longer be assigned to any tasks. * It also enables applying different policies on pool, which helps optimization of pooling for each pool. * We concerned about multiple tasks pointing same topic partition as well as same group id, and existing code can't handle this hence excess seek and fetch could happen. This patch properly handles the case. * It also makes the code always safe to leverage cache, hence no need to maintain reuseCache parameter. Moreover, pooling kafka consumers is implemented based on Apache Commons Pool, which also gives couple of benefits: * We can get rid of synchronization of KafkaDataConsumer object while acquiring and returning InternalKafkaConsumer. * We can extract the feature of object pool to outside of the class, so that the behaviors of the pool can be tested easily. * We can get various statistics for the object pool, and also be able to enable JMX for the pool. FetchedData instances are pooled by custom implementation of pool instead of leveraging Apache Commons Pool, because they have CacheKey as first key and "desired offset" as second key which "desired offset" is changing - I haven't found any general pool implementations supporting this. This patch brings additional dependency, Apache Commons Pool 2.6.0 into `spark-sql-kafka-0-10` module. ## How was this patch tested? Existing unit tests as well as new tests for object pool. Also did some experiment regarding proving concurrent access of consumers for same topic partition. * Made change on both sides (master and patch) to log when creating Kafka consumer or fetching records from Kafka is happening. * branches * master: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-master-ref-debugging * patch: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-debugging * Test query (doing self-join) * https://gist.github.com/HeartSaVioR/d831974c3f25c02846f4b15b8d232cc2 * Ran query from spark-shell, with using `local[*]` to maximize the chance to have concurrent access * Collected the count of fetch requests on Kafka via command: `grep "creating new Kafka consumer" logfile | wc -l` * Collected the count of creating Kafka consumers via command: `grep "fetching data from Kafka consumer" logfile | wc -l` Topic and data distribution is follow: ``` truck_speed_events_stream_spark_25151_v1:0:99440 truck_speed_events_stream_spark_25151_v1:1:99489 truck_speed_events_stream_spark_25151_v1:2:397759 truck_speed_events_stream_spark_25151_v1:3:198917 truck_speed_events_stream_spark_25151_v1:4:99484 truck_speed_events_stream_spark_25151_v1:5:497320 truck_speed_events_stream_spark_25151_v1:6:99430 truck_speed_events_stream_spark_25151_v1:7:397887 truck_speed_events_stream_spark_25151_v1:8:397813 truck_speed_events_stream_spark_25151_v1:9:0 ``` The experiment only used smallest 4 partitions (0, 1, 4, 6) from these partitions to finish the query earlier. The result of experiment is below: branch | create Kafka consumer | fetch request -- | -- | -- master | 1986 | 2837 patch | 8 | 1706 Closes #22138 from HeartSaVioR/SPARK-25151. Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Co-authored-by: Jungtaek Lim <kabhwan@gmail.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-09-04 13:17:38 -04:00
</tr>
<tr>
<td>spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval</td>
<td>1m (1 minute)</td>
<td>The interval of time between runs of the idle evictor thread for fetched data pool. When non-positive, no idle evictor thread will be run.</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>3.0.0</td>
[SPARK-25151][SS] Apply Apache Commons Pool to KafkaDataConsumer ## What changes were proposed in this pull request? This patch does pooling for both kafka consumers as well as fetched data. The overall benefits of the patch are following: * Both pools support eviction on idle objects, which will help closing invalid idle objects which topic or partition are no longer be assigned to any tasks. * It also enables applying different policies on pool, which helps optimization of pooling for each pool. * We concerned about multiple tasks pointing same topic partition as well as same group id, and existing code can't handle this hence excess seek and fetch could happen. This patch properly handles the case. * It also makes the code always safe to leverage cache, hence no need to maintain reuseCache parameter. Moreover, pooling kafka consumers is implemented based on Apache Commons Pool, which also gives couple of benefits: * We can get rid of synchronization of KafkaDataConsumer object while acquiring and returning InternalKafkaConsumer. * We can extract the feature of object pool to outside of the class, so that the behaviors of the pool can be tested easily. * We can get various statistics for the object pool, and also be able to enable JMX for the pool. FetchedData instances are pooled by custom implementation of pool instead of leveraging Apache Commons Pool, because they have CacheKey as first key and "desired offset" as second key which "desired offset" is changing - I haven't found any general pool implementations supporting this. This patch brings additional dependency, Apache Commons Pool 2.6.0 into `spark-sql-kafka-0-10` module. ## How was this patch tested? Existing unit tests as well as new tests for object pool. Also did some experiment regarding proving concurrent access of consumers for same topic partition. * Made change on both sides (master and patch) to log when creating Kafka consumer or fetching records from Kafka is happening. * branches * master: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-master-ref-debugging * patch: https://github.com/HeartSaVioR/spark/tree/SPARK-25151-debugging * Test query (doing self-join) * https://gist.github.com/HeartSaVioR/d831974c3f25c02846f4b15b8d232cc2 * Ran query from spark-shell, with using `local[*]` to maximize the chance to have concurrent access * Collected the count of fetch requests on Kafka via command: `grep "creating new Kafka consumer" logfile | wc -l` * Collected the count of creating Kafka consumers via command: `grep "fetching data from Kafka consumer" logfile | wc -l` Topic and data distribution is follow: ``` truck_speed_events_stream_spark_25151_v1:0:99440 truck_speed_events_stream_spark_25151_v1:1:99489 truck_speed_events_stream_spark_25151_v1:2:397759 truck_speed_events_stream_spark_25151_v1:3:198917 truck_speed_events_stream_spark_25151_v1:4:99484 truck_speed_events_stream_spark_25151_v1:5:497320 truck_speed_events_stream_spark_25151_v1:6:99430 truck_speed_events_stream_spark_25151_v1:7:397887 truck_speed_events_stream_spark_25151_v1:8:397813 truck_speed_events_stream_spark_25151_v1:9:0 ``` The experiment only used smallest 4 partitions (0, 1, 4, 6) from these partitions to finish the query earlier. The result of experiment is below: branch | create Kafka consumer | fetch request -- | -- | -- master | 1986 | 2837 patch | 8 | 1706 Closes #22138 from HeartSaVioR/SPARK-25151. Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com> Co-authored-by: Jungtaek Lim <kabhwan@gmail.com> Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-09-04 13:17:38 -04:00
</tr>
</table>
## Writing Data to Kafka
Here, we describe the support for writing Streaming Queries and Batch Queries to Apache Kafka. Take note that
Apache Kafka only supports at least once write semantics. Consequently, when writing---either Streaming Queries
or Batch Queries---to Kafka, some records may be duplicated; this can happen, for example, if Kafka needs
to retry a message that was not acknowledged by a Broker, even though that Broker received and wrote the message record.
Structured Streaming cannot prevent such duplicates from occurring due to these Kafka write semantics. However,
if writing the query is successful, then you can assume that the query output was written at least once. A possible
solution to remove duplicates when reading the written data could be to introduce a primary (unique) key
that can be used to perform de-duplication when reading.
The Dataframe being written to Kafka should have the following columns in schema:
<table class="table">
<tr><th>Column</th><th>Type</th></tr>
<tr>
<td>key (optional)</td>
<td>string or binary</td>
</tr>
<tr>
<td>value (required)</td>
<td>string or binary</td>
</tr>
<tr>
<td>headers (optional)</td>
<td>array</td>
</tr>
<tr>
<td>topic (*optional)</td>
<td>string</td>
</tr>
<tr>
<td>partition (optional)</td>
<td>int</td>
</tr>
</table>
\* The topic column is required if the "topic" configuration option is not specified.<br>
The value column is the only required option. If a key column is not specified then
a ```null``` valued key column will be automatically added (see Kafka semantics on
how ```null``` valued key values are handled). If a topic column exists then its value
is used as the topic when writing the given row to Kafka, unless the "topic" configuration
option is set i.e., the "topic" configuration option overrides the topic column.
If a "partition" column is not specified (or its value is ```null```)
then the partition is calculated by the Kafka producer.
A Kafka partitioner can be specified in Spark by setting the
```kafka.partitioner.class``` option. If not present, Kafka default partitioner
will be used.
The following options must be set for the Kafka sink
for both batch and streaming queries.
<table class="table">
<tr><th>Option</th><th>value</th><th>meaning</th></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">
<tr><th>Option</th><th>value</th><th>default</th><th>query type</th><th>meaning</th></tr>
<tr>
<td>topic</td>
<td>string</td>
<td>none</td>
<td>streaming and batch</td>
<td>Sets the topic that all rows will be written to in Kafka. This option overrides any
topic column that may exist in the data.</td>
</tr>
<tr>
<td>includeHeaders</td>
<td>boolean</td>
<td>false</td>
<td>streaming and batch</td>
<td>Whether to include the Kafka headers in the row.</td>
</tr>
</table>
### Creating a Kafka Sink for Streaming Queries
<div class="codetabs">
<div data-lang="scala" markdown="1">
{% highlight scala %}
// Write key-value data from a DataFrame to a specific Kafka topic specified in an option
val ds = df
.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
.writeStream
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("topic", "topic1")
.start()
// Write key-value data from a DataFrame to Kafka using a topic specified in the data
val ds = df
.selectExpr("topic", "CAST(key AS STRING)", "CAST(value AS STRING)")
.writeStream
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.start()
{% endhighlight %}
</div>
<div data-lang="java" markdown="1">
{% highlight java %}
// Write key-value data from a DataFrame to a specific Kafka topic specified in an option
StreamingQuery ds = df
.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
.writeStream()
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("topic", "topic1")
.start();
// Write key-value data from a DataFrame to Kafka using a topic specified in the data
StreamingQuery ds = df
.selectExpr("topic", "CAST(key AS STRING)", "CAST(value AS STRING)")
.writeStream()
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.start();
{% endhighlight %}
</div>
<div data-lang="python" markdown="1">
{% highlight python %}
# Write key-value data from a DataFrame to a specific Kafka topic specified in an option
ds = df \
.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") \
.writeStream \
.format("kafka") \
.option("kafka.bootstrap.servers", "host1:port1,host2:port2") \
.option("topic", "topic1") \
.start()
# Write key-value data from a DataFrame to Kafka using a topic specified in the data
ds = df \
.selectExpr("topic", "CAST(key AS STRING)", "CAST(value AS STRING)") \
.writeStream \
.format("kafka") \
.option("kafka.bootstrap.servers", "host1:port1,host2:port2") \
.start()
{% endhighlight %}
</div>
</div>
### Writing the output of Batch Queries to Kafka
<div class="codetabs">
<div data-lang="scala" markdown="1">
{% highlight scala %}
// Write key-value data from a DataFrame to a specific Kafka topic specified in an option
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
.write
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("topic", "topic1")
.save()
// Write key-value data from a DataFrame to Kafka using a topic specified in the data
df.selectExpr("topic", "CAST(key AS STRING)", "CAST(value AS STRING)")
.write
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.save()
{% endhighlight %}
</div>
<div data-lang="java" markdown="1">
{% highlight java %}
// Write key-value data from a DataFrame to a specific Kafka topic specified in an option
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
.write()
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.option("topic", "topic1")
.save();
// Write key-value data from a DataFrame to Kafka using a topic specified in the data
df.selectExpr("topic", "CAST(key AS STRING)", "CAST(value AS STRING)")
.write()
.format("kafka")
.option("kafka.bootstrap.servers", "host1:port1,host2:port2")
.save();
{% endhighlight %}
</div>
<div data-lang="python" markdown="1">
{% highlight python %}
# Write key-value data from a DataFrame to a specific Kafka topic specified in an option
df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") \
.write \
.format("kafka") \
.option("kafka.bootstrap.servers", "host1:port1,host2:port2") \
.option("topic", "topic1") \
.save()
# Write key-value data from a DataFrame to Kafka using a topic specified in the data
df.selectExpr("topic", "CAST(key AS STRING)", "CAST(value AS STRING)") \
.write \
.format("kafka") \
.option("kafka.bootstrap.servers", "host1:port1,host2:port2") \
.save()
{% endhighlight %}
</div>
</div>
### Producer Caching
Given Kafka producer instance is designed to be thread-safe, Spark initializes a Kafka producer instance and co-use across tasks for same caching key.
The caching key is built up from the following information:
* Kafka producer configuration
This includes configuration for authorization, which Spark will automatically include when delegation token is being used. Even we take authorization into account, you can expect same Kafka producer instance will be used among same Kafka producer configuration.
It will use different Kafka producer when delegation token is renewed; Kafka producer instance for old delegation token will be evicted according to the cache policy.
The following properties are available to configure the producer pool:
<table class="table">
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
<tr>
<td>spark.kafka.producer.cache.timeout</td>
<td>10m (10 minutes)</td>
<td>The minimum amount of time a producer may sit idle in the pool before it is eligible for eviction by the evictor.</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>2.2.1</td>
</tr>
<tr>
<td>spark.kafka.producer.cache.evictorThreadRunInterval</td>
<td>1m (1 minute)</td>
<td>The interval of time between runs of the idle evictor thread for producer pool. When non-positive, no idle evictor thread will be run.</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>3.0.0</td>
</tr>
</table>
Idle eviction thread periodically removes producers which are not used longer than given timeout. Note that the producer is shared and used concurrently, so the last used timestamp is determined by the moment the producer instance is returned and reference count is 0.
## Kafka Specific Configurations
Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g,
`stream.option("kafka.bootstrap.servers", "host:port")`. For possible kafka parameters, see
[Kafka consumer config docs](http://kafka.apache.org/documentation.html#newconsumerconfigs) for
parameters related to reading data, and [Kafka producer config docs](http://kafka.apache.org/documentation/#producerconfigs)
for parameters related to writing data.
[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.
2016-10-05 19:45:45 -04:00
Note that the following Kafka params cannot be set and the Kafka source or sink will throw an exception:
- **group.id**: Kafka source will create a unique group id for each query automatically. The user can
set the prefix of the automatically generated group.id's via the optional source option `groupIdPrefix`,
default value is "spark-kafka-source". You can also set "kafka.group.id" to force Spark to use a special
group id, however, please read warnings for this option and use it with caution.
- **auto.offset.reset**: Set the source option `startingOffsets` 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 new
topics/partitions are dynamically subscribed. Note that `startingOffsets` only applies when a new
streaming query is started, and that resuming will always pick up from where the query left off. Note
that when the offsets consumed by a streaming application no longer exist in Kafka (e.g., topics are deleted,
offsets are out of range, or offsets are removed after retention period), the offsets will not be reset
and the streaming application will see data loss. In extreme cases, for example the throughput of the
streaming application cannot catch up the retention speed of Kafka, the input rows of a batch might be
gradually reduced until zero when the offset ranges of the batch are completely not in Kafka. Enabling
`failOnDataLoss` option can ask Structured Streaming to fail the query for such cases.
- **key.deserializer**: Keys are always deserialized as byte arrays with ByteArrayDeserializer. Use
[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.
2016-10-05 19:45:45 -04:00
DataFrame operations to explicitly deserialize the keys.
- **value.deserializer**: Values are always deserialized as byte arrays with ByteArrayDeserializer.
[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.
2016-10-05 19:45:45 -04:00
Use DataFrame operations to explicitly deserialize the values.
- **key.serializer**: Keys are always serialized with ByteArraySerializer or StringSerializer. Use
DataFrame operations to explicitly serialize the keys into either strings or byte arrays.
- **value.serializer**: values are always serialized with ByteArraySerializer or StringSerializer. Use
DataFrame operations to explicitly serialize the values into either strings or byte arrays.
[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.
2016-10-05 19:45:45 -04:00
- **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
[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.
2016-10-05 19:45:45 -04:00
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}} ...
For experimenting on `spark-shell`, you can also use `--packages` to add `spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}` and its dependencies directly,
./bin/spark-shell --packages org.apache.spark:spark-sql-kafka-0-10_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ...
[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.
2016-10-05 19:45:45 -04:00
See [Application Submission Guide](submitting-applications.html) for more details about submitting
applications with external dependencies.
## Security
Kafka 0.9.0.0 introduced several features that increases security in a cluster. For detailed
description about these possibilities, see [Kafka security docs](http://kafka.apache.org/documentation.html#security).
It's worth noting that security is optional and turned off by default.
Spark supports the following ways to authenticate against Kafka cluster:
- **Delegation token (introduced in Kafka broker 1.1.0)**
- **JAAS login configuration**
### Delegation token
This way the application can be configured via Spark parameters and may not need JAAS login
configuration (Spark can use Kafka's dynamic JAAS configuration feature). For further information
about delegation tokens, see [Kafka delegation token docs](http://kafka.apache.org/documentation/#security_delegation_token).
The process is initiated by Spark's Kafka delegation token provider. When `spark.kafka.clusters.${cluster}.auth.bootstrap.servers` is set,
Spark considers the following log in options, in order of preference:
- **JAAS login configuration**, please see example below.
- **Keytab file**, such as,
./bin/spark-submit \
--keytab <KEYTAB_FILE> \
--principal <PRINCIPAL> \
--conf spark.kafka.clusters.${cluster}.auth.bootstrap.servers=<KAFKA_SERVERS> \
...
- **Kerberos credential cache**, such as,
./bin/spark-submit \
--conf spark.kafka.clusters.${cluster}.auth.bootstrap.servers=<KAFKA_SERVERS> \
...
The Kafka delegation token provider can be turned off by setting `spark.security.credentials.kafka.enabled` to `false` (default: `true`).
Spark can be configured to use the following authentication protocols to obtain token (it must match with
Kafka broker configuration):
- **SASL SSL (default)**
- **SSL**
- **SASL PLAINTEXT (for testing)**
After obtaining delegation token successfully, Spark distributes it across nodes and renews it accordingly.
Delegation token uses `SCRAM` login module for authentication and because of that the appropriate
`spark.kafka.clusters.${cluster}.sasl.token.mechanism` (default: `SCRAM-SHA-512`) has to be configured. Also, this parameter
must match with Kafka broker configuration.
When delegation token is available on an executor Spark considers the following log in options, in order of preference:
- **JAAS login configuration**, please see example below.
- **Delegation token**, please see <code>spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex</code> parameter for further details.
When none of the above applies then unsecure connection assumed.
#### Configuration
Delegation tokens can be obtained from multiple clusters and <code>${cluster}</code> is an arbitrary unique identifier which helps to group different configurations.
<table class="table">
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<tr><th>Property Name</th><th>Default</th><th>Meaning</th><th>Since Version</th></tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.auth.bootstrap.servers</code></td>
<td>None</td>
<td>
A list of coma separated host/port pairs to use for establishing the initial connection
to the Kafka cluster. For further details please see Kafka documentation. Only used to obtain delegation token.
</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex</code></td>
<td>.*</td>
<td>
Regular expression to match against the <code>bootstrap.servers</code> config for sources and sinks in the application.
If a server address matches this regex, the delegation token obtained from the respective bootstrap servers will be used when connecting.
If multiple clusters match the address, an exception will be thrown and the query won't be started.
Kafka's secure and unsecure listeners are bound to different ports. When both used the secure listener port has to be part of the regular expression.
</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.security.protocol</code></td>
<td>SASL_SSL</td>
<td>
Protocol used to communicate with brokers. For further details please see Kafka documentation. Protocol is applied on all the sources and sinks as default where
<code>bootstrap.servers</code> config matches (for further details please see <code>spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex</code>),
and can be overridden by setting <code>kafka.security.protocol</code> on the source or sink.
</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.sasl.kerberos.service.name</code></td>
<td>kafka</td>
<td>
The Kerberos principal name that Kafka runs as. This can be defined either in Kafka's JAAS config or in Kafka's config.
For further details please see Kafka documentation. Only used to obtain delegation token.
</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.ssl.truststore.type</code></td>
<td>None</td>
<td>
The file format of the trust store file. For further details please see Kafka documentation. Only used to obtain delegation token.
</td>
<td>3.2.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.ssl.truststore.location</code></td>
<td>None</td>
<td>
The location of the trust store file. For further details please see Kafka documentation. Only used to obtain delegation token.
</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.ssl.truststore.password</code></td>
<td>None</td>
<td>
The store password for the trust store file. This is optional and only needed if <code>spark.kafka.clusters.${cluster}.ssl.truststore.location</code> is configured.
For further details please see Kafka documentation. Only used to obtain delegation token.
</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.ssl.keystore.type</code></td>
<td>None</td>
<td>
The file format of the key store file. This is optional for client.
For further details please see Kafka documentation. Only used to obtain delegation token.
</td>
<td>3.2.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.ssl.keystore.location</code></td>
<td>None</td>
<td>
The location of the key store file. This is optional for client and can be used for two-way authentication for client.
For further details please see Kafka documentation. Only used to obtain delegation token.
</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.ssl.keystore.password</code></td>
<td>None</td>
<td>
The store password for the key store file. This is optional and only needed if <code>spark.kafka.clusters.${cluster}.ssl.keystore.location</code> is configured.
For further details please see Kafka documentation. Only used to obtain delegation token.
</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.ssl.key.password</code></td>
<td>None</td>
<td>
The password of the private key in the key store file. This is optional for client.
For further details please see Kafka documentation. Only used to obtain delegation token.
</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>3.0.0</td>
</tr>
<tr>
<td><code>spark.kafka.clusters.${cluster}.sasl.token.mechanism</code></td>
<td>SCRAM-SHA-512</td>
<td>
SASL mechanism used for client connections with delegation token. Because SCRAM login module used for authentication a compatible mechanism has to be set here.
For further details please see Kafka documentation (<code>sasl.mechanism</code>). Only used to authenticate against Kafka broker with delegation token.
</td>
[SPARK-31228][DSTREAMS] Add version information to the configuration of Kafka ### What changes were proposed in this pull request? Add version information to the configuration of Kafka. I sorted out some information show below. Item name | Since version | JIRA ID | Commit ID | Note -- | -- | -- | -- | -- spark.streaming.kafka.consumer.cache.enabled | 2.2.1 | SPARK-19185 | 02cf178bb2a7dc8b4c06eb040c44b6453e41ed15#diff-c465bbcc83b2ecc7530d1c0128e4432b |   spark.streaming.kafka.consumer.poll.ms | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.initialCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.maxCapacity | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.consumer.cache.loadFactor | 2.0.1 | SPARK-12177 | 3134f116a3565c3a299fa2e7094acd7304d64280#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.streaming.kafka.maxRatePerPartition | 1.3.0 | SPARK-4964 | a119cae48030520da9f26ee9a1270bed7f33031e#diff-26cb4369f86050dc2e75cd16291b2844 |   spark.streaming.kafka.minRatePerPartition | 2.4.0 | SPARK-25233 | 135ff16a3510a4dfb3470904004dae9848005019#diff-815f6ec5caf9e4beb355f5f981171f1f |   spark.streaming.kafka.allowNonConsecutiveOffsets | 2.3.1 | SPARK-24067 | 1d598b771de3b588a2f377ae7ccf8193156641f2#diff-4597d93a0e951f7199697dba7dd0dc32 |   spark.kafka.producer.cache.timeout | 2.2.1 | SPARK-19968 | f6730a70cb47ebb3df7f42209df7b076aece1093#diff-ac8844e8d791a75aaee3d0d10bfc1f2a |   spark.kafka.producer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-21869 | 7bff2db9ed803e05a43c2d875c1dea819d81248a#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.capacity | 3.0.0 | SPARK-27687 | efa303581ac61d6f517aacd08883da2d01530bd2#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.jmx.enable | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.timeout | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.consumer.fetchedData.cache.evictorThreadRunInterval | 3.0.0 | SPARK-25151 | 594c9c5a3ece0e913949c7160bb4925e5d289e44#diff-ea8349d528fe8d1b0a8ffa2840ff4bcd |   spark.kafka.clusters.${cluster}.auth.bootstrap.servers | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.security.protocol | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.kerberos.service.name | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.truststore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.location | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.keystore.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.ssl.key.password | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   spark.kafka.clusters.${cluster}.sasl.token.mechanism | 3.0.0 | SPARK-27294 | 2f558094257c38d26650049f2ac93be6d65d6d85#diff-7df71bd47f5a3428ebdb05ced3c31f49 |   ### Why are the changes needed? Supplemental configuration version information. ### Does this PR introduce any user-facing change? 'No'. ### How was this patch tested? Exists UT Closes #27989 from beliefer/add-version-to-kafka-config. Authored-by: beliefer <beliefer@163.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-03-26 07:11:15 -04:00
<td>3.0.0</td>
</tr>
</table>
#### Kafka Specific Configurations
Kafka's own configurations can be set with `kafka.` prefix, e.g, `--conf spark.kafka.clusters.${cluster}.kafka.retries=1`.
For possible Kafka parameters, see [Kafka adminclient config docs](http://kafka.apache.org/documentation.html#adminclientconfigs).
#### Caveats
- Obtaining delegation token for proxy user is not yet supported ([KAFKA-6945](https://issues.apache.org/jira/browse/KAFKA-6945)).
### JAAS login configuration
JAAS login configuration must placed on all nodes where Spark tries to access Kafka cluster.
This provides the possibility to apply any custom authentication logic with a higher cost to maintain.
This can be done several ways. One possibility is to provide additional JVM parameters, such as,
./bin/spark-submit \
--driver-java-options "-Djava.security.auth.login.config=/path/to/custom_jaas.conf" \
--conf spark.executor.extraJavaOptions=-Djava.security.auth.login.config=/path/to/custom_jaas.conf \
...