Github user koeninger commented on a diff in the pull request:

    https://github.com/apache/spark/pull/15820#discussion_r87129817
  
    --- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
 ---
    @@ -83,6 +86,113 @@ private[kafka010] case class CachedKafkaConsumer 
private(
         record
       }
     
    +  @tailrec
    +  final def getAndIgnoreLostData(
    +      offset: Long,
    +      untilOffset: Long,
    +      pollTimeoutMs: Long): ConsumerRecord[Array[Byte], Array[Byte]] = {
    +    // scalastyle:off
    +    // When `failOnDataLoss` is `false`, we need to handle the following 
cases (note: untilOffset and latestOffset are exclusive):
    +    // 1. Some data are aged out, and `offset < beginningOffset <= 
untilOffset - 1 <= latestOffset - 1`
    +    //      Seek to the beginningOffset and fetch the data.
    +    // 2. Some data are aged out, and `offset <= untilOffset - 1 < 
beginningOffset`.
    +    //      There is nothing to fetch, return null.
    +    // 3. The topic is deleted.
    +    //      There is nothing to fetch, return null.
    +    // 4. The topic is deleted and recreated, and `beginningOffset <= 
offset <= untilOffset - 1 <= latestOffset - 1`.
    +    //      We cannot detect this case. We can still fetch data like 
nothing happens.
    +    // 5. The topic is deleted and recreated, and `beginningOffset <= 
offset < latestOffset - 1 < untilOffset - 1`.
    +    //      Same as 4.
    +    // 6. The topic is deleted and recreated, and `beginningOffset <= 
latestOffset - 1 < offset <= untilOffset - 1`.
    +    //      There is nothing to fetch, return null.
    +    // 7. The topic is deleted and recreated, and `offset < 
beginningOffset <= untilOffset - 1`.
    +    //      Same as 1.
    +    // 8. The topic is deleted and recreated, and `offset <= untilOffset - 
1 < beginningOffset`.
    +    //      There is nothing to fetch, return null.
    +    // scalastyle:on
    +    if (offset >= untilOffset) {
    +      // Case 2 or 8
    +      // We seek to beginningOffset but beginningOffset >= untilOffset
    +      reset()
    +      return null
    +    }
    +
    +    logDebug(s"Get $groupId $topicPartition nextOffset 
$nextOffsetInFetchedData requested $offset")
    +    var outOfOffset = false
    +    if (offset != nextOffsetInFetchedData) {
    +      logInfo(s"Initial fetch for $topicPartition $offset")
    +      seek(offset)
    +      try {
    +        poll(pollTimeoutMs)
    +      } catch {
    +        case e: OffsetOutOfRangeException =>
    +          logWarning(s"Cannot fetch offset $offset, try to recover from 
the beginning offset", e)
    +          outOfOffset = true
    +      }
    +    } else if (!fetchedData.hasNext()) {
    +      // The last pre-fetched data has been drained.
    +      seek(offset)
    --- End diff --
    
    I don't think it's necessary to seek every time the fetched data is empty, 
in normal operation the poll should return the next offset, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to