Github user tdas commented on a diff in the pull request:
https://github.com/apache/spark/pull/15820#discussion_r88758444
--- Diff:
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaConsumer.scala
---
@@ -47,40 +51,191 @@ 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.
+ *
+ * When `failOnDataLoss` is `true`, this will either return record at
offset if available, or
+ * throw exception.
+ *
+ * When `failOnDataLoss` is `false`, this will either return record at
offset if available, or
+ * return the next earliest available record less than untilOffset, or
null. It will not throw
+ * any exception.
*/
- 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 must always be less than untilOffset [offset: $offset,
untilOffset: $untilOffset]")
logDebug(s"Get $groupId $topicPartition nextOffset
$nextOffsetInFetchedData requested $offset")
- if (offset != nextOffsetInFetchedData) {
- logInfo(s"Initial fetch for $topicPartition $offset")
- seek(offset)
- poll(pollTimeoutMs)
+ var toFetchOffset = offset
+ while (toFetchOffset != UNKNOWN_OFFSET) {
+ try {
+ val record = fetchData(toFetchOffset, untilOffset, pollTimeoutMs,
failOnDataLoss)
+ if (record == null) {
+ reset()
+ }
+ return record
+ } catch {
+ case e: OffsetOutOfRangeException =>
+ val message =
+ if (failOnDataLoss) {
+ s"Cannot fetch offset $toFetchOffset"
+ } else {
+ s"Cannot fetch offset $toFetchOffset. Some data may be lost.
" +
+ "Recovering from the earliest offset"
+ }
+ reportDataLoss(failOnDataLoss, message, e)
+ toFetchOffset = getNextEarliestOffset(toFetchOffset, untilOffset)
+ }
}
+ reset()
+ null
+ }
- if (!fetchedData.hasNext()) { poll(pollTimeoutMs) }
- assert(fetchedData.hasNext(),
- s"Failed to get records for $groupId $topicPartition $offset " +
- s"after polling for $pollTimeoutMs")
- var record = fetchedData.next()
+ /**
+ * Return the next earliest available offset in [offset, untilOffset).
If all offsets in
+ * [offset, untilOffset) are invalid (e.g., the topic is deleted and
recreated), it will return
+ * `UNKNOWN_OFFSET`.
+ */
+ private def getNextEarliestOffset(offset: Long, untilOffset: Long): Long
= {
+ val (earliestOffset, latestOffset) = getAvailableOffsetRange()
+ if (offset >= latestOffset || earliestOffset >= untilOffset) {
+ // [offset, untilOffset) and [earliestOffset, latestOffset) have no
overlap,
+ // either
+ // --------------------------------------------------------
+ // ^ ^ ^ ^
+ // | | | |
+ // earliestOffset latestOffset offset untilOffset
+ //
+ // or
+ // --------------------------------------------------------
+ // ^ ^ ^ ^
+ // | | | |
+ // offset untilOffset earliestOffset latestOffset
+ val warningMessage =
+ s"""
+ |The current available offset range is [$earliestOffset,
$latestOffset).
+ | Offset ${offset} is out of range, and records in [$offset,
$untilOffset) will be
+ | skipped ${additionalMessage(failOnDataLoss = false)}
+ """.stripMargin
+ logWarning(warningMessage)
+ UNKNOWN_OFFSET
+ } else if (offset >= earliestOffset) {
+ //
-----------------------------------------------------------------------------
+ // ^ ^ ^
^
+ // | | |
|
+ // earliestOffset offset min(untilOffset,latestOffset)
max(untilOffset, latestOffset)
+ //
+ // This will happen when a topic is deleted and recreated, and new
data are pushed very fast,
+ // then we will see `offset` disappears first then appears again.
Although the parameters
+ // are same, the state in Kafka cluster is changed, so the outer
loop won't be endless.
+ logWarning(s"Found a disappeared offset $offset. " +
+ s"Some data may be lost ${additionalMessage(failOnDataLoss =
false)}")
+ offset
+ } else {
+ //
------------------------------------------------------------------------------
+ // ^ ^ ^
^
+ // | | |
|
+ // offset earliestOffset min(untilOffset,latestOffset)
max(untilOffset, latestOffset)
+ val warningMessage =
+ s"""
+ |The current available offset range is [$earliestOffset,
$latestOffset).
+ | Offset ${offset} is out of range, and records in [$offset,
$earliestOffset) will be
+ | skipped ${additionalMessage(failOnDataLoss = false)}
+ """.stripMargin
+ logWarning(warningMessage)
+ earliestOffset
+ }
+ }
- if (record.offset != offset) {
- logInfo(s"Buffer miss for $groupId $topicPartition $offset")
+ /**
+ * Get the earliest record in [offset, untilOffset). If there is not
such record, return null and
+ * clear the fetched data.
+ */
+ private def fetchData(
+ offset: Long,
+ untilOffset: Long,
--- End diff --
untilOffset and failOnDataLoss not really needed.
---
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]