[ https://issues.apache.org/jira/browse/SPARK-17147?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15728326#comment-15728326 ]
caolan commented on SPARK-17147: -------------------------------- Will createDirectStream also be fixed. I hit the same issue, but I use KafkaUtils.createDirectStream() for compact mode kafka topics. BTW, I hit another issue recently, this is not a delete mode kafka topic, but it had the similar error log. ----------------------------------------- User class threw exception: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 501.0 failed 4 times, most recent failure: Lost task 0.3 in stage 501.0 (TID 505, siwu24yarn21xxxxx.com): java.lang.AssertionError: assertion failed: Failed to get records for spark-executor-xxxxxContext..dXa3AOoH 1029.job.dXa3AOoH.data 0 128825069 after polling for 512 at scala.Predef$.assert(Predef.scala:170) at org.apache.spark.streaming.kafka010.CachedKafkaConsumer.get(CachedKafkaConsumer.scala:74) at org.apache.spark.streaming.kafka010.KafkaRDD$KafkaRDDIterator.next(KafkaRDD.scala:227) at org.apache.spark.streaming.kafka010.KafkaRDD$KafkaRDDIterator.next(KafkaRDD.scala:193) at scala.collection.convert.Wrappers$IteratorWrapper.next(Wrappers.scala:31) > Spark Streaming Kafka 0.10 Consumer Can't Handle Non-consecutive Offsets > (i.e. Log Compaction) > ---------------------------------------------------------------------------------------------- > > Key: SPARK-17147 > URL: https://issues.apache.org/jira/browse/SPARK-17147 > Project: Spark > Issue Type: Bug > Components: DStreams > 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: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org