chia7712 commented on code in PR #16602:
URL: https://github.com/apache/kafka/pull/16602#discussion_r1759315310
##########
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 explanation, but "trigger the delayed operation completion in
the same thread" seems to work by `CompletableFuture.supplyAsync`, right?
```java
CompletableFuture<Either<Exception,
Option<FileRecords.TimestampAndOffset>>> taskFuture =
CompletableFuture.supplyAsync(() -> {
Either<Exception, Option<FileRecords.TimestampAndOffset>> result;
try {
// If it is not found in remote storage, then search in the
local storage starting with local log start offset.
Option<FileRecords.TimestampAndOffset> timestampAndOffsetOpt
=
OptionConverters.toScala(findOffsetByTimestamp(topicPartition, timestamp,
startingOffset, leaderEpochCache))
.orElse(searchLocalLog::get);
result = Right.apply(timestampAndOffsetOpt);
} catch (Exception e) {
// NOTE: All the exceptions from the secondary storage are
catched instead of only the KafkaException.
LOGGER.error("Error occurred while reading the remote log
offset for {}", topicPartition, e);
result = Left.apply(e);
} finally {
TopicPartitionOperationKey key = new
TopicPartitionOperationKey(topicPartition.topic(), topicPartition.partition());
delayedRemoteListOffsetsPurgatory.checkAndComplete(key);
}
return result;
}, remoteStorageReaderThreadPool);
```
I notice there is another similar pattern `DelayedRemoteFetch`, so it is OK
to keep current design for consistency. However, it would be great to let me
known (for my own education) what the side-effect happens if using
`CompletableFuture.supplyAsync` :)
--
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]