Github user tdas commented on a diff in the pull request:
https://github.com/apache/spark/pull/15820#discussion_r88366202
--- 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)
}
+ }
- 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()
+ /**
+ * Try to advance to the beginning offset and fetch again.
`earliestOffset` should be in
+ * `[offset, untilOffset]`. If not, it will try to fetch `offset` again
if it's in
+ * `[earliestOffset, latestOffset)`. Otherwise, it will return null and
reset the pre-fetched
+ * data.
+ */
+ private def advanceToEarliestOffsetAndFetch(
+ offset: Long,
+ untilOffset: Long,
+ pollTimeoutMs: Long): ConsumerRecord[Array[Byte], Array[Byte]] = {
+ val (earliestOffset, latestOffset) = getCurrentOffsetRange()
+ 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 (GroupId: $groupId, TopicPartition: $topicPartition).
+ | $INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE
+ """.stripMargin
+ logWarning(warningMessage)
+ reset()
+ null
+ } 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 it's not an
endless loop.
+ //
+ // In addition, the stack here won't be deep unless the user keeps
deleting and creating the
+ // topic very fast.
+ //
+ // Therefore, this recursive call is safe.
+ get(offset, untilOffset, pollTimeoutMs, failOnDataLoss = false)
+ } 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 (GroupId: $groupId, TopicPartition: $topicPartition).
+ | $INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE
+ """.stripMargin
+ logWarning(warningMessage)
+ get(earliestOffset, untilOffset, pollTimeoutMs, failOnDataLoss =
false)
+ }
+ }
- if (record.offset != offset) {
- logInfo(s"Buffer miss for $groupId $topicPartition $offset")
- seek(offset)
- poll(pollTimeoutMs)
- assert(fetchedData.hasNext(),
- s"Failed to get records for $groupId $topicPartition $offset " +
- s"after polling for $pollTimeoutMs")
- record = fetchedData.next()
- assert(record.offset == offset,
- s"Got wrong record for $groupId $topicPartition even after seeking
to offset $offset")
+ /**
+ * Get the earliest record in [offset, untilOffset) from the fetched
data. If there is no such
+ * record, return null. Note that this must be called after some data
has already been fetched
+ * using poll.
--- End diff --
... if there is not such record, return null and clear the fetched data.
---
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]