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


##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiHandler.java:
##########
@@ -70,6 +71,22 @@
      */
     ApiResult<K, V> handleResponse(Node broker, Set<K> keys, AbstractResponse 
response);
 
+    /**
+     * Callback that is invoked when a request hits an 
UnsupportedVersionException. If the
+     * exception can be handled and the request retried, the appropriate 
changes to the request
+     * should be applied and that should be indicated via the returned boolean 
value. If the
+     * exception is thrown during the lookup stage the handler should delegate 
the handling to the
+     * lookup strategy.
+     *
+     * @return True if the exception can be handled; false otherwise.
+     */
+    default boolean handleUnsupportedVersionException(

Review Comment:
   I forgot the details about the whole call chain, but I think we should 
decide which keys (if any) can retry with lower version and which keys can not, 
keys which are not retriable should be put into the result, and others are left 
out of the result which will be retried automatically, similar to what we do in 
`handleResponse`.
   



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.function.Predicate;
+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.LogContext;
+import org.slf4j.Logger;
+
+public final class ListOffsetsHandler extends Batched<TopicPartition, 
ListOffsetsResultInfo> {
+
+    private final ListOffsetsOptions options;
+    private final Map<TopicPartition, OffsetSpec> offsetSpecsByPartition;
+    private final Logger log;
+    private final AdminApiLookupStrategy<TopicPartition> lookupStrategy;
+
+    private volatile boolean skipMaxTimestampRequests = false;
+
+    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) {
+        final Map<String, ListOffsetsTopic> topicsByName = new HashMap<>();
+        boolean supportsMaxTimestamp = false;
+        for (TopicPartition topicPartition : keys) {
+            OffsetSpec offsetSpec = offsetSpecsByPartition.get(topicPartition);
+            long offsetQuery = getOffsetFromSpec(offsetSpec);
+            if (offsetQuery == ListOffsetsRequest.MAX_TIMESTAMP) {
+                if (skipMaxTimestampRequests) {

Review Comment:
   Can we simplify this to `this.supportsMaxTimestamp = 
topicPartitionOffsets.values().stream()
               .anyMatch(timestamp -> timestamp == 
ListOffsetsRequest.MAX_TIMESTAMP)`?



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.function.Predicate;
+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.LogContext;
+import org.slf4j.Logger;
+
+public final class ListOffsetsHandler extends Batched<TopicPartition, 
ListOffsetsResultInfo> {
+
+    private final ListOffsetsOptions options;
+    private final Map<TopicPartition, OffsetSpec> offsetSpecsByPartition;
+    private final Logger log;
+    private final AdminApiLookupStrategy<TopicPartition> lookupStrategy;
+
+    private volatile boolean skipMaxTimestampRequests = false;
+
+    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) {
+        final Map<String, ListOffsetsTopic> topicsByName = new HashMap<>();
+        boolean supportsMaxTimestamp = false;
+        for (TopicPartition topicPartition : keys) {

Review Comment:
   How about reuse `CollectionUtils.groupPartitionsByTopic` to avoid this for 
loop.



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java:
##########
@@ -261,19 +262,30 @@ public void onFailure(
                 .collect(Collectors.toSet());
             retryLookup(keysToUnmap);
         } else {
+            boolean isFulfillmentStage = spec.scope instanceof 
FulfillmentScope;
+            if (handleUnsupportedVersionException(t, isFulfillmentStage)) {

Review Comment:
   I think we should add a branch like `else if (t instanceof 
UnsupportedVersionException)`, and return the keys which are not retriable to 
put into the result using `completeExceptionally(failed)`, and left the others 
to be retried.



##########
clients/src/main/java/org/apache/kafka/clients/admin/internals/ListOffsetsHandler.java:
##########
@@ -0,0 +1,256 @@
+/*
+ * 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.function.Predicate;
+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.LogContext;
+import org.slf4j.Logger;
+
+public final class ListOffsetsHandler extends Batched<TopicPartition, 
ListOffsetsResultInfo> {
+
+    private final ListOffsetsOptions options;
+    private final Map<TopicPartition, OffsetSpec> offsetSpecsByPartition;
+    private final Logger log;
+    private final AdminApiLookupStrategy<TopicPartition> lookupStrategy;
+
+    private volatile boolean skipMaxTimestampRequests = false;
+
+    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) {
+        final Map<String, ListOffsetsTopic> topicsByName = new HashMap<>();
+        boolean supportsMaxTimestamp = false;
+        for (TopicPartition topicPartition : keys) {
+            OffsetSpec offsetSpec = offsetSpecsByPartition.get(topicPartition);
+            long offsetQuery = getOffsetFromSpec(offsetSpec);
+            if (offsetQuery == ListOffsetsRequest.MAX_TIMESTAMP) {
+                if (skipMaxTimestampRequests) {
+                    continue;
+                }
+                supportsMaxTimestamp = true;
+            }
+            ListOffsetsTopic topic = topicsByName.computeIfAbsent(
+                topicPartition.topic(), topicName -> new 
ListOffsetsTopic().setName(topicName));
+            topic.partitions().add(
+                new ListOffsetsPartition()
+                    .setPartitionIndex(topicPartition.partition())
+                    .setTimestamp(offsetQuery));
+        }
+
+        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);
+                }
+            }
+        }
+
+        maybeMarkOtherFailures(

Review Comment:
   What does this method do? we have already handled all partitions.



-- 
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