From 447b2b5309251f3ae37857de73c157e59a0d76df Mon Sep 17 00:00:00 2001 From: Tyson Condie Date: Tue, 14 Feb 2017 18:50:14 -0800 Subject: [PATCH] [SPARK-19584][SS][DOCS] update structured streaming documentation around batch mode ## What changes were proposed in this pull request? Revision to structured-streaming-kafka-integration.md to reflect new Batch query specification and options. zsxwing tdas Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Tyson Condie Closes #16918 from tcondie/kafka-docs. --- .../structured-streaming-kafka-integration.md | 160 ++++++++++++++++-- 1 file changed, 149 insertions(+), 11 deletions(-) diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 8b2f51a378dc7..522e669568678 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -119,6 +119,124 @@ ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") +### Creating a Kafka Source Batch +If you have a use case that is better suited to batch processing, +you can create an Dataset/DataFrame for a defined range of offsets. + +
+
+{% highlight scala %} + +// Subscribe to 1 topic defaults to the earliest and latest offsets +val ds1 = spark + .read + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .load() +ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + +// Subscribe to multiple topics, specifying explicit Kafka offsets +val ds2 = 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() +ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + +// Subscribe to a pattern, at the earliest and latest offsets +val ds3 = spark + .read + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribePattern", "topic.*") + .option("startingOffsets", "earliest") + .option("endingOffsets", "latest") + .load() +ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + +{% endhighlight %} +
+
+{% highlight java %} + +// Subscribe to 1 topic defaults to the earliest and latest offsets +Dataset ds1 = spark + .read() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .load(); +ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)"); + +// Subscribe to multiple topics, specifying explicit Kafka offsets +Dataset ds2 = 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(); +ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)"); + +// Subscribe to a pattern, at the earliest and latest offsets +Dataset ds3 = spark + .read() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribePattern", "topic.*") + .option("startingOffsets", "earliest") + .option("endingOffsets", "latest") + .load(); +ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)"); + +{% endhighlight %} +
+
+{% highlight python %} + +# Subscribe to 1 topic defaults to the earliest and latest offsets +ds1 = spark \ + .read \ + .format("kafka") \ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ + .option("subscribe", "topic1") \ + .load() +ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + +# Subscribe to multiple topics, specifying explicit Kafka offsets +ds2 = 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() +ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + +# Subscribe to a pattern, at the earliest and latest offsets +ds3 = spark \ + .read \ + .format("kafka") \ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ + .option("subscribePattern", "topic.*") \ + .option("startingOffsets", "earliest") \ + .option("endingOffsets", "latest") \ + .load() +ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + +{% endhighlight %} +
+
+ Each row in the source has the following schema: @@ -152,7 +270,8 @@ Each row in the source has the following schema:
ColumnType
-The following options must be set for the Kafka source. +The following options must be set for the Kafka source +for both batch and streaming queries. @@ -187,50 +306,69 @@ The following options must be set for the Kafka source. The following configurations are optional:
Optionvaluemeaning
- + - - + + + + + + + + + - + + as you expected. Batch queries will always fail if it fails to read any data from the provided + offsets due to lost data. + - + + + +
Optionvaluedefaultmeaning
Optionvaluedefaultquery typemeaning
startingOffsetsearliest, latest, or json string - {"topicA":{"0":23,"1":-1},"topicB":{"0":-2}} + "earliest", "latest" (streaming only), or json string + """ {"topicA":{"0":23,"1":-1},"topicB":{"0":-2}} """ latest"latest" for streaming, "earliest" for batchstreaming and batch 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: This only applies when a new Streaming 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 + 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.
endingOffsetslatest or json string + {"topicA":{"0":23,"1":-1},"topicB":{"0":-1}} + latestbatch queryThe 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.
failOnDataLoss true or false trueWhether to fail the query when it's possible that data is lost (e.g., topics are deleted, or + streaming queryWhether 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.
kafkaConsumer.pollTimeoutMs long 512streaming and batch The timeout in milliseconds to poll data from Kafka in executors.
fetchOffset.numRetries int 3Number of times to retry before giving up fatch Kafka latest offsets.streaming and batchNumber of times to retry before giving up fetching Kafka offsets.
fetchOffset.retryIntervalMs long 10streaming and batch milliseconds to wait before retrying to fetch Kafka offsets
maxOffsetsPerTrigger long nonestreaming and batch 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.
@@ -246,7 +384,7 @@ Note that the following Kafka params cannot be set and the Kafka source will thr 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. + streaming query is started, and that resuming will always pick up from where the query left off. - **key.deserializer**: Keys are always deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations to explicitly deserialize the keys. - **value.deserializer**: Values are always deserialized as byte arrays with ByteArrayDeserializer.