kamalcph commented on code in PR #16602:
URL: https://github.com/apache/kafka/pull/16602#discussion_r1759255906
##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -620,6 +631,23 @@ private Optional<Integer> maybeLeaderEpoch(int
leaderEpoch) {
return leaderEpoch == RecordBatch.NO_PARTITION_LEADER_EPOCH ?
Optional.empty() : Optional.of(leaderEpoch);
}
+ public AsyncOffsetReadFutureHolder<Either<Exception,
Option<FileRecords.TimestampAndOffset>>> asyncOffsetRead(
+ TopicPartition topicPartition,
+ Long timestamp,
+ Long startingOffset,
+ LeaderEpochFileCache leaderEpochCache,
+ Supplier<Option<FileRecords.TimestampAndOffset>> searchLocalLog) {
+ CompletableFuture<Either<Exception,
Option<FileRecords.TimestampAndOffset>>> taskFuture = new CompletableFuture<>();
+ Future<Void> jobFuture = remoteStorageReaderThreadPool.submit(
+ new RemoteLogOffsetReader(this, topicPartition, timestamp,
startingOffset, leaderEpochCache, searchLocalLog, result -> {
+ TopicPartitionOperationKey key = new
TopicPartitionOperationKey(topicPartition.topic(), topicPartition.partition());
+ taskFuture.complete(result);
+ delayedRemoteListOffsetsPurgatory.checkAndComplete(key);
+ })
+ );
+ return new AsyncOffsetReadFutureHolder<>(jobFuture, taskFuture);
Review Comment:
Thanks for the review!
The reason for maintaining 2 futures: jobFuture and taskFuture. They are
required to trigger the delayed operation completion
(delayedRemoteListOffsetsPurgatory#checkAndComplete(key)) in the same
remote-log-reader thread after the `findOffsetByTimestamp` operation completes.
In DelayedRemoteListOffsets purgatory, we return the result when all the
partitions results are received. Then, the delayedOperation gets completed.
We have ActionQueue to complete the pending actions but the LIST_OFFSETS
request can be served by any replica (least loaded node). If the node serving
the request doesn't have leadership for any of the partitions, then the result
might not be complete.
--
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]