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

    https://github.com/apache/spark/pull/15820#discussion_r87483402
  
    --- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
 ---
    @@ -83,6 +86,129 @@ private[kafka010] case class CachedKafkaConsumer 
private(
         record
       }
     
    +  /**
    +   * Get the record at the `offset`. If it doesn't exist, try to get the 
earliest record in
    +   * `[offset, untilOffset)`.
    +   */
    +  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
    +    require(offset < untilOffset, s"offset: $offset, untilOffset: 
$untilOffset")
    +    logDebug(s"Get $groupId $topicPartition nextOffset 
$nextOffsetInFetchedData requested $offset")
    +    try {
    +      if (offset != nextOffsetInFetchedData) {
    +        logInfo(s"Initial fetch for $topicPartition $offset")
    +        seek(offset)
    +        poll(pollTimeoutMs)
    +      } else if (!fetchedData.hasNext()) {
    +        // The last pre-fetched data has been drained.
    +        poll(pollTimeoutMs)
    +      }
    +      getRecordFromFetchedData(offset, untilOffset)
    +    } catch {
    +      case e: OffsetOutOfRangeException =>
    +        logWarning(s"Cannot fetch offset $offset, try to recover from the 
beginning offset", e)
    +        advanceToBeginningOffsetAndFetch(offset, untilOffset, 
pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Try to advance to the beginning offset and fetch again. 
`beginningOffset` should be in
    +   * `[offset, untilOffset]`. If not, it will try to fetch `offset` again 
if it's in
    +   * `[beginningOffset, latestOffset)`. Otherwise, it will return null and 
reset the pre-fetched
    +   * data.
    +   */
    +  private def advanceToBeginningOffsetAndFetch(
    +      offset: Long,
    +      untilOffset: Long,
    +      pollTimeoutMs: Long): ConsumerRecord[Array[Byte], Array[Byte]] = {
    +    val beginningOffset = getBeginningOffset()
    +    if (beginningOffset <= offset) {
    +      val latestOffset = getLatestOffset()
    +      if (latestOffset <= offset) {
    +        // beginningOffset <= latestOffset - 1 < offset <= untilOffset - 1
    +        logWarning(s"Offset ${offset} is later than the latest offset 
$latestOffset. " +
    +          s"Skipped [$offset, $untilOffset)")
    +        reset()
    +        null
    +      } else {
    +        // beginningOffset <= offset <= min(latestOffset - 1, untilOffset 
- 1)
    +        getAndIgnoreLostData(offset, untilOffset, pollTimeoutMs)
    --- End diff --
    
    +1
    Especially since with the loss of the use of `@tailrec`, this must now 
prove it will terminate within a limited stack size, and should prove it will 
under most stack size configurations.


---
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 [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to