Nikita-Shupletsov commented on code in PR #21464:
URL: https://github.com/apache/kafka/pull/21464#discussion_r2806543578
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##########
@@ -552,24 +557,99 @@ private void fetchOffsetsWithRetries(final
OffsetFetchRequestState fetchRequest,
log.warn("A duplicated, inflight, request was identified, but
unable to find it in the " +
"outbound buffer: {}", fetchRequest);
}
- if (error == null) {
- maybeUpdateLastSeenEpochIfNewer(res);
- result.complete(res);
- } else {
- if (error instanceof RetriableException ||
isStaleEpochErrorAndValidEpochAvailable(error)) {
- if (fetchRequest.isExpired()) {
- log.debug("OffsetFetch request for {} timed out and
won't be retried anymore", fetchRequest.requestedPartitions);
-
result.completeExceptionally(maybeWrapAsTimeoutException(error));
- } else {
- fetchRequest.resetFuture();
- fetchOffsetsWithRetries(fetchRequest, result);
- }
- } else
- result.completeExceptionally(error);
+
+ // Success: no group-level error and no partition-level errors
+ if (error == null && !res.hasRetriablePartitionErrors()) {
Review Comment:
nit: it may be just me, but it feels like it may be easier to read if the
order of ifs were:
```
if (error != null)
...
if (res.hasRetriablePartitionErrors())
...
// success
...
```
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##########
@@ -284,8 +284,15 @@ private void process(final FetchCommittedOffsetsEvent
event) {
return;
}
CommitRequestManager manager =
requestManagers.commitRequestManager.get();
- CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> future =
manager.fetchOffsets(event.partitions(), event.deadlineMs());
- future.whenComplete(complete(event.future()));
+ CompletableFuture<CommitRequestManager.OffsetFetchResult> future =
manager.fetchOffsets(event.partitions(), event.deadlineMs());
+ future.whenComplete((result, error) -> {
+ if (error != null) {
+ event.future().completeExceptionally(error);
+ } else {
+ Map<TopicPartition, OffsetAndMetadata> offsetMap =
result.toOffsetMapWithNulls();
Review Comment:
I understand that it's okay to return nulls here, as it's documented
already:
https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L1315,
but is it possible now to receive a null without this change?
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##########
@@ -1194,13 +1336,17 @@ private void onSuccess(final long currentTimeMs,
future.completeExceptionally(new
UnstableOffsetCommitException("There are " +
"unstable offsets for the requested topic partitions"));
} else {
Review Comment:
nit: else if
--
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]