FrankChen021 commented on issue #14344:
URL: https://github.com/apache/druid/issues/14344#issuecomment-2969460351
look at the exception handler of OffsetOutOfRangeException, I can't remember
why it waits to retry if auto-reset is not enabled.
under such case, the retry seems useless, there's no way to recover from it
automatically.
```java
private void possiblyResetOffsetsOrWait(
Map<TopicPartition, Long> outOfRangePartitions,
RecordSupplier<KafkaTopicPartition, Long, KafkaRecordEntity>
recordSupplier,
TaskToolbox taskToolbox
) throws InterruptedException, IOException
{
final String stream =
task.getIOConfig().getStartSequenceNumbers().getStream();
final boolean isMultiTopic = task.getIOConfig().isMultiTopic();
final Map<TopicPartition, Long> resetPartitions = new HashMap<>();
boolean doReset = false;
if (task.getTuningConfig().isResetOffsetAutomatically()) {
for (Map.Entry<TopicPartition, Long> outOfRangePartition :
outOfRangePartitions.entrySet()) {
final TopicPartition topicPartition = outOfRangePartition.getKey();
final long nextOffset = outOfRangePartition.getValue();
// seek to the beginning to get the least available offset
StreamPartition<KafkaTopicPartition> streamPartition =
StreamPartition.of(
stream,
new KafkaTopicPartition(isMultiTopic, topicPartition.topic(),
topicPartition.partition())
);
final Long leastAvailableOffset =
recordSupplier.getEarliestSequenceNumber(streamPartition);
if (leastAvailableOffset == null) {
throw new ISE(
"got null sequence number for partition[%s] when fetching from
kafka!",
topicPartition.partition()
);
}
// reset the seek
recordSupplier.seek(streamPartition, nextOffset);
// Reset consumer offset if resetOffsetAutomatically is set to true
// and the current message offset in the kafka partition is more
than the
// next message offset that we are trying to fetch
if (leastAvailableOffset > nextOffset) {
doReset = true;
resetPartitions.put(topicPartition, nextOffset);
}
}
}
if (doReset) {
sendResetRequestAndWait(CollectionUtils.mapKeys(resetPartitions,
topicPartition -> StreamPartition.of(
stream,
new KafkaTopicPartition(isMultiTopic, topicPartition.topic(),
topicPartition.partition())
)), taskToolbox);
} else {
log.warn("Retrying in %dms", task.getPollRetryMs());
pollRetryLock.lockInterruptibly();
try {
long nanos = TimeUnit.MILLISECONDS.toNanos(task.getPollRetryMs());
while (nanos > 0L && !pauseRequested && !stopRequested.get()) {
nanos = isAwaitingRetry.awaitNanos(nanos);
}
}
finally {
pollRetryLock.unlock();
}
}
}
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]