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

    https://github.com/apache/spark/pull/15820#discussion_r88363196
  
    --- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
 ---
    @@ -47,40 +51,229 @@ private[kafka010] case class CachedKafkaConsumer 
private(
     
       /** Iterator to the already fetch data */
       private var fetchedData = 
ju.Collections.emptyIterator[ConsumerRecord[Array[Byte], Array[Byte]]]
    -  private var nextOffsetInFetchedData = -2L
    +  private var nextOffsetInFetchedData = UNKNOWN_OFFSET
     
       /**
        * Get the record for the given offset, waiting up to timeout ms if IO 
is necessary.
        * Sequential forward access will use buffers, but random access will be 
horribly inefficient.
    +   *
    +   * If `failOnDataLoss` is `false`, it will try to get the earliest 
record in
    +   * `[offset, untilOffset)` when some illegal state happens. Otherwise, 
an `IllegalStateException`
    +   * will be thrown.
    +   *
    +   * It returns `null` only when `failOnDataLoss` is `false` and it cannot 
fetch any record between
    +   * [offset, untilOffset).
        */
    -  def get(offset: Long, pollTimeoutMs: Long): ConsumerRecord[Array[Byte], 
Array[Byte]] = {
    +  def get(
    +      offset: Long,
    +      untilOffset: Long,
    +      pollTimeoutMs: Long,
    +      failOnDataLoss: Boolean): ConsumerRecord[Array[Byte], Array[Byte]] = 
{
    +    require(offset < untilOffset, s"offset: $offset, untilOffset: 
$untilOffset")
         logDebug(s"Get $groupId $topicPartition nextOffset 
$nextOffsetInFetchedData requested $offset")
    +    try {
    +      fetchDataIfNeeded(offset, pollTimeoutMs)
    +      getRecordFromFetchedData(offset, untilOffset, failOnDataLoss)
    +    } catch {
    +      case e: OffsetOutOfRangeException =>
    +        val message =
    +          if (failOnDataLoss) {
    +            s"""Cannot fetch offset $offset. (GroupId: $groupId, 
TopicPartition: $topicPartition).
    +               | $INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE""".stripMargin
    +          } else {
    +            s"""Cannot fetch offset $offset. Some data may be lost. 
Recovering from the earliest
    +               | offset (GroupId: $groupId, TopicPartition: 
$topicPartition).""".stripMargin
    +          }
    +        reportDataLoss(failOnDataLoss, message, e)
    +        advanceToEarliestOffsetAndFetch(offset, untilOffset, pollTimeoutMs)
    +    }
    +  }
    +
    +  /**
    +   * Check the pre-fetched data with `offset` and try to fetch from Kafka 
if they don't match.
    +   */
    +  private def fetchDataIfNeeded(offset: Long, pollTimeoutMs: Long): Unit = 
{
         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.
    +      // Seek to the offset because we may call seekToBeginning or 
seekToEnd before this.
    +      seek(offset)
    +      poll(pollTimeoutMs)
    --- End diff --
    
    Why havent these two cases been merged? 


---
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