Skip to content

Commit

Permalink
Fixed "Single message comes late" (apache#135)
Browse files Browse the repository at this point in the history
  • Loading branch information
mgorbov committed Aug 11, 2017
1 parent 93f0b54 commit b27ea82
Show file tree
Hide file tree
Showing 2 changed files with 9 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,8 @@ class CachedKafkaConsumer[K, V] private(
c
}

val isStreams = topic.startsWith("/") && topic.contains(":")

// TODO if the buffer was kept around as a random-access structure,
// could possibly optimize re-calculating of an RDD in the same batch
protected var buffer = ju.Collections.emptyList[ConsumerRecord[K, V]]().iterator
Expand All @@ -68,13 +70,15 @@ class CachedKafkaConsumer[K, V] private(
poll(timeout)
}

if (!buffer.hasNext()) { poll(timeout) }
assert(buffer.hasNext(),
if (!buffer.hasNext) { poll(timeout) }

assert(buffer.hasNext,
s"Failed to get records for $groupId $topic $partition $offset after polling for $timeout")
val record = buffer.next()

nextOffset = offset + 1
record

if (record.offset() == 0 && isStreams && buffer.hasNext) buffer.next() else record
// Offsets in MapR-streams can contains gaps
/* if (record.offset < offset) {
logInfo(s"Buffer miss for $groupId $topic $partition $offset")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -227,15 +227,13 @@ private[spark] class KafkaRDD[K, V](
// override def hasNext(): Boolean = requestOffset < part.untilOffset
override def getNext(): ConsumerRecord[K, V] = {

val isStreams = consumer.topic.startsWith("/") && consumer.topic.contains(":")

@tailrec
def skipGapsAndGetNext: ConsumerRecord[K, V] = {
if (requestOffset < part.untilOffset) {
val r = consumer.get(requestOffset, pollTimeout)

if (isStreams && r.offset() == 0) {
requestOffset = requestOffset + 1
if (consumer.isStreams && r.offset() == 0) {
requestOffset = part.untilOffset
skipGapsAndGetNext
} else {
requestOffset = r.offset() + 1
Expand Down

0 comments on commit b27ea82

Please sign in to comment.