aboutsummaryrefslogtreecommitdiff
path: root/docs/streaming-kafka-0-10-integration.md
diff options
context:
space:
mode:
authorcody koeninger <cody@koeninger.org>2016-10-12 00:40:47 -0700
committerReynold Xin <rxin@databricks.com>2016-10-12 00:40:47 -0700
commitc264ef9b1918256a5018c7a42a1a2b42308ea3f7 (patch)
treeade9f901b88634e412936c8f2bee030f72067a12 /docs/streaming-kafka-0-10-integration.md
parentb512f04f8e546843d5a3f35dcc6b675b5f4f5bc0 (diff)
downloadspark-c264ef9b1918256a5018c7a42a1a2b42308ea3f7.tar.gz
spark-c264ef9b1918256a5018c7a42a1a2b42308ea3f7.tar.bz2
spark-c264ef9b1918256a5018c7a42a1a2b42308ea3f7.zip
[SPARK-17853][STREAMING][KAFKA][DOC] make it clear that reusing group.id is bad
## What changes were proposed in this pull request? Documentation fix to make it clear that reusing group id for different streams is super duper bad, just like it is with the underlying Kafka consumer. ## How was this patch tested? I built jekyll doc and made sure it looked ok. Author: cody koeninger <cody@koeninger.org> Closes #15442 from koeninger/SPARK-17853.
Diffstat (limited to 'docs/streaming-kafka-0-10-integration.md')
-rw-r--r--docs/streaming-kafka-0-10-integration.md7
1 files changed, 5 insertions, 2 deletions
diff --git a/docs/streaming-kafka-0-10-integration.md b/docs/streaming-kafka-0-10-integration.md
index 44c39e3944..456b845338 100644
--- a/docs/streaming-kafka-0-10-integration.md
+++ b/docs/streaming-kafka-0-10-integration.md
@@ -27,7 +27,7 @@ For Scala/Java applications using SBT/Maven project definitions, link your strea
"bootstrap.servers" -> "localhost:9092,anotherhost:9092",
"key.deserializer" -> classOf[StringDeserializer],
"value.deserializer" -> classOf[StringDeserializer],
- "group.id" -> "example",
+ "group.id" -> "use_a_separate_group_id_for_each_stream",
"auto.offset.reset" -> "latest",
"enable.auto.commit" -> (false: java.lang.Boolean)
)
@@ -48,7 +48,7 @@ Each item in the stream is a [ConsumerRecord](http://kafka.apache.org/0100/javad
</div>
For possible kafkaParams, see [Kafka consumer config docs](http://kafka.apache.org/documentation.html#newconsumerconfigs).
-Note that enable.auto.commit is disabled, for discussion see [Storing Offsets](streaming-kafka-0-10-integration.html#storing-offsets) below.
+Note that the example sets enable.auto.commit to false, for discussion see [Storing Offsets](streaming-kafka-0-10-integration.html#storing-offsets) below.
### LocationStrategies
The new Kafka consumer API will pre-fetch messages into buffers. Therefore it is important for performance reasons that the Spark integration keep cached consumers on executors (rather than recreating them for each batch), and prefer to schedule partitions on the host locations that have the appropriate consumers.
@@ -57,6 +57,9 @@ In most cases, you should use `LocationStrategies.PreferConsistent` as shown abo
The cache for consumers has a default maximum size of 64. If you expect to be handling more than (64 * number of executors) Kafka partitions, you can change this setting via `spark.streaming.kafka.consumer.cache.maxCapacity`
+The cache is keyed by topicpartition and group.id, so use a **separate** `group.id` for each call to `createDirectStream`.
+
+
### ConsumerStrategies
The new Kafka consumer API has a number of different ways to specify topics, some of which require considerable post-object-instantiation setup. `ConsumerStrategies` provides an abstraction that allows Spark to obtain properly configured consumers even after restart from checkpoint.