[
https://issues.apache.org/jira/browse/SPARK-17147?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15429846#comment-15429846
]
Robert Conrad commented on SPARK-17147:
---------------------------------------
Nope, I started the project with kafka 0.10 and spark 2.0.0-preview. I could
take a crack at building a small test project that illuminates the issue, or
maybe it would be easier if I wrote a test in whichever spark suite hits the
streaming connector.
> Spark Streaming Kafka 0.10 Consumer Can't Handle Non-consecutive Offsets
> ------------------------------------------------------------------------
>
> Key: SPARK-17147
> URL: https://issues.apache.org/jira/browse/SPARK-17147
> Project: Spark
> Issue Type: Bug
> Components: Streaming
> Affects Versions: 2.0.0
> Reporter: Robert Conrad
>
> When Kafka does log compaction offsets often end up with gaps, meaning the
> next requested offset will be frequently not be offset+1. The logic in
> KafkaRDD & CachedKafkaConsumer has a baked in assumption that the next offset
> will always be just an increment of 1 above the previous offset.
> I have worked around this problem by changing CachedKafkaConsumer to use the
> returned record's offset, from:
> {{nextOffset = offset + 1}}
> to:
> {{nextOffset = record.offset + 1}}
> and changed KafkaRDD from:
> {{requestOffset += 1}}
> to:
> {{requestOffset = r.offset() + 1}}
> (I also had to change some assert logic in CachedKafkaConsumer).
> There's a strong possibility that I have misconstrued how to use the
> streaming kafka consumer, and I'm happy to close this out if that's the case.
> If, however, it is supposed to support non-consecutive offsets (e.g. due to
> log compaction) I am also happy to contribute a PR.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]