-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-26267][SS]Retry when detecting incorrect offsets from Kafka #23324
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,6 +21,7 @@ import java.{util => ju} | |
| import java.util.concurrent.{Executors, ThreadFactory} | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
| import scala.collection.mutable.ArrayBuffer | ||
| import scala.concurrent.{ExecutionContext, Future} | ||
| import scala.concurrent.duration.Duration | ||
| import scala.util.control.NonFatal | ||
|
|
@@ -144,16 +145,22 @@ private[kafka010] class KafkaOffsetReader( | |
| s"Specified: ${partitionOffsets.keySet} Assigned: ${partitions.asScala}") | ||
| logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") | ||
|
|
||
| partitionOffsets.foreach { | ||
| case (tp, KafkaOffsetRangeLimit.LATEST) => | ||
| consumer.seekToEnd(ju.Arrays.asList(tp)) | ||
| case (tp, KafkaOffsetRangeLimit.EARLIEST) => | ||
| consumer.seekToBeginning(ju.Arrays.asList(tp)) | ||
| case (tp, off) => consumer.seek(tp, off) | ||
| } | ||
| partitionOffsets.map { | ||
| case (tp, _) => tp -> consumer.position(tp) | ||
| def _fetchOffsets(): PartitionOffsetMap = { | ||
| partitionOffsets.foreach { | ||
| case (tp, KafkaOffsetRangeLimit.LATEST) => | ||
| consumer.seekToEnd(ju.Arrays.asList(tp)) | ||
| case (tp, KafkaOffsetRangeLimit.EARLIEST) => | ||
| consumer.seekToBeginning(ju.Arrays.asList(tp)) | ||
| case (tp, off) => consumer.seek(tp, off) | ||
| } | ||
| partitionOffsets.map { | ||
| case (tp, _) => tp -> consumer.position(tp) | ||
| } | ||
| } | ||
|
|
||
| // Fetch the offsets twice to reduce the chance to hit KAFKA-7703. | ||
|
||
| _fetchOffsets() | ||
| _fetchOffsets() | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -192,19 +199,76 @@ private[kafka010] class KafkaOffsetReader( | |
| /** | ||
| * Fetch the latest offsets for the topic partitions that are indicated | ||
| * in the [[ConsumerStrategy]]. | ||
| * | ||
| * Kafka may return earliest offsets when we are requesting latest offsets (KAFKA-7703). To avoid | ||
|
||
| * hitting this issue, we will use the given `knownOffsets` to audit the latest offsets returned | ||
| * by Kafka, if we find some incorrect offsets (a latest offset is less than an offset in | ||
| * `knownOffsets`), we will retry at most `maxOffsetFetchAttempts` times. If `knownOffsets` is not | ||
| * provided, we simply fetch the latest offsets twice and use the second result which is more | ||
| * likely correct. | ||
| * | ||
| * When a topic is recreated, the latest offsets may be less than offsets in `knownOffsets`. We | ||
| * cannot distinguish this with KAFKA-7703, so we just return whatever we get from Kafka after | ||
| * retrying. | ||
| */ | ||
| def fetchLatestOffsets(): Map[TopicPartition, Long] = runUninterruptibly { | ||
| def fetchLatestOffsets( | ||
| knownOffsets: Option[PartitionOffsetMap]): PartitionOffsetMap = runUninterruptibly { | ||
| withRetriesWithoutInterrupt { | ||
| // Poll to get the latest assigned partitions | ||
| consumer.poll(0) | ||
| val partitions = consumer.assignment() | ||
| consumer.pause(partitions) | ||
| logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the end.") | ||
|
|
||
| consumer.seekToEnd(partitions) | ||
| val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap | ||
| logDebug(s"Got latest offsets for partition : $partitionOffsets") | ||
| partitionOffsets | ||
| if (knownOffsets.isEmpty) { | ||
| // Fetch the latest offsets twice and use the second result which is more likely correct. | ||
| consumer.seekToEnd(partitions) | ||
| partitions.asScala.map(p => p -> consumer.position(p)).toMap | ||
| consumer.seekToEnd(partitions) | ||
| partitions.asScala.map(p => p -> consumer.position(p)).toMap | ||
| } else { | ||
| var partitionOffsets: PartitionOffsetMap = Map.empty | ||
|
|
||
| /** | ||
| * Compare `knownOffsets` and `partitionOffsets`. Returns all partitions that have incorrect | ||
| * latest offset (offset in `knownOffsets` is great than the one in `partitionOffsets`). | ||
| */ | ||
| def findIncorrectOffsets: Seq[(TopicPartition, Long, Long)] = { | ||
| var incorrectOffsets = ArrayBuffer[(TopicPartition, Long, Long)]() | ||
| partitionOffsets.foreach { case (tp, offset) => | ||
| knownOffsets.foreach(_.get(tp).foreach { knownOffset => | ||
| if (knownOffset > offset) { | ||
| val incorrectOffset = (tp, knownOffset, offset) | ||
| incorrectOffsets += incorrectOffset | ||
| } | ||
| }) | ||
| } | ||
| incorrectOffsets | ||
| } | ||
|
|
||
| // Retry to fetch latest offsets when detecting incorrect offsets. We don't use | ||
| // `withRetriesWithoutInterrupt` to retry because: | ||
| // | ||
| // - `withRetriesWithoutInterrupt` will reset the consumer for each attempt but a fresh | ||
| // consumer has a much bigger chance to hit KAFKA-7703. | ||
| // - Avoid calling `consumer.poll(0)` which may cause KAFKA-7703. | ||
| var incorrectOffsets: Seq[(TopicPartition, Long, Long)] = Nil | ||
| var attempt = 0 | ||
| do { | ||
| consumer.seekToEnd(partitions) | ||
| partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap | ||
| attempt += 1 | ||
|
|
||
| incorrectOffsets = findIncorrectOffsets | ||
|
||
| if (incorrectOffsets.nonEmpty && attempt < maxOffsetFetchAttempts) { | ||
| logWarning("Retrying to fetch latest offsets because of incorrect offsets " + | ||
| "(partition, previous offset, fetched offset): " + incorrectOffsets) | ||
HyukjinKwon marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| } | ||
| } while (incorrectOffsets.nonEmpty && attempt < maxOffsetFetchAttempts) | ||
|
|
||
| logDebug(s"Got latest offsets for partition : $partitionOffsets") | ||
| partitionOffsets | ||
| } | ||
| } | ||
| } | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This was missing when we wrote Kafka source v2.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
That's my bad. But no tests caught it :(
Is it possible to write a test for this?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@tdas Very good call. I found the fix here was not correct when writing a unit test.