Github user koeninger commented on a diff in the pull request: https://github.com/apache/spark/pull/15820#discussion_r87438788 --- 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) --- End diff -- I think it's worth the warning explicitly stating that data has been lost
--- 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