dengziming commented on code in PR #13432:
URL: https://github.com/apache/kafka/pull/13432#discussion_r1167684205


##########
clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java:
##########
@@ -4579,7 +4374,7 @@ void maybeRetry(long currentTimeMs, Throwable throwable) {
         };
     }
 
-    private long getOffsetFromOffsetSpec(OffsetSpec offsetSpec) {
+    private static long getOffsetFromSpec(OffsetSpec offsetSpec) {

Review Comment:
   nit: this change seems unnecessary.



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java:
##########
@@ -260,12 +261,29 @@ public void onFailure(
                 .filter(future.lookupKeys()::contains)
                 .collect(Collectors.toSet());
             retryLookup(keysToUnmap);
+        } else if (t instanceof UnsupportedVersionException) {

Review Comment:
   We should add 2 test cases in AdminApiDriverTest to test this:
   1. testFulfillmentUnsupportedVersion
   2. testFulfillmentRetribleUnsupportedVersion
   
   The first is to verify unsupportedVersionException will return in failed, 
the code like this:
   ```
   ctx.poll(emptyMap(), map(mkSet("foo"), failed("foo", new 
UnsupportedVersionException("unsupported api"))));
   ctx.poll(emptyMap(), emptyMap());
   ```
   The second is more complex, you should implement `handleUnsupportedVersion` 
in `MockAdminApiHandler` to support retry on unsupported, the code may like 
this:
   ```
       ctx.handler.addRetriableUnsupportedVersionKey("foo");
       ctx.handler.expectRequest(mkSet("foo"), failed("foo", new 
UnsupportedVersionException("unsupported api")));
       ctx.handler.expectRequest(mkSet("bar"), failed("foo", new 
UnsupportedVersionException("unsupported api")));
       
       // verify retry on foo and not retry on bar.
       List<RequestSpec<String>> requestSpecs = ctx.driver.poll();
   
       requestSpecs.forEach(requestSpec -> {
               ctx.driver.onFailure(ctx.time.milliseconds(), requestSpec, new 
UnsupportedVersionException("unsupported api"));
           });
           ctx.poll(emptyMap(), map(
               mkSet("foo"), failed("foo", new 
UnsupportedVersionException("unsupported api"))
           ));
   ```



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java:
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin.internals;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.kafka.clients.admin.ListOffsetsOptions;
+import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo;
+import org.apache.kafka.clients.admin.OffsetSpec;
+import org.apache.kafka.clients.admin.OffsetSpec.TimestampSpec;
+import 
org.apache.kafka.clients.admin.internals.AdminApiFuture.SimpleAdminApiFuture;
+import org.apache.kafka.clients.admin.internals.AdminApiHandler.Batched;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.InvalidMetadataException;
+import org.apache.kafka.common.errors.RetriableException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import 
org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.requests.ListOffsetsRequest;
+import org.apache.kafka.common.requests.ListOffsetsResponse;
+import org.apache.kafka.common.utils.CollectionUtils;
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+public final class ListOffsetsHandler extends Batched<TopicPartition, 
ListOffsetsResultInfo> {
+
+    private final Map<TopicPartition, OffsetSpec> offsetSpecsByPartition;
+    private final ListOffsetsOptions options;
+    private final Logger log;
+    private final AdminApiLookupStrategy<TopicPartition> lookupStrategy;
+
+    public ListOffsetsHandler(
+        Map<TopicPartition, OffsetSpec> offsetSpecsByPartition,
+        ListOffsetsOptions options,
+        LogContext logContext
+    ) {
+        this.offsetSpecsByPartition = offsetSpecsByPartition;
+        this.options = options;
+        this.log = logContext.logger(ListOffsetsHandler.class);
+        this.lookupStrategy = new PartitionLeaderStrategy(logContext);
+    }
+
+    @Override
+    public String apiName() {
+        return "listOffsets";
+    }
+
+    @Override
+    public AdminApiLookupStrategy<TopicPartition> lookupStrategy() {
+        return this.lookupStrategy;
+    }
+
+    @Override
+    ListOffsetsRequest.Builder buildBatchedRequest(int brokerId, 
Set<TopicPartition> keys) {
+        Map<String, ListOffsetsTopic> topicsByName = 
CollectionUtils.groupPartitionsByTopic(
+            keys,
+            topicName -> new ListOffsetsTopic().setName(topicName),
+            (listOffsetsTopic, partitionId) -> {
+                TopicPartition topicPartition = new 
TopicPartition(listOffsetsTopic.name(), partitionId);
+                OffsetSpec offsetSpec = 
offsetSpecsByPartition.get(topicPartition);
+                long offsetQuery = getOffsetFromSpec(offsetSpec);
+                listOffsetsTopic.partitions().add(
+                    new ListOffsetsPartition()
+                        .setPartitionIndex(partitionId)
+                        .setTimestamp(offsetQuery));
+            });
+        boolean supportsMaxTimestamp = keys
+            .stream()
+            .anyMatch(key -> 
getOffsetFromSpec(offsetSpecsByPartition.get(key)) == 
ListOffsetsRequest.MAX_TIMESTAMP);
+
+        return ListOffsetsRequest.Builder
+            .forConsumer(true, options.isolationLevel(), supportsMaxTimestamp)
+            .setTargetTimes(new ArrayList<>(topicsByName.values()));
+    }
+
+    @Override
+    public ApiResult<TopicPartition, ListOffsetsResultInfo> handleResponse(
+        Node broker,
+        Set<TopicPartition> keys,
+        AbstractResponse abstractResponse
+    ) {
+        ListOffsetsResponse response = (ListOffsetsResponse) abstractResponse;
+        Map<TopicPartition, ListOffsetsResultInfo> completed = new HashMap<>();
+        Map<TopicPartition, Throwable> failed = new HashMap<>();
+        List<TopicPartition> unmapped = new ArrayList<>();
+        Set<TopicPartition> retriable = new HashSet<>();
+
+        for (ListOffsetsTopicResponse topic : response.topics()) {
+            for (ListOffsetsPartitionResponse partition : topic.partitions()) {
+                TopicPartition topicPartition = new 
TopicPartition(topic.name(), partition.partitionIndex());
+                Errors error = Errors.forCode(partition.errorCode());
+                OffsetSpec offsetRequestSpec = 
offsetSpecsByPartition.get(topicPartition);
+                if (offsetRequestSpec == null) {
+                    log.warn("ListOffsets response includes unknown topic 
partition {}", topicPartition);
+                } else if (error == Errors.NONE) {
+                    Optional<Integer> leaderEpoch = (partition.leaderEpoch() 
== ListOffsetsResponse.UNKNOWN_EPOCH)
+                        ? Optional.empty()
+                        : Optional.of(partition.leaderEpoch());
+                    completed.put(
+                        topicPartition,
+                        new ListOffsetsResultInfo(partition.offset(), 
partition.timestamp(), leaderEpoch));
+                } else {
+                    handlePartitionError(topicPartition, error, failed, 
unmapped, retriable);
+                }
+            }
+        }
+
+        // Sanity-check if the current leader for these partitions returned 
results for all of them
+        for (TopicPartition topicPartition : keys) {
+            if (unmapped.isEmpty()
+                && !completed.containsKey(topicPartition)
+                && !failed.containsKey(topicPartition)
+                && !retriable.contains(topicPartition)
+            ) {
+                ApiException sanityCheckException = new ApiException(
+                    "The response from broker " + broker.id() +
+                        " did not contain a result for topic partition " + 
topicPartition);
+                log.error(
+                    "ListOffsets request for topic partition {} failed sanity 
check",
+                    topicPartition,
+                    sanityCheckException);
+                failed.put(topicPartition, sanityCheckException);
+            }
+        }
+
+        return new ApiResult<>(completed, failed, unmapped);
+    }
+
+    private void handlePartitionError(
+        TopicPartition topicPartition,
+        Errors error,
+        Map<TopicPartition, Throwable> failed,
+        List<TopicPartition> unmapped,
+        Set<TopicPartition> retriable
+    ) {
+        ApiException apiException = error.exception();
+        if (apiException instanceof InvalidMetadataException) {
+            log.debug(
+                "ListOffsets lookup request for topic partition {} will be 
retried due to invalid metadata",
+                topicPartition,
+                apiException);
+            unmapped.add(topicPartition);
+        } else if (apiException instanceof RetriableException) {
+            log.debug(
+                "ListOffsets fulfillment request for topic partition {} will 
be retried due to {}",
+                topicPartition,
+                apiException);
+            retriable.add(topicPartition);
+        } else {
+            log.error(
+                "ListOffsets request for topic partition {} failed due to an 
unexpected error",
+                topicPartition,
+                apiException);
+            failed.put(topicPartition, apiException);
+        }
+    }
+
+    @Override
+    public Map<TopicPartition, Throwable> handleUnsupportedVersionException(
+        UnsupportedVersionException exception, Set<TopicPartition> keys, 
boolean isFulfillmentStage
+    ) {
+        // An UnsupportedVersionException can be addressed only in the 
fulfillment stage...
+        if (!isFulfillmentStage) {
+            return keys.stream().collect(Collectors.toMap(k -> k, k -> 
exception));
+        }
+        Map<TopicPartition, Throwable> maxTimestampPartitions = new 
HashMap<>();
+        boolean containsNonMaxTimestampSpec = false;
+        for (TopicPartition topicPartition : keys) {
+            OffsetSpec offsetSpec = offsetSpecsByPartition.get(topicPartition);
+            if (getOffsetFromSpec(offsetSpec) == 
ListOffsetsRequest.MAX_TIMESTAMP) {
+                maxTimestampPartitions.put(topicPartition, exception);
+            } else {
+                containsNonMaxTimestampSpec = true;
+            }
+        }
+        // ...only if it's caused by a MAX_TIMESTAMP spec for some partition, 
and only if there is at least
+        // one other partition with a non-MAX_TIMESTAMP spec for which the 
fulfillment stage can be retried.
+        if (maxTimestampPartitions.isEmpty() || !containsNonMaxTimestampSpec) {
+            return keys.stream().collect(Collectors.toMap(k -> k, k -> 
exception));

Review Comment:
   Aha, I was mislead by the complex logic here since it's not straightforward. 
we can replace it with `boolean containsNonMaxTimestampSpec = 
maxTimestampPartitions.size() != keys.size()` or other simpler expression, so 
`if (maxTimestampPartitions.isEmpty() || !containsNonMaxTimestampSpec)` can be 
changed to `if (maxTimestampPartitions.isEmpty() || 
maxTimestampPartitions.size()==keys.size())`.
   And I think we can remove `containsNonMaxTimestampSpec`, if 
`containsNonMaxTimestampSpec` is false, then the 
`keys.stream().collect(Collectors.toMap(k -> k, k -> exception))` and 
`maxTimestampPartitions` are the same. 



##########
clients/src/test/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandlerTest.java:
##########
@@ -0,0 +1,308 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin.internals;
+
+import static java.util.Collections.emptyList;
+import static java.util.Collections.emptyMap;
+import static java.util.Collections.emptySet;
+import static java.util.Collections.singleton;
+import static org.apache.kafka.common.utils.Utils.mkSet;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.kafka.clients.admin.ListOffsetsOptions;
+import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo;
+import org.apache.kafka.clients.admin.OffsetSpec;
+import org.apache.kafka.clients.admin.internals.AdminApiHandler.ApiResult;
+import org.apache.kafka.common.IsolationLevel;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import 
org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
+import org.apache.kafka.common.message.ListOffsetsResponseData;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ListOffsetsRequest;
+import org.apache.kafka.common.requests.ListOffsetsResponse;
+import org.apache.kafka.common.utils.LogContext;
+import org.junit.jupiter.api.Test;
+
+public final class ListOffsetsHandlerTest {
+
+    private final LogContext logContext = new LogContext();
+
+    private final TopicPartition t0p0 = new TopicPartition("t0", 0);
+    private final TopicPartition t0p1 = new TopicPartition("t0", 1);
+    private final TopicPartition t1p0 = new TopicPartition("t1", 0);
+    private final TopicPartition t1p1 = new TopicPartition("t1", 1);
+
+    private final Node node = new Node(1, "host", 1234);
+
+    private final Map<TopicPartition, Long> offsetTimestampsByPartition = new 
HashMap<TopicPartition, Long>() {
+        {
+            put(t0p0, ListOffsetsRequest.LATEST_TIMESTAMP);
+            put(t0p1, ListOffsetsRequest.EARLIEST_TIMESTAMP);
+            put(t1p0, 123L);
+            put(t1p1, ListOffsetsRequest.MAX_TIMESTAMP);
+        }
+    };
+
+    @Test
+    public void testBuildRequestSimple() {
+        ListOffsetsHandler handler =
+            new ListOffsetsHandler(offsetTimestampsByPartition, new 
ListOffsetsOptions(), logContext);
+        ListOffsetsRequest request = handler.buildBatchedRequest(node.id(), 
mkSet(t0p0, t0p1)).build();
+        List<ListOffsetsTopic> topics = request.topics();
+        assertEquals(1, topics.size());
+        ListOffsetsTopic topic = topics.get(0);
+        assertEquals(2, topic.partitions().size());
+        for (ListOffsetsPartition partition : topic.partitions()) {
+            TopicPartition topicPartition = new TopicPartition(topic.name(), 
partition.partitionIndex());
+            assertExpectedTimestamp(topicPartition, partition.timestamp());
+        }
+        assertEquals(IsolationLevel.READ_UNCOMMITTED, 
request.isolationLevel());
+    }
+
+    @Test
+    public void testBuildRequestMultipleTopicsWithReadCommitted() {
+        ListOffsetsHandler handler =
+            new ListOffsetsHandler(
+                offsetTimestampsByPartition, new 
ListOffsetsOptions(IsolationLevel.READ_COMMITTED), logContext);
+        ListOffsetsRequest request =
+            handler.buildBatchedRequest(node.id(), 
offsetTimestampsByPartition.keySet()).build();
+        List<ListOffsetsTopic> topics = request.topics();
+        assertEquals(2, topics.size());
+        Map<TopicPartition, ListOffsetsPartition> partitions = new HashMap<>();
+        for (ListOffsetsTopic topic : topics) {
+            for (ListOffsetsPartition partition : topic.partitions()) {
+                partitions.put(new TopicPartition(topic.name(), 
partition.partitionIndex()), partition);
+            }
+        }
+        assertEquals(4, partitions.size());
+        for (Map.Entry<TopicPartition, ListOffsetsPartition> entry : 
partitions.entrySet()) {
+            assertExpectedTimestamp(entry.getKey(), 
entry.getValue().timestamp());
+        }
+        assertEquals(IsolationLevel.READ_COMMITTED, request.isolationLevel());
+    }
+
+    @Test
+    public void testBuildRequestAllowedVersions() {
+        ListOffsetsHandler defaultOptionsHandler =
+            new ListOffsetsHandler(offsetTimestampsByPartition, new 
ListOffsetsOptions(), logContext);
+        ListOffsetsRequest.Builder builder =
+            defaultOptionsHandler.buildBatchedRequest(node.id(), mkSet(t0p0, 
t0p1, t1p0));
+        assertEquals(1, builder.oldestAllowedVersion());
+
+        ListOffsetsHandler readCommittedHandler =
+            new ListOffsetsHandler(
+                offsetTimestampsByPartition, new 
ListOffsetsOptions(IsolationLevel.READ_COMMITTED), logContext);
+        builder = readCommittedHandler.buildBatchedRequest(node.id(), 
mkSet(t0p0, t0p1, t1p0));
+        assertEquals(2, builder.oldestAllowedVersion());
+
+        builder = readCommittedHandler.buildBatchedRequest(node.id(), 
mkSet(t0p0, t0p1, t1p0, t1p1));
+        assertEquals(7, builder.oldestAllowedVersion());
+    }
+
+    @Test
+    public void testHandleSuccessfulResponse() {
+        ApiResult<TopicPartition, ListOffsetsResultInfo> result =
+            handleResponse(createResponse(emptyMap()));
+
+        assertResult(result, offsetTimestampsByPartition.keySet(), emptyMap(), 
emptyList(), emptySet());
+    }
+
+    @Test
+    public void testHandlePartitionTimeoutResponse() {
+        TopicPartition errorPartition = t0p0;
+        Map<TopicPartition, Short> errorsByPartition = new HashMap<>();
+        errorsByPartition.put(errorPartition, Errors.REQUEST_TIMED_OUT.code());
+
+        ApiResult<TopicPartition, ListOffsetsResultInfo> result =
+            handleResponse(createResponse(errorsByPartition));
+
+        // Timeouts should be retried within the fulfillment stage as they are 
a common type of
+        // retriable error.
+        Set<TopicPartition> retriable = singleton(errorPartition);
+        Set<TopicPartition> completed = new 
HashSet<>(offsetTimestampsByPartition.keySet());
+        completed.removeAll(retriable);
+        assertResult(result, completed, emptyMap(), emptyList(), retriable);
+    }
+
+    @Test
+    public void testHandlePartitionInvalidMetadataResponse() {
+        TopicPartition errorPartition = t0p0;
+        Errors error = Errors.NOT_LEADER_OR_FOLLOWER;
+        Map<TopicPartition, Short> errorsByPartition = new HashMap<>();
+        errorsByPartition.put(errorPartition, error.code());
+
+        ApiResult<TopicPartition, ListOffsetsResultInfo> result =
+            handleResponse(createResponse(errorsByPartition));
+
+        // Invalid metadata errors should be retried from the lookup stage as 
the partition-to-leader
+        // mappings should be recalculated.
+        List<TopicPartition> unmapped = new ArrayList<>();
+        unmapped.add(errorPartition);
+        Set<TopicPartition> completed = new 
HashSet<>(offsetTimestampsByPartition.keySet());
+        completed.removeAll(unmapped);
+        assertResult(result, completed, emptyMap(), unmapped, emptySet());
+    }
+
+    @Test
+    public void testHandlePartitionErrorResponse() {

Review Comment:
   Ditto, We should change method name to testHandleUnexpectedXXX to be more 
clear.



##########
clients/src/test/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandlerTest.java:
##########
@@ -0,0 +1,308 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin.internals;
+
+import static java.util.Collections.emptyList;
+import static java.util.Collections.emptyMap;
+import static java.util.Collections.emptySet;
+import static java.util.Collections.singleton;
+import static org.apache.kafka.common.utils.Utils.mkSet;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.kafka.clients.admin.ListOffsetsOptions;
+import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo;
+import org.apache.kafka.clients.admin.OffsetSpec;
+import org.apache.kafka.clients.admin.internals.AdminApiHandler.ApiResult;
+import org.apache.kafka.common.IsolationLevel;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import 
org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
+import org.apache.kafka.common.message.ListOffsetsResponseData;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ListOffsetsRequest;
+import org.apache.kafka.common.requests.ListOffsetsResponse;
+import org.apache.kafka.common.utils.LogContext;
+import org.junit.jupiter.api.Test;
+
+public final class ListOffsetsHandlerTest {
+
+    private final LogContext logContext = new LogContext();
+
+    private final TopicPartition t0p0 = new TopicPartition("t0", 0);
+    private final TopicPartition t0p1 = new TopicPartition("t0", 1);
+    private final TopicPartition t1p0 = new TopicPartition("t1", 0);
+    private final TopicPartition t1p1 = new TopicPartition("t1", 1);
+
+    private final Node node = new Node(1, "host", 1234);
+
+    private final Map<TopicPartition, Long> offsetTimestampsByPartition = new 
HashMap<TopicPartition, Long>() {
+        {
+            put(t0p0, ListOffsetsRequest.LATEST_TIMESTAMP);
+            put(t0p1, ListOffsetsRequest.EARLIEST_TIMESTAMP);
+            put(t1p0, 123L);
+            put(t1p1, ListOffsetsRequest.MAX_TIMESTAMP);
+        }
+    };
+
+    @Test
+    public void testBuildRequestSimple() {
+        ListOffsetsHandler handler =
+            new ListOffsetsHandler(offsetTimestampsByPartition, new 
ListOffsetsOptions(), logContext);
+        ListOffsetsRequest request = handler.buildBatchedRequest(node.id(), 
mkSet(t0p0, t0p1)).build();
+        List<ListOffsetsTopic> topics = request.topics();
+        assertEquals(1, topics.size());
+        ListOffsetsTopic topic = topics.get(0);
+        assertEquals(2, topic.partitions().size());
+        for (ListOffsetsPartition partition : topic.partitions()) {
+            TopicPartition topicPartition = new TopicPartition(topic.name(), 
partition.partitionIndex());
+            assertExpectedTimestamp(topicPartition, partition.timestamp());
+        }
+        assertEquals(IsolationLevel.READ_UNCOMMITTED, 
request.isolationLevel());
+    }
+
+    @Test
+    public void testBuildRequestMultipleTopicsWithReadCommitted() {
+        ListOffsetsHandler handler =
+            new ListOffsetsHandler(
+                offsetTimestampsByPartition, new 
ListOffsetsOptions(IsolationLevel.READ_COMMITTED), logContext);
+        ListOffsetsRequest request =
+            handler.buildBatchedRequest(node.id(), 
offsetTimestampsByPartition.keySet()).build();
+        List<ListOffsetsTopic> topics = request.topics();
+        assertEquals(2, topics.size());
+        Map<TopicPartition, ListOffsetsPartition> partitions = new HashMap<>();
+        for (ListOffsetsTopic topic : topics) {
+            for (ListOffsetsPartition partition : topic.partitions()) {
+                partitions.put(new TopicPartition(topic.name(), 
partition.partitionIndex()), partition);
+            }
+        }
+        assertEquals(4, partitions.size());
+        for (Map.Entry<TopicPartition, ListOffsetsPartition> entry : 
partitions.entrySet()) {
+            assertExpectedTimestamp(entry.getKey(), 
entry.getValue().timestamp());
+        }
+        assertEquals(IsolationLevel.READ_COMMITTED, request.isolationLevel());
+    }
+
+    @Test
+    public void testBuildRequestAllowedVersions() {
+        ListOffsetsHandler defaultOptionsHandler =
+            new ListOffsetsHandler(offsetTimestampsByPartition, new 
ListOffsetsOptions(), logContext);
+        ListOffsetsRequest.Builder builder =
+            defaultOptionsHandler.buildBatchedRequest(node.id(), mkSet(t0p0, 
t0p1, t1p0));
+        assertEquals(1, builder.oldestAllowedVersion());
+
+        ListOffsetsHandler readCommittedHandler =
+            new ListOffsetsHandler(
+                offsetTimestampsByPartition, new 
ListOffsetsOptions(IsolationLevel.READ_COMMITTED), logContext);
+        builder = readCommittedHandler.buildBatchedRequest(node.id(), 
mkSet(t0p0, t0p1, t1p0));
+        assertEquals(2, builder.oldestAllowedVersion());
+
+        builder = readCommittedHandler.buildBatchedRequest(node.id(), 
mkSet(t0p0, t0p1, t1p0, t1p1));
+        assertEquals(7, builder.oldestAllowedVersion());
+    }
+
+    @Test
+    public void testHandleSuccessfulResponse() {
+        ApiResult<TopicPartition, ListOffsetsResultInfo> result =
+            handleResponse(createResponse(emptyMap()));
+
+        assertResult(result, offsetTimestampsByPartition.keySet(), emptyMap(), 
emptyList(), emptySet());
+    }
+
+    @Test
+    public void testHandlePartitionTimeoutResponse() {

Review Comment:
   We should change method name to testHandleRetrialbeXXX to be more clear.



##########
clients/src/test/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandlerTest.java:
##########
@@ -0,0 +1,308 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin.internals;
+
+import static java.util.Collections.emptyList;
+import static java.util.Collections.emptyMap;
+import static java.util.Collections.emptySet;
+import static java.util.Collections.singleton;
+import static org.apache.kafka.common.utils.Utils.mkSet;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.kafka.clients.admin.ListOffsetsOptions;
+import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo;
+import org.apache.kafka.clients.admin.OffsetSpec;
+import org.apache.kafka.clients.admin.internals.AdminApiHandler.ApiResult;
+import org.apache.kafka.common.IsolationLevel;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import 
org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
+import org.apache.kafka.common.message.ListOffsetsResponseData;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ListOffsetsRequest;
+import org.apache.kafka.common.requests.ListOffsetsResponse;
+import org.apache.kafka.common.utils.LogContext;
+import org.junit.jupiter.api.Test;
+
+public final class ListOffsetsHandlerTest {
+
+    private final LogContext logContext = new LogContext();
+
+    private final TopicPartition t0p0 = new TopicPartition("t0", 0);
+    private final TopicPartition t0p1 = new TopicPartition("t0", 1);
+    private final TopicPartition t1p0 = new TopicPartition("t1", 0);
+    private final TopicPartition t1p1 = new TopicPartition("t1", 1);
+
+    private final Node node = new Node(1, "host", 1234);
+
+    private final Map<TopicPartition, Long> offsetTimestampsByPartition = new 
HashMap<TopicPartition, Long>() {
+        {
+            put(t0p0, ListOffsetsRequest.LATEST_TIMESTAMP);
+            put(t0p1, ListOffsetsRequest.EARLIEST_TIMESTAMP);
+            put(t1p0, 123L);
+            put(t1p1, ListOffsetsRequest.MAX_TIMESTAMP);
+        }
+    };
+
+    @Test
+    public void testBuildRequestSimple() {
+        ListOffsetsHandler handler =
+            new ListOffsetsHandler(offsetTimestampsByPartition, new 
ListOffsetsOptions(), logContext);
+        ListOffsetsRequest request = handler.buildBatchedRequest(node.id(), 
mkSet(t0p0, t0p1)).build();
+        List<ListOffsetsTopic> topics = request.topics();
+        assertEquals(1, topics.size());
+        ListOffsetsTopic topic = topics.get(0);
+        assertEquals(2, topic.partitions().size());
+        for (ListOffsetsPartition partition : topic.partitions()) {
+            TopicPartition topicPartition = new TopicPartition(topic.name(), 
partition.partitionIndex());
+            assertExpectedTimestamp(topicPartition, partition.timestamp());
+        }
+        assertEquals(IsolationLevel.READ_UNCOMMITTED, 
request.isolationLevel());
+    }
+
+    @Test
+    public void testBuildRequestMultipleTopicsWithReadCommitted() {
+        ListOffsetsHandler handler =
+            new ListOffsetsHandler(
+                offsetTimestampsByPartition, new 
ListOffsetsOptions(IsolationLevel.READ_COMMITTED), logContext);
+        ListOffsetsRequest request =
+            handler.buildBatchedRequest(node.id(), 
offsetTimestampsByPartition.keySet()).build();
+        List<ListOffsetsTopic> topics = request.topics();
+        assertEquals(2, topics.size());
+        Map<TopicPartition, ListOffsetsPartition> partitions = new HashMap<>();
+        for (ListOffsetsTopic topic : topics) {
+            for (ListOffsetsPartition partition : topic.partitions()) {
+                partitions.put(new TopicPartition(topic.name(), 
partition.partitionIndex()), partition);
+            }
+        }
+        assertEquals(4, partitions.size());
+        for (Map.Entry<TopicPartition, ListOffsetsPartition> entry : 
partitions.entrySet()) {
+            assertExpectedTimestamp(entry.getKey(), 
entry.getValue().timestamp());
+        }
+        assertEquals(IsolationLevel.READ_COMMITTED, request.isolationLevel());
+    }
+
+    @Test
+    public void testBuildRequestAllowedVersions() {

Review Comment:
   nit: allowedVersion check is not the coverage of this PR, I think we can 
consolidate these 3 BuildRequest-tests in one test.



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin.internals;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.kafka.clients.admin.ListOffsetsOptions;
+import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo;
+import 
org.apache.kafka.clients.admin.internals.AdminApiFuture.SimpleAdminApiFuture;
+import org.apache.kafka.clients.admin.internals.AdminApiHandler.Batched;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.InvalidMetadataException;
+import org.apache.kafka.common.errors.RetriableException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import 
org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.requests.ListOffsetsRequest;
+import org.apache.kafka.common.requests.ListOffsetsResponse;
+import org.apache.kafka.common.utils.CollectionUtils;
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+public final class ListOffsetsHandler extends Batched<TopicPartition, 
ListOffsetsResultInfo> {
+
+    private final Map<TopicPartition, Long> offsetTimestampsByPartition;
+    private final ListOffsetsOptions options;
+    private final Logger log;
+    private final AdminApiLookupStrategy<TopicPartition> lookupStrategy;
+
+    public ListOffsetsHandler(
+        Map<TopicPartition, Long> offsetTimestampsByPartition,
+        ListOffsetsOptions options,
+        LogContext logContext
+    ) {
+        this.offsetTimestampsByPartition = offsetTimestampsByPartition;
+        this.options = options;
+        this.log = logContext.logger(ListOffsetsHandler.class);
+        this.lookupStrategy = new PartitionLeaderStrategy(logContext);
+    }
+
+    @Override
+    public String apiName() {
+        return "listOffsets";
+    }
+
+    @Override
+    public AdminApiLookupStrategy<TopicPartition> lookupStrategy() {
+        return this.lookupStrategy;
+    }
+
+    @Override
+    ListOffsetsRequest.Builder buildBatchedRequest(int brokerId, 
Set<TopicPartition> keys) {
+        Map<String, ListOffsetsTopic> topicsByName = 
CollectionUtils.groupPartitionsByTopic(
+            keys,
+            topicName -> new ListOffsetsTopic().setName(topicName),
+            (listOffsetsTopic, partitionId) -> {
+                TopicPartition topicPartition = new 
TopicPartition(listOffsetsTopic.name(), partitionId);
+                long offsetTimestamp = 
offsetTimestampsByPartition.get(topicPartition);
+                listOffsetsTopic.partitions().add(
+                    new ListOffsetsPartition()
+                        .setPartitionIndex(partitionId)
+                        .setTimestamp(offsetTimestamp));
+            });
+        boolean supportsMaxTimestamp = keys
+            .stream()
+            .anyMatch(key -> offsetTimestampsByPartition.get(key) == 
ListOffsetsRequest.MAX_TIMESTAMP);
+
+        return ListOffsetsRequest.Builder
+            .forConsumer(true, options.isolationLevel(), supportsMaxTimestamp)
+            .setTargetTimes(new ArrayList<>(topicsByName.values()));
+    }
+
+    @Override
+    public ApiResult<TopicPartition, ListOffsetsResultInfo> handleResponse(
+        Node broker,
+        Set<TopicPartition> keys,
+        AbstractResponse abstractResponse
+    ) {
+        ListOffsetsResponse response = (ListOffsetsResponse) abstractResponse;
+        Map<TopicPartition, ListOffsetsResultInfo> completed = new HashMap<>();
+        Map<TopicPartition, Throwable> failed = new HashMap<>();
+        List<TopicPartition> unmapped = new ArrayList<>();
+        Set<TopicPartition> retriable = new HashSet<>();
+
+        for (ListOffsetsTopicResponse topic : response.topics()) {
+            for (ListOffsetsPartitionResponse partition : topic.partitions()) {
+                TopicPartition topicPartition = new 
TopicPartition(topic.name(), partition.partitionIndex());
+                Errors error = Errors.forCode(partition.errorCode());
+                if (!offsetTimestampsByPartition.containsKey(topicPartition)) {
+                    log.warn("ListOffsets response includes unknown topic 
partition {}", topicPartition);
+                } else if (error == Errors.NONE) {
+                    Optional<Integer> leaderEpoch = (partition.leaderEpoch() 
== ListOffsetsResponse.UNKNOWN_EPOCH)
+                        ? Optional.empty()
+                        : Optional.of(partition.leaderEpoch());
+                    completed.put(
+                        topicPartition,
+                        new ListOffsetsResultInfo(partition.offset(), 
partition.timestamp(), leaderEpoch));
+                } else {
+                    handlePartitionError(topicPartition, error, failed, 
unmapped, retriable);
+                }
+            }
+        }
+
+        // Sanity-check if the current leader for these partitions returned 
results for all of them
+        for (TopicPartition topicPartition : keys) {
+            if (unmapped.isEmpty()
+                && !completed.containsKey(topicPartition)
+                && !failed.containsKey(topicPartition)
+                && !retriable.contains(topicPartition)
+            ) {
+                ApiException sanityCheckException = new ApiException(
+                    "The response from broker " + broker.id() +
+                        " did not contain a result for topic partition " + 
topicPartition);
+                log.error(
+                    "ListOffsets request for topic partition {} failed sanity 
check",
+                    topicPartition,
+                    sanityCheckException);
+                failed.put(topicPartition, sanityCheckException);
+            }
+        }
+
+        return new ApiResult<>(completed, failed, unmapped);
+    }
+
+    private void handlePartitionError(
+        TopicPartition topicPartition,
+        Errors error,
+        Map<TopicPartition, Throwable> failed,
+        List<TopicPartition> unmapped,
+        Set<TopicPartition> retriable
+    ) {
+        ApiException apiException = error.exception();
+        if (apiException instanceof InvalidMetadataException) {
+            log.debug(
+                "ListOffsets lookup request for topic partition {} will be 
retried due to invalid metadata",
+                topicPartition,
+                apiException);
+            unmapped.add(topicPartition);

Review Comment:
   It's a little radical to retry on all kinds of InvalidMetadataException, but 
it's OK to do this to avoid too much if else.



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin.internals;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.kafka.clients.admin.ListOffsetsOptions;
+import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo;
+import 
org.apache.kafka.clients.admin.internals.AdminApiFuture.SimpleAdminApiFuture;
+import org.apache.kafka.clients.admin.internals.AdminApiHandler.Batched;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.InvalidMetadataException;
+import org.apache.kafka.common.errors.RetriableException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import 
org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.requests.ListOffsetsRequest;
+import org.apache.kafka.common.requests.ListOffsetsResponse;
+import org.apache.kafka.common.utils.CollectionUtils;
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+public final class ListOffsetsHandler extends Batched<TopicPartition, 
ListOffsetsResultInfo> {
+
+    private final Map<TopicPartition, Long> offsetTimestampsByPartition;
+    private final ListOffsetsOptions options;
+    private final Logger log;
+    private final AdminApiLookupStrategy<TopicPartition> lookupStrategy;
+
+    public ListOffsetsHandler(
+        Map<TopicPartition, Long> offsetTimestampsByPartition,
+        ListOffsetsOptions options,
+        LogContext logContext
+    ) {
+        this.offsetTimestampsByPartition = offsetTimestampsByPartition;
+        this.options = options;
+        this.log = logContext.logger(ListOffsetsHandler.class);
+        this.lookupStrategy = new PartitionLeaderStrategy(logContext);
+    }
+
+    @Override
+    public String apiName() {
+        return "listOffsets";
+    }
+
+    @Override
+    public AdminApiLookupStrategy<TopicPartition> lookupStrategy() {
+        return this.lookupStrategy;
+    }
+
+    @Override
+    ListOffsetsRequest.Builder buildBatchedRequest(int brokerId, 
Set<TopicPartition> keys) {
+        Map<String, ListOffsetsTopic> topicsByName = 
CollectionUtils.groupPartitionsByTopic(
+            keys,
+            topicName -> new ListOffsetsTopic().setName(topicName),
+            (listOffsetsTopic, partitionId) -> {
+                TopicPartition topicPartition = new 
TopicPartition(listOffsetsTopic.name(), partitionId);
+                long offsetTimestamp = 
offsetTimestampsByPartition.get(topicPartition);
+                listOffsetsTopic.partitions().add(
+                    new ListOffsetsPartition()
+                        .setPartitionIndex(partitionId)
+                        .setTimestamp(offsetTimestamp));
+            });
+        boolean supportsMaxTimestamp = keys
+            .stream()
+            .anyMatch(key -> offsetTimestampsByPartition.get(key) == 
ListOffsetsRequest.MAX_TIMESTAMP);
+
+        return ListOffsetsRequest.Builder
+            .forConsumer(true, options.isolationLevel(), supportsMaxTimestamp)
+            .setTargetTimes(new ArrayList<>(topicsByName.values()));
+    }
+
+    @Override
+    public ApiResult<TopicPartition, ListOffsetsResultInfo> handleResponse(
+        Node broker,
+        Set<TopicPartition> keys,
+        AbstractResponse abstractResponse
+    ) {
+        ListOffsetsResponse response = (ListOffsetsResponse) abstractResponse;
+        Map<TopicPartition, ListOffsetsResultInfo> completed = new HashMap<>();
+        Map<TopicPartition, Throwable> failed = new HashMap<>();
+        List<TopicPartition> unmapped = new ArrayList<>();
+        Set<TopicPartition> retriable = new HashSet<>();
+
+        for (ListOffsetsTopicResponse topic : response.topics()) {
+            for (ListOffsetsPartitionResponse partition : topic.partitions()) {
+                TopicPartition topicPartition = new 
TopicPartition(topic.name(), partition.partitionIndex());
+                Errors error = Errors.forCode(partition.errorCode());
+                if (!offsetTimestampsByPartition.containsKey(topicPartition)) {
+                    log.warn("ListOffsets response includes unknown topic 
partition {}", topicPartition);
+                } else if (error == Errors.NONE) {
+                    Optional<Integer> leaderEpoch = (partition.leaderEpoch() 
== ListOffsetsResponse.UNKNOWN_EPOCH)
+                        ? Optional.empty()
+                        : Optional.of(partition.leaderEpoch());
+                    completed.put(
+                        topicPartition,
+                        new ListOffsetsResultInfo(partition.offset(), 
partition.timestamp(), leaderEpoch));
+                } else {
+                    handlePartitionError(topicPartition, error, failed, 
unmapped, retriable);
+                }
+            }
+        }
+
+        // Sanity-check if the current leader for these partitions returned 
results for all of them
+        for (TopicPartition topicPartition : keys) {
+            if (unmapped.isEmpty()
+                && !completed.containsKey(topicPartition)
+                && !failed.containsKey(topicPartition)
+                && !retriable.contains(topicPartition)
+            ) {
+                ApiException sanityCheckException = new ApiException(
+                    "The response from broker " + broker.id() +
+                        " did not contain a result for topic partition " + 
topicPartition);
+                log.error(
+                    "ListOffsets request for topic partition {} failed sanity 
check",
+                    topicPartition,
+                    sanityCheckException);
+                failed.put(topicPartition, sanityCheckException);
+            }
+        }
+
+        return new ApiResult<>(completed, failed, unmapped);
+    }
+
+    private void handlePartitionError(
+        TopicPartition topicPartition,
+        Errors error,
+        Map<TopicPartition, Throwable> failed,
+        List<TopicPartition> unmapped,
+        Set<TopicPartition> retriable
+    ) {
+        ApiException apiException = error.exception();
+        if (apiException instanceof InvalidMetadataException) {
+            log.debug(
+                "ListOffsets lookup request for topic partition {} will be 
retried due to invalid metadata",
+                topicPartition,
+                apiException);
+            unmapped.add(topicPartition);
+        } else if (apiException instanceof RetriableException) {
+            log.debug(
+                "ListOffsets fulfillment request for topic partition {} will 
be retried due to {}",
+                topicPartition,
+                apiException);
+            retriable.add(topicPartition);
+        } else {
+            log.error(
+                "ListOffsets request for topic partition {} failed due to an 
unexpected error",
+                topicPartition,
+                apiException);
+            failed.put(topicPartition, apiException);
+        }
+    }
+
+    @Override
+    public Map<TopicPartition, Throwable> handleUnsupportedVersionException(
+        UnsupportedVersionException exception, Set<TopicPartition> keys

Review Comment:
   I think we can also add broker to this method, and add some logs such as: 
"Broker " + broker + " does not support MAX_TIMESTAMP offset spec".



##########
clients/src/test/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandlerTest.java:
##########
@@ -0,0 +1,308 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin.internals;
+
+import static java.util.Collections.emptyList;
+import static java.util.Collections.emptyMap;
+import static java.util.Collections.emptySet;
+import static java.util.Collections.singleton;
+import static org.apache.kafka.common.utils.Utils.mkSet;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.kafka.clients.admin.ListOffsetsOptions;
+import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo;
+import org.apache.kafka.clients.admin.OffsetSpec;
+import org.apache.kafka.clients.admin.internals.AdminApiHandler.ApiResult;
+import org.apache.kafka.common.IsolationLevel;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import 
org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
+import org.apache.kafka.common.message.ListOffsetsResponseData;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ListOffsetsRequest;
+import org.apache.kafka.common.requests.ListOffsetsResponse;
+import org.apache.kafka.common.utils.LogContext;
+import org.junit.jupiter.api.Test;
+
+public final class ListOffsetsHandlerTest {
+
+    private final LogContext logContext = new LogContext();
+
+    private final TopicPartition t0p0 = new TopicPartition("t0", 0);
+    private final TopicPartition t0p1 = new TopicPartition("t0", 1);
+    private final TopicPartition t1p0 = new TopicPartition("t1", 0);
+    private final TopicPartition t1p1 = new TopicPartition("t1", 1);
+
+    private final Node node = new Node(1, "host", 1234);
+
+    private final Map<TopicPartition, Long> offsetTimestampsByPartition = new 
HashMap<TopicPartition, Long>() {
+        {
+            put(t0p0, ListOffsetsRequest.LATEST_TIMESTAMP);
+            put(t0p1, ListOffsetsRequest.EARLIEST_TIMESTAMP);
+            put(t1p0, 123L);
+            put(t1p1, ListOffsetsRequest.MAX_TIMESTAMP);
+        }
+    };
+
+    @Test
+    public void testBuildRequestSimple() {
+        ListOffsetsHandler handler =
+            new ListOffsetsHandler(offsetTimestampsByPartition, new 
ListOffsetsOptions(), logContext);
+        ListOffsetsRequest request = handler.buildBatchedRequest(node.id(), 
mkSet(t0p0, t0p1)).build();
+        List<ListOffsetsTopic> topics = request.topics();
+        assertEquals(1, topics.size());
+        ListOffsetsTopic topic = topics.get(0);
+        assertEquals(2, topic.partitions().size());
+        for (ListOffsetsPartition partition : topic.partitions()) {
+            TopicPartition topicPartition = new TopicPartition(topic.name(), 
partition.partitionIndex());
+            assertExpectedTimestamp(topicPartition, partition.timestamp());
+        }
+        assertEquals(IsolationLevel.READ_UNCOMMITTED, 
request.isolationLevel());
+    }
+
+    @Test
+    public void testBuildRequestMultipleTopicsWithReadCommitted() {
+        ListOffsetsHandler handler =
+            new ListOffsetsHandler(
+                offsetTimestampsByPartition, new 
ListOffsetsOptions(IsolationLevel.READ_COMMITTED), logContext);
+        ListOffsetsRequest request =
+            handler.buildBatchedRequest(node.id(), 
offsetTimestampsByPartition.keySet()).build();
+        List<ListOffsetsTopic> topics = request.topics();
+        assertEquals(2, topics.size());
+        Map<TopicPartition, ListOffsetsPartition> partitions = new HashMap<>();
+        for (ListOffsetsTopic topic : topics) {
+            for (ListOffsetsPartition partition : topic.partitions()) {
+                partitions.put(new TopicPartition(topic.name(), 
partition.partitionIndex()), partition);
+            }
+        }
+        assertEquals(4, partitions.size());
+        for (Map.Entry<TopicPartition, ListOffsetsPartition> entry : 
partitions.entrySet()) {
+            assertExpectedTimestamp(entry.getKey(), 
entry.getValue().timestamp());
+        }
+        assertEquals(IsolationLevel.READ_COMMITTED, request.isolationLevel());
+    }
+
+    @Test
+    public void testBuildRequestAllowedVersions() {
+        ListOffsetsHandler defaultOptionsHandler =
+            new ListOffsetsHandler(offsetTimestampsByPartition, new 
ListOffsetsOptions(), logContext);
+        ListOffsetsRequest.Builder builder =
+            defaultOptionsHandler.buildBatchedRequest(node.id(), mkSet(t0p0, 
t0p1, t1p0));
+        assertEquals(1, builder.oldestAllowedVersion());
+
+        ListOffsetsHandler readCommittedHandler =
+            new ListOffsetsHandler(
+                offsetTimestampsByPartition, new 
ListOffsetsOptions(IsolationLevel.READ_COMMITTED), logContext);
+        builder = readCommittedHandler.buildBatchedRequest(node.id(), 
mkSet(t0p0, t0p1, t1p0));
+        assertEquals(2, builder.oldestAllowedVersion());
+
+        builder = readCommittedHandler.buildBatchedRequest(node.id(), 
mkSet(t0p0, t0p1, t1p0, t1p1));
+        assertEquals(7, builder.oldestAllowedVersion());
+    }
+
+    @Test
+    public void testHandleSuccessfulResponse() {
+        ApiResult<TopicPartition, ListOffsetsResultInfo> result =
+            handleResponse(createResponse(emptyMap()));
+
+        assertResult(result, offsetTimestampsByPartition.keySet(), emptyMap(), 
emptyList(), emptySet());
+    }
+
+    @Test
+    public void testHandlePartitionTimeoutResponse() {
+        TopicPartition errorPartition = t0p0;
+        Map<TopicPartition, Short> errorsByPartition = new HashMap<>();
+        errorsByPartition.put(errorPartition, Errors.REQUEST_TIMED_OUT.code());
+
+        ApiResult<TopicPartition, ListOffsetsResultInfo> result =
+            handleResponse(createResponse(errorsByPartition));
+
+        // Timeouts should be retried within the fulfillment stage as they are 
a common type of
+        // retriable error.
+        Set<TopicPartition> retriable = singleton(errorPartition);
+        Set<TopicPartition> completed = new 
HashSet<>(offsetTimestampsByPartition.keySet());
+        completed.removeAll(retriable);
+        assertResult(result, completed, emptyMap(), emptyList(), retriable);
+    }
+
+    @Test
+    public void testHandlePartitionInvalidMetadataResponse() {
+        TopicPartition errorPartition = t0p0;
+        Errors error = Errors.NOT_LEADER_OR_FOLLOWER;
+        Map<TopicPartition, Short> errorsByPartition = new HashMap<>();
+        errorsByPartition.put(errorPartition, error.code());
+
+        ApiResult<TopicPartition, ListOffsetsResultInfo> result =
+            handleResponse(createResponse(errorsByPartition));
+
+        // Invalid metadata errors should be retried from the lookup stage as 
the partition-to-leader
+        // mappings should be recalculated.
+        List<TopicPartition> unmapped = new ArrayList<>();
+        unmapped.add(errorPartition);
+        Set<TopicPartition> completed = new 
HashSet<>(offsetTimestampsByPartition.keySet());
+        completed.removeAll(unmapped);
+        assertResult(result, completed, emptyMap(), unmapped, emptySet());
+    }
+
+    @Test
+    public void testHandlePartitionErrorResponse() {
+        TopicPartition errorPartition = t0p0;
+        Errors error = Errors.UNKNOWN_SERVER_ERROR;
+        Map<TopicPartition, Short> errorsByPartition = new HashMap<>();
+        errorsByPartition.put(errorPartition, error.code());
+
+        ApiResult<TopicPartition, ListOffsetsResultInfo> result =
+            handleResponse(createResponse(errorsByPartition));
+
+        Map<TopicPartition, Throwable> failed = new HashMap<>();
+        failed.put(errorPartition, error.exception());
+        Set<TopicPartition> completed = new 
HashSet<>(offsetTimestampsByPartition.keySet());
+        completed.removeAll(failed.keySet());
+        assertResult(result, completed, failed, emptyList(), emptySet());
+    }
+
+    @Test
+    public void testHandleResponseSanityCheck() {
+        TopicPartition errorPartition = t0p0;
+        Map<TopicPartition, Long> specsByPartition = new 
HashMap<>(offsetTimestampsByPartition);
+        specsByPartition.remove(errorPartition);
+
+        ApiResult<TopicPartition, ListOffsetsResultInfo> result =
+            handleResponse(createResponse(emptyMap(), specsByPartition));
+
+        assertEquals(offsetTimestampsByPartition.size() - 1, 
result.completedKeys.size());
+        assertEquals(1, result.failedKeys.size());
+        assertEquals(errorPartition, 
result.failedKeys.keySet().iterator().next());
+        String sanityCheckMessage = 
result.failedKeys.get(errorPartition).getMessage();
+        assertTrue(sanityCheckMessage.contains("did not contain a result for 
topic partition"));
+        assertTrue(result.unmappedKeys.isEmpty());
+    }
+
+    @Test
+    public void testHandleResponseUnsupportedVersion() {
+        UnsupportedVersionException uve = new UnsupportedVersionException("");
+        Map<TopicPartition, OffsetSpec> maxTimestampPartitions = new 
HashMap<>();
+        maxTimestampPartitions.put(t1p1, OffsetSpec.maxTimestamp());
+
+        ListOffsetsHandler handler =
+            new ListOffsetsHandler(offsetTimestampsByPartition, new 
ListOffsetsOptions(), logContext);
+
+        final Map<TopicPartition, Long> nonMaxTimestampPartitions = new 
HashMap<>(offsetTimestampsByPartition);
+        maxTimestampPartitions.forEach((k, v) -> 
nonMaxTimestampPartitions.remove(k));
+        // Unsupported version exceptions currently cannot be handled if 
there's no partition with a
+        // MAX_TIMESTAMP spec...
+        Set<TopicPartition> keysToTest = nonMaxTimestampPartitions.keySet();
+        Set<TopicPartition> expectedFailures = keysToTest;
+        assertEquals(
+            mapToError(expectedFailures, uve),
+            handler.handleUnsupportedVersionException(uve, keysToTest));
+
+        // ...or if there are only partitions with MAX_TIMESTAMP specs...
+        keysToTest = maxTimestampPartitions.keySet();
+        expectedFailures = keysToTest;
+        assertEquals(
+            mapToError(expectedFailures, uve),
+            handler.handleUnsupportedVersionException(uve, keysToTest));
+
+        keysToTest = offsetTimestampsByPartition.keySet();

Review Comment:
   nit: add a comment about this part, for example, xxx partitions both 
MAX_TIMESTAMP and not.



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java:
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin.internals;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.kafka.clients.admin.ListOffsetsOptions;
+import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo;
+import org.apache.kafka.clients.admin.OffsetSpec;
+import org.apache.kafka.clients.admin.OffsetSpec.TimestampSpec;
+import 
org.apache.kafka.clients.admin.internals.AdminApiFuture.SimpleAdminApiFuture;
+import org.apache.kafka.clients.admin.internals.AdminApiHandler.Batched;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.InvalidMetadataException;
+import org.apache.kafka.common.errors.RetriableException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import 
org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.requests.ListOffsetsRequest;
+import org.apache.kafka.common.requests.ListOffsetsResponse;
+import org.apache.kafka.common.utils.CollectionUtils;
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+public final class ListOffsetsHandler extends Batched<TopicPartition, 
ListOffsetsResultInfo> {
+
+    private final Map<TopicPartition, OffsetSpec> offsetSpecsByPartition;
+    private final ListOffsetsOptions options;
+    private final Logger log;
+    private final AdminApiLookupStrategy<TopicPartition> lookupStrategy;
+
+    public ListOffsetsHandler(
+        Map<TopicPartition, OffsetSpec> offsetSpecsByPartition,
+        ListOffsetsOptions options,
+        LogContext logContext
+    ) {
+        this.offsetSpecsByPartition = offsetSpecsByPartition;
+        this.options = options;
+        this.log = logContext.logger(ListOffsetsHandler.class);
+        this.lookupStrategy = new PartitionLeaderStrategy(logContext);
+    }
+
+    @Override
+    public String apiName() {
+        return "listOffsets";
+    }
+
+    @Override
+    public AdminApiLookupStrategy<TopicPartition> lookupStrategy() {
+        return this.lookupStrategy;
+    }
+
+    @Override
+    ListOffsetsRequest.Builder buildBatchedRequest(int brokerId, 
Set<TopicPartition> keys) {
+        Map<String, ListOffsetsTopic> topicsByName = 
CollectionUtils.groupPartitionsByTopic(
+            keys,
+            topicName -> new ListOffsetsTopic().setName(topicName),
+            (listOffsetsTopic, partitionId) -> {
+                TopicPartition topicPartition = new 
TopicPartition(listOffsetsTopic.name(), partitionId);
+                OffsetSpec offsetSpec = 
offsetSpecsByPartition.get(topicPartition);
+                long offsetQuery = getOffsetFromSpec(offsetSpec);
+                listOffsetsTopic.partitions().add(
+                    new ListOffsetsPartition()
+                        .setPartitionIndex(partitionId)
+                        .setTimestamp(offsetQuery));
+            });
+        boolean supportsMaxTimestamp = keys

Review Comment:
   The original logic is a little messy since we are trying to maintain 
retriable state using `handleUnsupportedVersionException`, but return value is 
only boolean, so we add `supportsMaxTimestamp` to maintain more information, 
and the logic will become more messy if we upgrade it once more. 
   Your change makes sense to me.



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java:
##########
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.admin.internals;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.kafka.clients.admin.ListOffsetsOptions;
+import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo;
+import org.apache.kafka.clients.admin.OffsetSpec;
+import org.apache.kafka.clients.admin.OffsetSpec.TimestampSpec;
+import 
org.apache.kafka.clients.admin.internals.AdminApiFuture.SimpleAdminApiFuture;
+import org.apache.kafka.clients.admin.internals.AdminApiHandler.Batched;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.InvalidMetadataException;
+import org.apache.kafka.common.errors.RetriableException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import 
org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsPartition;
+import org.apache.kafka.common.message.ListOffsetsRequestData.ListOffsetsTopic;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse;
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.requests.ListOffsetsRequest;
+import org.apache.kafka.common.requests.ListOffsetsResponse;
+import org.apache.kafka.common.utils.CollectionUtils;
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+public final class ListOffsetsHandler extends Batched<TopicPartition, 
ListOffsetsResultInfo> {
+
+    private final Map<TopicPartition, OffsetSpec> offsetSpecsByPartition;
+    private final ListOffsetsOptions options;
+    private final Logger log;
+    private final AdminApiLookupStrategy<TopicPartition> lookupStrategy;
+
+    public ListOffsetsHandler(
+        Map<TopicPartition, OffsetSpec> offsetSpecsByPartition,
+        ListOffsetsOptions options,
+        LogContext logContext
+    ) {
+        this.offsetSpecsByPartition = offsetSpecsByPartition;
+        this.options = options;
+        this.log = logContext.logger(ListOffsetsHandler.class);
+        this.lookupStrategy = new PartitionLeaderStrategy(logContext);
+    }
+
+    @Override
+    public String apiName() {
+        return "listOffsets";
+    }
+
+    @Override
+    public AdminApiLookupStrategy<TopicPartition> lookupStrategy() {
+        return this.lookupStrategy;
+    }
+
+    @Override
+    ListOffsetsRequest.Builder buildBatchedRequest(int brokerId, 
Set<TopicPartition> keys) {
+        Map<String, ListOffsetsTopic> topicsByName = 
CollectionUtils.groupPartitionsByTopic(
+            keys,
+            topicName -> new ListOffsetsTopic().setName(topicName),
+            (listOffsetsTopic, partitionId) -> {
+                TopicPartition topicPartition = new 
TopicPartition(listOffsetsTopic.name(), partitionId);
+                OffsetSpec offsetSpec = 
offsetSpecsByPartition.get(topicPartition);
+                long offsetQuery = getOffsetFromSpec(offsetSpec);
+                listOffsetsTopic.partitions().add(
+                    new ListOffsetsPartition()
+                        .setPartitionIndex(partitionId)
+                        .setTimestamp(offsetQuery));
+            });
+        boolean supportsMaxTimestamp = keys
+            .stream()
+            .anyMatch(key -> 
getOffsetFromSpec(offsetSpecsByPartition.get(key)) == 
ListOffsetsRequest.MAX_TIMESTAMP);
+
+        return ListOffsetsRequest.Builder
+            .forConsumer(true, options.isolationLevel(), supportsMaxTimestamp)
+            .setTargetTimes(new ArrayList<>(topicsByName.values()));
+    }
+
+    @Override
+    public ApiResult<TopicPartition, ListOffsetsResultInfo> handleResponse(
+        Node broker,
+        Set<TopicPartition> keys,
+        AbstractResponse abstractResponse
+    ) {
+        ListOffsetsResponse response = (ListOffsetsResponse) abstractResponse;
+        Map<TopicPartition, ListOffsetsResultInfo> completed = new HashMap<>();
+        Map<TopicPartition, Throwable> failed = new HashMap<>();
+        List<TopicPartition> unmapped = new ArrayList<>();
+        Set<TopicPartition> retriable = new HashSet<>();
+
+        for (ListOffsetsTopicResponse topic : response.topics()) {
+            for (ListOffsetsPartitionResponse partition : topic.partitions()) {
+                TopicPartition topicPartition = new 
TopicPartition(topic.name(), partition.partitionIndex());
+                Errors error = Errors.forCode(partition.errorCode());
+                OffsetSpec offsetRequestSpec = 
offsetSpecsByPartition.get(topicPartition);
+                if (offsetRequestSpec == null) {
+                    log.warn("ListOffsets response includes unknown topic 
partition {}", topicPartition);
+                } else if (error == Errors.NONE) {
+                    Optional<Integer> leaderEpoch = (partition.leaderEpoch() 
== ListOffsetsResponse.UNKNOWN_EPOCH)
+                        ? Optional.empty()
+                        : Optional.of(partition.leaderEpoch());
+                    completed.put(
+                        topicPartition,
+                        new ListOffsetsResultInfo(partition.offset(), 
partition.timestamp(), leaderEpoch));
+                } else {
+                    handlePartitionError(topicPartition, error, failed, 
unmapped, retriable);
+                }
+            }
+        }
+
+        // Sanity-check if the current leader for these partitions returned 
results for all of them
+        for (TopicPartition topicPartition : keys) {
+            if (unmapped.isEmpty()
+                && !completed.containsKey(topicPartition)
+                && !failed.containsKey(topicPartition)
+                && !retriable.contains(topicPartition)
+            ) {
+                ApiException sanityCheckException = new ApiException(
+                    "The response from broker " + broker.id() +
+                        " did not contain a result for topic partition " + 
topicPartition);
+                log.error(
+                    "ListOffsets request for topic partition {} failed sanity 
check",
+                    topicPartition,
+                    sanityCheckException);
+                failed.put(topicPartition, sanityCheckException);
+            }
+        }
+
+        return new ApiResult<>(completed, failed, unmapped);
+    }
+
+    private void handlePartitionError(
+        TopicPartition topicPartition,
+        Errors error,
+        Map<TopicPartition, Throwable> failed,
+        List<TopicPartition> unmapped,
+        Set<TopicPartition> retriable
+    ) {
+        ApiException apiException = error.exception();
+        if (apiException instanceof InvalidMetadataException) {
+            log.debug(
+                "ListOffsets lookup request for topic partition {} will be 
retried due to invalid metadata",
+                topicPartition,
+                apiException);
+            unmapped.add(topicPartition);
+        } else if (apiException instanceof RetriableException) {
+            log.debug(
+                "ListOffsets fulfillment request for topic partition {} will 
be retried due to {}",
+                topicPartition,
+                apiException);
+            retriable.add(topicPartition);
+        } else {
+            log.error(
+                "ListOffsets request for topic partition {} failed due to an 
unexpected error",
+                topicPartition,
+                apiException);
+            failed.put(topicPartition, apiException);
+        }
+    }
+
+    @Override
+    public Map<TopicPartition, Throwable> handleUnsupportedVersionException(
+        UnsupportedVersionException exception, Set<TopicPartition> keys, 
boolean isFulfillmentStage
+    ) {
+        // An UnsupportedVersionException can be addressed only in the 
fulfillment stage...
+        if (!isFulfillmentStage) {
+            return keys.stream().collect(Collectors.toMap(k -> k, k -> 
exception));
+        }
+        Map<TopicPartition, Throwable> maxTimestampPartitions = new 
HashMap<>();
+        boolean containsNonMaxTimestampSpec = false;
+        for (TopicPartition topicPartition : keys) {
+            OffsetSpec offsetSpec = offsetSpecsByPartition.get(topicPartition);
+            if (getOffsetFromSpec(offsetSpec) == 
ListOffsetsRequest.MAX_TIMESTAMP) {
+                maxTimestampPartitions.put(topicPartition, exception);
+            } else {
+                containsNonMaxTimestampSpec = true;
+            }
+        }
+        // ...only if it's caused by a MAX_TIMESTAMP spec for some partition, 
and only if there is at least
+        // one other partition with a non-MAX_TIMESTAMP spec for which the 
fulfillment stage can be retried.
+        if (maxTimestampPartitions.isEmpty() || !containsNonMaxTimestampSpec) {
+            return keys.stream().collect(Collectors.toMap(k -> k, k -> 
exception));

Review Comment:
   To clarify, there are 3 situations we would meet here:
   1. keys are all non-MAX_TIMESTAMP, then all keys will fail directly;
   2. keys are all MAX_TIMESTAMP, then all keys will fail directly;
   3. keys are mixture of them, MAX_TIMESTAMP keys will fail directly and 
non-MAX_TIMESTAMP will left to retry(and will fail in the next round if it's 
still unsupported).
   
   This is the same with the original logic:
   1. keys are all non-MAX_TIMESTAMP, supportsMaxTimestamp=false, so all keys 
will fail directly.
   2. keys are all MAX_TIMESTAMP, then all keys will fail directly whether 
supportsMaxTimestamp Is true or false.
   3. keys are mixture, supportsMaxTimestamp=true, MAX_TIMESTAMP keys will fail 
directly and non-MAX_TIMESTAMP will left to retry(and will fail in the next 
round if it's still unsupported since supportsMaxTimestamp=false).
   
   The new logic will be clearer.



-- 
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: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to