You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Cody Koeninger (JIRA)" <ji...@apache.org> on 2016/10/11 12:22:20 UTC

[jira] [Commented] (SPARK-17853) Kafka OffsetOutOfRangeException on DStreams union from separate Kafka clusters with identical topic names.

    [ https://issues.apache.org/jira/browse/SPARK-17853?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15565278#comment-15565278 ] 

Cody Koeninger commented on SPARK-17853:
----------------------------------------

Which version of DStream are you using, 0-10 or 0-8?
Are you using the same group id for both streams?

> Kafka OffsetOutOfRangeException on DStreams union from separate Kafka clusters with identical topic names.
> ----------------------------------------------------------------------------------------------------------
>
>                 Key: SPARK-17853
>                 URL: https://issues.apache.org/jira/browse/SPARK-17853
>             Project: Spark
>          Issue Type: Bug
>          Components: Streaming
>    Affects Versions: 2.0.0
>            Reporter: Marcin Kuthan
>
> During migration from Spark 1.6 to 2.0 I observed OffsetOutOfRangeException  reported by Kafka client. In our scenario we create single DStream as a union of multiple DStreams. One DStream for one Kafka cluster (multi dc solution). Both Kafka clusters have the same topics and number of partitions.
> After quick investigation, I found that class DirectKafkaInputDStream keeps offset state for topic and partitions, but it is not aware of different Kafka clusters. 
> For every topic, single DStream is created as a union from all configured Kafka clusters.
> {code}
> class KafkaDStreamSource(configs: Iterable[Map[String, String]]) {
> def createSource(ssc: StreamingContext, topic: String): DStream[(String, Array[Byte])] = {
>     val streams = configs.map { config =>
>       val kafkaParams = config
>       val kafkaTopics = Set(topic)
>       KafkaUtils.
>           createDirectStream[String, Array[Byte]](
>         ssc,
>         LocationStrategies.PreferConsistent,
>         ConsumerStrategies.Subscribe[String, Array[Byte]](kafkaTopics, kafkaParams)
>       ).map { record =>
>         (record.key, record.value)
>       }
>     }
>     ssc.union(streams.toSeq)
>   }
> }
> {code}
> At the end, offsets from one Kafka cluster overwrite offsets from second one. Fortunately OffsetOutOfRangeException was thrown because offsets in both Kafka clusters are significantly different.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org