chia7712 commented on a change in pull request #10275:
URL: https://github.com/apache/kafka/pull/10275#discussion_r589631318



##########
File path: 
clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersResult.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.annotation.InterfaceStability;
+import org.apache.kafka.common.internals.KafkaFutureImpl;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+
+@InterfaceStability.Evolving
+public class DescribeProducersResult {
+
+    private final Map<TopicPartition, KafkaFutureImpl<PartitionProducerState>> 
futures;
+
+    DescribeProducersResult(Map<TopicPartition, 
KafkaFutureImpl<PartitionProducerState>> futures) {
+        this.futures = futures;
+    }
+
+    public KafkaFuture<PartitionProducerState> partitionResult(final 
TopicPartition partition) {
+        KafkaFuture<PartitionProducerState> future = futures.get(partition);
+        if (future == null) {
+            throw new IllegalArgumentException("Topic partition " + partition +
+                " was not included in the request");
+        }
+        return future;
+    }
+
+    public KafkaFuture<Map<TopicPartition, PartitionProducerState>> all() {
+        return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0]))
+            .thenApply(nil -> {
+                Map<TopicPartition, PartitionProducerState> results = new 
HashMap<>(futures.size());
+                for (Map.Entry<TopicPartition, 
KafkaFutureImpl<PartitionProducerState>> entry : futures.entrySet()) {
+                    try {
+                        results.put(entry.getKey(), entry.getValue().get());
+                    } catch (InterruptedException | ExecutionException e) {
+                        // This should be unreachable, because allOf ensured 
that all the futures completed successfully.
+                        throw new KafkaException(e);
+                    }
+                }
+                return results;
+            });
+    }
+
+    public static class PartitionProducerState {
+        private final List<ProducerState> activeProducers;
+
+        public PartitionProducerState(List<ProducerState> activeProducers) {

Review comment:
       Could we change the modifier from public to package-private? Exposing 
this constructor to users is useless.

##########
File path: 
clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersOptions.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Collection;
+import java.util.Objects;
+import java.util.OptionalInt;
+
+/**
+ * Options for {@link Admin#describeProducers(Collection)}.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeProducersOptions extends 
AbstractOptions<DescribeProducersOptions> {

Review comment:
       Could we make it override `timeoutMs` in order to write following code.
   ```java
   new DescribeProducersOptions()
     .timeoutMs(11)
     .setBrokerId(1)
   ```

##########
File path: 
clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersOptions.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Collection;
+import java.util.Objects;
+import java.util.OptionalInt;
+
+/**
+ * Options for {@link Admin#describeProducers(Collection)}.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeProducersOptions extends 
AbstractOptions<DescribeProducersOptions> {
+    private OptionalInt brokerId = OptionalInt.empty();
+
+    public DescribeProducersOptions setBrokerId(int brokerId) {

Review comment:
       Other options classes does not call it `setXXX`. Maybe we should 
following consistent naming?

##########
File path: 
clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersOptions.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Collection;
+import java.util.Objects;
+import java.util.OptionalInt;
+
+/**
+ * Options for {@link Admin#describeProducers(Collection)}.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeProducersOptions extends 
AbstractOptions<DescribeProducersOptions> {
+    private OptionalInt brokerId = OptionalInt.empty();
+
+    public DescribeProducersOptions setBrokerId(int brokerId) {
+        this.brokerId = OptionalInt.of(brokerId);
+        return this;
+    }
+
+    public OptionalInt brokerId() {
+        return brokerId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        DescribeProducersOptions that = (DescribeProducersOptions) o;
+        return Objects.equals(brokerId, that.brokerId);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(brokerId);

Review comment:
       include `timeoutMs`?

##########
File path: 
clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java
##########
@@ -0,0 +1,464 @@
+/*
+ * 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 
org.apache.kafka.clients.admin.internals.AdminApiHandler.DynamicKeyMapping;
+import org.apache.kafka.clients.admin.internals.AdminApiHandler.KeyMappings;
+import 
org.apache.kafka.clients.admin.internals.AdminApiHandler.StaticKeyMapping;
+import 
org.apache.kafka.clients.admin.internals.AdminApiLookupStrategy.RequestScope;
+import org.apache.kafka.common.errors.DisconnectException;
+import org.apache.kafka.common.internals.KafkaFutureImpl;
+import org.apache.kafka.common.requests.AbstractRequest;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The `KafkaAdminClient`'s internal `Call` primitive is not a good fit for 
multi-stage
+ * request workflows such as we see with the group coordinator APIs or any 
request which
+ * needs to be sent to a partition leader. Typically these APIs have two 
concrete stages:
+ *
+ * 1. Lookup: Find the broker that can fulfill the request (e.g. partition 
leader or group
+ *            coordinator)
+ * 2. Fulfillment: Send the request to the broker found in the first step
+ *
+ * This is complicated by the fact that `Admin` APIs are typically batched, 
which
+ * means the Lookup stage may result in a set of brokers. For example, take a 
`ListOffsets`
+ * request for a set of topic partitions. In the Lookup stage, we will find 
the partition
+ * leaders for this set of partitions; in the Fulfillment stage, we will group 
together
+ * partition according to the IDs of the discovered leaders.
+ *
+ * Additionally, the flow between these two stages is bi-directional. We may 
find after
+ * sending a `ListOffsets` request to an expected leader that there was a 
leader change.
+ * This would result in a topic partition being sent back to the Lookup stage.
+ *
+ * Managing this complexity by chaining together `Call` implementations is 
challenging
+ * and messy, so instead we use this class to do the bookkeeping. It handles 
both the
+ * batching aspect as well as the transitions between the Lookup and 
Fulfillment stages.
+ *
+ * Note that the interpretation of the `retries` configuration becomes 
ambiguous
+ * for this kind of pipeline. We could treat it as an overall limit on the 
number
+ * of requests that can be sent, but that is not very useful because each 
pipeline
+ * has a minimum number of requests that need to be sent in order to satisfy 
the request.
+ * Instead, we treat this number of retries independently at each stage so 
that each
+ * stage has at least one opportunity to complete. So if a user sets 
`retries=1`, then
+ * the full pipeline can still complete as long as there are no request 
failures.
+ *
+ * @param <K> The key type, which is also the granularity of the request 
routing (e.g.
+ *            this could be `TopicPartition` in the case of requests intended 
for a partition
+ *            leader or the `GroupId` in the case of consumer group requests 
intended for
+ *            the group coordinator)
+ * @param <V> The fulfillment type for each key (e.g. this could be consumer 
group state
+ *            when the key type is a consumer `GroupId`)
+ */
+public class AdminApiDriver<K, V> {
+    private final Logger log;
+    private final long retryBackoffMs;
+    private final long deadlineMs;
+    private final AdminApiHandler<K, V> handler;
+    private final Optional<StaticKeyMapping<K>> staticMapping;
+    private final Optional<DynamicKeyMapping<K>> dynamicMapping;
+    private final Map<K, KafkaFutureImpl<V>> futures;
+
+    private final BiMultimap<RequestScope, K> lookupMap = new BiMultimap<>();
+    private final BiMultimap<BrokerScope, K> fulfillmentMap = new 
BiMultimap<>();
+    private final Map<RequestScope, RequestState> requestStates = new 
HashMap<>();
+
+
+    public AdminApiDriver(
+        AdminApiHandler<K, V> handler,
+        long deadlineMs,
+        long retryBackoffMs,
+        LogContext logContext
+    ) {
+        this.handler = handler;
+        this.deadlineMs = deadlineMs;
+        this.retryBackoffMs = retryBackoffMs;
+        this.log = logContext.logger(AdminApiDriver.class);
+        this.futures = new HashMap<>();
+
+        KeyMappings<K> result = handler.initializeKeys();
+        this.dynamicMapping = result.dynamicMapping;
+        this.staticMapping = result.staticMapping;
+
+        initializeKeys();
+    }
+
+    private void initializeKeys() {
+        staticMapping.ifPresent(mapping -> {
+            for (Map.Entry<K, Integer> staticMapping : 
mapping.keys.entrySet()) {
+                K key = staticMapping.getKey();
+                Integer brokerId = staticMapping.getValue();
+                futures.put(key, new KafkaFutureImpl<>());
+                map(key, brokerId);
+            }
+        });
+
+        dynamicMapping.ifPresent(mapping -> {
+            for (K key : mapping.keys) {
+                futures.put(key, new KafkaFutureImpl<>());
+                lookupMap.put(mapping.lookupStrategy.lookupScope(key), key);
+            }
+        });
+    }
+
+    /**
+     * Associate a key with a brokerId. This is called after a response in the 
Lookup
+     * stage reveals the mapping (e.g. when the `FindCoordinator` tells us the 
the
+     * group coordinator for a specific consumer group).
+     */
+    private void map(K key, Integer brokerId) {
+        lookupMap.remove(key);
+        fulfillmentMap.put(new BrokerScope(brokerId), key);
+
+        // To allow for derived keys, we create futures dynamically if they
+        // do not already exist in the future map
+        futures.computeIfAbsent(key, k -> new KafkaFutureImpl<>());
+    }
+
+    /**
+     * Disassociate a key from the currently mapped brokerId. This will send 
the key
+     * back to the Lookup stage, which will allow us to attempt lookup again.
+     */
+    private void unmap(K key) {
+        DynamicKeyMapping<K> mapping = dynamicMapping.orElseThrow(() ->
+            new IllegalStateException("Attempt to unmap key " + key + " which 
is not dynamically mapped")
+        );
+        fulfillmentMap.remove(key);
+        lookupMap.put(mapping.lookupStrategy.lookupScope(key), key);
+    }
+
+    private void clear(K key) {
+        lookupMap.remove(key);
+        fulfillmentMap.remove(key);
+    }
+
+    OptionalInt keyToBrokerId(K key) {

Review comment:
       It seems `keyToBrokerId` (and `getKey`) are used by testing only. Is it 
worthwhile to keep a `reverseMap` in production code for testing? 

##########
File path: 
clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java
##########
@@ -0,0 +1,464 @@
+/*
+ * 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 
org.apache.kafka.clients.admin.internals.AdminApiHandler.DynamicKeyMapping;
+import org.apache.kafka.clients.admin.internals.AdminApiHandler.KeyMappings;
+import 
org.apache.kafka.clients.admin.internals.AdminApiHandler.StaticKeyMapping;
+import 
org.apache.kafka.clients.admin.internals.AdminApiLookupStrategy.RequestScope;
+import org.apache.kafka.common.errors.DisconnectException;
+import org.apache.kafka.common.internals.KafkaFutureImpl;
+import org.apache.kafka.common.requests.AbstractRequest;
+import org.apache.kafka.common.requests.AbstractResponse;
+import org.apache.kafka.common.utils.LogContext;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.Set;
+import java.util.function.BiFunction;
+
+/**
+ * The `KafkaAdminClient`'s internal `Call` primitive is not a good fit for 
multi-stage
+ * request workflows such as we see with the group coordinator APIs or any 
request which
+ * needs to be sent to a partition leader. Typically these APIs have two 
concrete stages:
+ *
+ * 1. Lookup: Find the broker that can fulfill the request (e.g. partition 
leader or group
+ *            coordinator)
+ * 2. Fulfillment: Send the request to the broker found in the first step
+ *
+ * This is complicated by the fact that `Admin` APIs are typically batched, 
which
+ * means the Lookup stage may result in a set of brokers. For example, take a 
`ListOffsets`
+ * request for a set of topic partitions. In the Lookup stage, we will find 
the partition
+ * leaders for this set of partitions; in the Fulfillment stage, we will group 
together
+ * partition according to the IDs of the discovered leaders.
+ *
+ * Additionally, the flow between these two stages is bi-directional. We may 
find after
+ * sending a `ListOffsets` request to an expected leader that there was a 
leader change.
+ * This would result in a topic partition being sent back to the Lookup stage.
+ *
+ * Managing this complexity by chaining together `Call` implementations is 
challenging
+ * and messy, so instead we use this class to do the bookkeeping. It handles 
both the
+ * batching aspect as well as the transitions between the Lookup and 
Fulfillment stages.
+ *
+ * Note that the interpretation of the `retries` configuration becomes 
ambiguous
+ * for this kind of pipeline. We could treat it as an overall limit on the 
number
+ * of requests that can be sent, but that is not very useful because each 
pipeline
+ * has a minimum number of requests that need to be sent in order to satisfy 
the request.
+ * Instead, we treat this number of retries independently at each stage so 
that each
+ * stage has at least one opportunity to complete. So if a user sets 
`retries=1`, then
+ * the full pipeline can still complete as long as there are no request 
failures.
+ *
+ * @param <K> The key type, which is also the granularity of the request 
routing (e.g.
+ *            this could be `TopicPartition` in the case of requests intended 
for a partition
+ *            leader or the `GroupId` in the case of consumer group requests 
intended for
+ *            the group coordinator)
+ * @param <V> The fulfillment type for each key (e.g. this could be consumer 
group state
+ *            when the key type is a consumer `GroupId`)
+ */
+public class AdminApiDriver<K, V> {
+    private final Logger log;
+    private final long retryBackoffMs;
+    private final long deadlineMs;
+    private final AdminApiHandler<K, V> handler;
+    private final Optional<StaticKeyMapping<K>> staticMapping;
+    private final Optional<DynamicKeyMapping<K>> dynamicMapping;
+    private final Map<K, KafkaFutureImpl<V>> futures;
+
+    private final BiMultimap<RequestScope, K> lookupMap = new BiMultimap<>();
+    private final BiMultimap<BrokerScope, K> fulfillmentMap = new 
BiMultimap<>();
+    private final Map<RequestScope, RequestState> requestStates = new 
HashMap<>();
+
+
+    public AdminApiDriver(
+        AdminApiHandler<K, V> handler,
+        long deadlineMs,
+        long retryBackoffMs,
+        LogContext logContext
+    ) {
+        this.handler = handler;
+        this.deadlineMs = deadlineMs;
+        this.retryBackoffMs = retryBackoffMs;
+        this.log = logContext.logger(AdminApiDriver.class);
+        this.futures = new HashMap<>();
+
+        KeyMappings<K> result = handler.initializeKeys();
+        this.dynamicMapping = result.dynamicMapping;
+        this.staticMapping = result.staticMapping;
+
+        initializeKeys();
+    }
+
+    private void initializeKeys() {
+        staticMapping.ifPresent(mapping -> {
+            for (Map.Entry<K, Integer> staticMapping : 
mapping.keys.entrySet()) {
+                K key = staticMapping.getKey();
+                Integer brokerId = staticMapping.getValue();
+                futures.put(key, new KafkaFutureImpl<>());
+                map(key, brokerId);
+            }
+        });
+
+        dynamicMapping.ifPresent(mapping -> {
+            for (K key : mapping.keys) {
+                futures.put(key, new KafkaFutureImpl<>());
+                lookupMap.put(mapping.lookupStrategy.lookupScope(key), key);
+            }
+        });
+    }
+
+    /**
+     * Associate a key with a brokerId. This is called after a response in the 
Lookup
+     * stage reveals the mapping (e.g. when the `FindCoordinator` tells us the 
the
+     * group coordinator for a specific consumer group).
+     */
+    private void map(K key, Integer brokerId) {
+        lookupMap.remove(key);
+        fulfillmentMap.put(new BrokerScope(brokerId), key);
+
+        // To allow for derived keys, we create futures dynamically if they
+        // do not already exist in the future map
+        futures.computeIfAbsent(key, k -> new KafkaFutureImpl<>());
+    }
+
+    /**
+     * Disassociate a key from the currently mapped brokerId. This will send 
the key
+     * back to the Lookup stage, which will allow us to attempt lookup again.
+     */
+    private void unmap(K key) {
+        DynamicKeyMapping<K> mapping = dynamicMapping.orElseThrow(() ->
+            new IllegalStateException("Attempt to unmap key " + key + " which 
is not dynamically mapped")
+        );
+        fulfillmentMap.remove(key);
+        lookupMap.put(mapping.lookupStrategy.lookupScope(key), key);
+    }
+
+    private void clear(K key) {
+        lookupMap.remove(key);
+        fulfillmentMap.remove(key);
+    }
+
+    OptionalInt keyToBrokerId(K key) {
+        Optional<BrokerScope> scope = fulfillmentMap.getKey(key);
+        if (scope.isPresent()) {
+            return OptionalInt.of(scope.get().destinationBrokerId);
+        } else {
+            return OptionalInt.empty();
+        }
+    }
+
+    /**
+     * Complete the future associated with the given key exceptionally. After 
is called,
+     * the key will be taken out of both the Lookup and Fulfillment stages so 
that request
+     * are not retried.
+     */
+    private void completeExceptionally(K key, Throwable t) {
+        KafkaFutureImpl<V> future = futures.get(key);
+        if (future == null) {
+            log.warn("Attempt to complete future for {}, which was not 
requested", key);
+        } else {
+            clear(key);
+            future.completeExceptionally(t);
+        }
+    }
+
+    /**
+     * Complete the future associated with the given key. After is called, the 
key will
+     * be taken out of both the Lookup and Fulfillment stages so that request 
are not retried.
+     */
+    private void complete(K key, V value) {
+        KafkaFutureImpl<V> future = futures.get(key);
+        if (future == null) {
+            log.warn("Attempt to complete future for {}, which was not 
requested", key);
+        } else {
+            clear(key);
+            future.complete(value);
+        }
+    }
+
+    /**
+     * Check whether any requests need to be sent. This should be called 
immediately
+     * after the driver is constructed and then again after each request 
returns
+     * (i.e. after {@link #onFailure(long, RequestSpec, Throwable)} or
+     * {@link #onResponse(long, RequestSpec, AbstractResponse)}).
+     *
+     * @return A list of requests that need to be sent
+     */
+    public List<RequestSpec<K>> poll() {
+        List<RequestSpec<K>> requests = new ArrayList<>();
+        collectLookupRequests(requests);
+        collectFulfillmentRequests(requests);
+        return requests;
+    }
+
+    /**
+     * Get a map of the futures that are awaiting completion.
+     */
+    public Map<K, KafkaFutureImpl<V>> futures() {
+        return futures;
+    }
+
+    /**
+     * Callback that is invoked when a `Call` returns a response successfully.
+     */
+    public void onResponse(
+        long currentTimeMs,
+        RequestSpec<K> spec,
+        AbstractResponse response
+    ) {
+        clearInflightRequest(currentTimeMs, spec);
+        if (spec.scope instanceof AdminApiDriver.BrokerScope) {
+            int brokerId = ((BrokerScope) spec.scope).destinationBrokerId;
+            AdminApiHandler.ApiResult<K, V> result = handler.handleResponse(
+                brokerId,
+                spec.keys,
+                response
+            );
+            result.completedKeys.forEach(this::complete);
+            result.failedKeys.forEach(this::completeExceptionally);
+            result.unmappedKeys.forEach(this::unmap);
+        } else {
+            DynamicKeyMapping<K> mapping = dynamicMapping.orElseThrow(() ->
+                new IllegalStateException("Received response for unexpected 
dynamic lookup scope " + spec.scope)
+            );
+
+            AdminApiLookupStrategy.LookupResult<K> result = 
mapping.lookupStrategy.handleResponse(
+                spec.keys,
+                response
+            );
+            result.failedKeys.forEach(this::completeExceptionally);
+            result.mappedKeys.forEach(this::map);
+        }
+    }
+
+    /**
+     * Callback that is invoked when a `Call` is failed.
+     */
+    public void onFailure(
+        long currentTimeMs,
+        RequestSpec<K> spec,
+        Throwable t
+    ) {
+        clearInflightRequest(currentTimeMs, spec);
+        if (t instanceof DisconnectException) {
+            log.debug("Node disconnected before response could be received for 
request {}. " +
+                "Will attempt retry", spec.request);
+            dynamicMapping.ifPresent(mapping -> {
+                if (mapping.keys.containsAll(spec.keys)) {
+                    spec.keys.forEach(this::unmap);
+                }
+            });
+        } else {
+            spec.keys.forEach(key -> completeExceptionally(key, t));
+        }
+    }
+
+    private void clearInflightRequest(long currentTimeMs, RequestSpec<K> spec) 
{
+        RequestState requestState = requestStates.get(spec.scope);
+        if (requestState != null) {
+            requestState.clearInflight(currentTimeMs);
+        }
+    }
+
+    private <T extends RequestScope> void collectRequests(
+        List<RequestSpec<K>> requests,
+        BiMultimap<T, K> multimap,
+        BiFunction<Set<K>, T, AbstractRequest.Builder<?>> buildRequest
+    ) {
+        for (Map.Entry<T, Set<K>> entry : multimap.entrySet()) {
+            T scope = entry.getKey();
+
+            Set<K> keys = entry.getValue();
+            if (keys.isEmpty()) {
+                continue;
+            }
+
+            RequestState requestState = requestStates.computeIfAbsent(scope, c 
-> new RequestState());
+            if (requestState.hasInflight()) {
+                continue;
+            }
+
+            AbstractRequest.Builder<?> request = buildRequest.apply(keys, 
scope);
+            RequestSpec<K> spec = new RequestSpec<>(
+                handler.apiName() + "(api=" + request.apiKey() + ")",
+                scope,
+                new HashSet<>(keys), // copy to avoid exposing mutable state
+                request,
+                requestState.nextAllowedRetryMs,
+                deadlineMs,
+                requestState.tries
+            );
+
+            requestState.setInflight(spec);
+            requests.add(spec);
+        }
+    }
+
+    private void collectLookupRequests(List<RequestSpec<K>> requests) {
+        dynamicMapping.ifPresent(mapping -> {
+            collectRequests(
+                requests,
+                lookupMap,
+                (keys, scope) -> mapping.lookupStrategy.buildRequest(keys)
+            );
+        });
+    }
+
+    private void collectFulfillmentRequests(List<RequestSpec<K>> requests) {
+        collectRequests(
+            requests,
+            fulfillmentMap,
+            (keys, scope) -> handler.buildRequest(scope.destinationBrokerId, 
keys)
+        );
+    }
+
+    /**
+     * This is a helper class which helps us to map requests that need to be 
sent
+     * to the internal `Call` implementation that is used internally in
+     * {@link org.apache.kafka.clients.admin.KafkaAdminClient}.
+     */
+    public static class RequestSpec<K> {
+        public final String name;
+        public final RequestScope scope;
+        public final Set<K> keys;
+        public final AbstractRequest.Builder<?> request;
+        public final long nextAllowedTryMs;
+        public final long deadlineMs;
+        public final int tries;
+
+        public RequestSpec(
+            String name,
+            RequestScope scope,
+            Set<K> keys,
+            AbstractRequest.Builder<?> request,
+            long nextAllowedTryMs,
+            long deadlineMs,
+            int tries
+        ) {
+            this.name = name;
+            this.scope = scope;
+            this.keys = keys;
+            this.request = request;
+            this.nextAllowedTryMs = nextAllowedTryMs;
+            this.deadlineMs = deadlineMs;
+            this.tries = tries;
+        }
+    }
+
+    /**
+     * Helper class used to track the request state within each request scope.
+     * This class enforces a maximum number of inflight request and keeps track
+     * of backoff/retry state.
+     */
+    private class RequestState {
+        private Optional<RequestSpec<K>> inflightRequest = Optional.empty();
+        private int tries = 0;
+        private long nextAllowedRetryMs = 0;
+
+        boolean hasInflight() {
+            return inflightRequest.isPresent();
+        }
+
+        public void clearInflight(long currentTimeMs) {
+            this.inflightRequest = Optional.empty();
+            this.nextAllowedRetryMs = currentTimeMs + retryBackoffMs;
+        }
+
+        public void setInflight(RequestSpec<K> spec) {
+            this.inflightRequest = Optional.of(spec);
+            this.tries++;
+        }
+    }
+
+    /**
+     * Completion of the Lookup stage results in a destination broker to send 
the
+     * fulfillment request to. Each destination broker in the Fulfillment stage
+     * gets its own request scope.
+     */
+    private static class BrokerScope implements RequestScope {
+        public final int destinationBrokerId;
+
+        private BrokerScope(int destinationBrokerId) {
+            this.destinationBrokerId = destinationBrokerId;
+        }
+
+        @Override
+        public OptionalInt destinationBrokerId() {
+            return OptionalInt.of(destinationBrokerId);
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            BrokerScope that = (BrokerScope) o;
+            return destinationBrokerId == that.destinationBrokerId;
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(destinationBrokerId);
+        }
+    }
+
+    /**
+     * Helper class which maintains a bi-directional mapping from a key to a 
set of values.
+     * Each value can map to one and only one key, but many values can be 
associated with
+     * a single key.
+     *
+     * @param <K> The key type
+     * @param <V> The value type
+     */
+    private static class BiMultimap<K, V> {
+        private final Map<V, K> reverseMap = new HashMap<>();
+        private final Map<K, Set<V>> map = new HashMap<>();
+
+        void put(K key, V value) {
+            remove(value);
+            reverseMap.put(value, key);
+            map.computeIfAbsent(key, k -> new HashSet<>()).add(value);
+        }
+
+        void remove(V value) {
+            K key = reverseMap.remove(value);
+            if (key != null) {
+                Set<V> set = map.get(key);
+                if (set != null) {
+                    set.remove(value);
+                    if (set.isEmpty()) {
+                        map.remove(key);
+                    }
+                }
+            }
+        }
+
+        Optional<K> getKey(V value) {
+            return Optional.ofNullable(reverseMap.get(value));

Review comment:
       Is this method safe? If there are same values, it would return unrelated 
key.

##########
File path: 
clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiHandler.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 org.apache.kafka.common.requests.AbstractRequest;
+import org.apache.kafka.common.requests.AbstractResponse;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+public interface AdminApiHandler<K, V> {
+
+    /**
+     * Get a user-friendly name for the API this handler is implementing.
+     */
+    String apiName();
+
+    /**
+     * Initialize the set of keys required to handle this API and how the 
driver
+     * should map them to the broker that will handle the request for these 
keys.
+     *
+     * Two mapping types are supported:
+     *
+     * - Static mapping: when the brokerId is known ahead of time
+     * - Dynamic mapping: when the brokerId must be discovered dynamically
+     *
+     * @return the key mappings
+     */
+    KeyMappings<K> initializeKeys();
+
+    /**
+     * Build the fulfillment request. The set of keys are derived during the 
Lookup stage
+     * as the set of keys which all map to the same destination broker.
+     *
+     * @param brokerId the target brokerId for the request
+     * @param keys the set of keys that should be handled by this request
+     *
+     * @return a builder for the request containing the given keys
+     */
+    AbstractRequest.Builder<?> buildRequest(Integer brokerId, Set<K> keys);
+
+    /**
+     * Callback that is invoked when a Fulfillment request returns 
successfully.
+     * The handler should parse the response, check for errors, and return a
+     * result which indicates which keys (if any) have either been completed or
+     * failed with an unrecoverable error.
+     *
+     * It is also possible that the response indicates an incorrect target 
brokerId
+     * (e.g. in the case of a NotLeader error when the request is bound for a 
partition
+     * leader). In this case the key will be "unmapped" from the target 
brokerId
+     * and lookup will be retried.
+     *
+     * Note that keys which received a retriable error should be left out of 
the
+     * result. They will be retried automatically.
+     *
+     * @param brokerId the brokerId that the associated request was sent to
+     * @param keys the set of keys from the associated request
+     * @param response the response received from the broker
+     *
+     * @return result indicating key complation, failure, and unmapping
+     */
+    ApiResult<K, V> handleResponse(Integer brokerId, Set<K> keys, 
AbstractResponse response);
+
+    class KeyMappings<K> {
+        public final Optional<StaticKeyMapping<K>> staticMapping;
+        public final Optional<DynamicKeyMapping<K>> dynamicMapping;
+
+        public KeyMappings(
+            Optional<StaticKeyMapping<K>> staticMapping,
+            Optional<DynamicKeyMapping<K>> dynamicMapping
+        ) {
+            this.staticMapping = staticMapping;
+            this.dynamicMapping = dynamicMapping;
+        }
+    }
+
+    class StaticKeyMapping<K> {
+        public final Map<K, Integer> keys;
+
+        public StaticKeyMapping(Map<K, Integer> keys) {
+            this.keys = keys;

Review comment:
       Why this field is not wrapped to unmodified stuff?

##########
File path: 
clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiLookupStrategy.java
##########
@@ -0,0 +1,94 @@
+/*
+ * 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 org.apache.kafka.common.requests.AbstractRequest;
+import org.apache.kafka.common.requests.AbstractResponse;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.OptionalInt;
+import java.util.Set;
+
+public interface AdminApiLookupStrategy<T> {
+
+    /**
+     * Define the scope of a given key for lookup. Key lookups are complicated
+     * by the need to accommodate different batching mechanics. For example,
+     * a `Metadata` request supports arbitrary batching of topic partitions in
+     * order to discover partitions leaders. This can be supported by returning
+     * a single scope object for all keys.
+     *
+     * On the other hand, `FindCoordinator` request only supports lookup of a
+     * single key. This can be supported by returning a different scope object
+     * for each lookup key.
+     *
+     * @param key the lookup key
+     *
+     * @return request scope indicating how lookup requests can be batched 
together
+     */
+    RequestScope lookupScope(T key);
+
+    /**
+     * Build the lookup request for a set of keys. The grouping of the keys is 
controlled
+     * through {@link #lookupScope(Object)}. In other words, each set of keys 
that map
+     * to the same request scope object will be sent to this method.
+     *
+     * @param keys the set of keys that require lookup
+     *
+     * @return a builder for the lookup request
+     */
+    AbstractRequest.Builder<?> buildRequest(Set<T> keys);
+
+    /**
+     * Callback that is invoked when a lookup request returns successfully. 
The handler
+     * should parse the response, check for errors, and return a result 
indicating
+     * which keys were mapped to a brokerId successfully and which keys 
received
+     * a fatal error (e.g. a topic authorization failure).
+     *
+     * Note that keys which receive a retriable error should be left out of the
+     * result. They will be retried automatically. For example, if the 
response of
+     * `FindCoordinator` request indicates an unavailable coordinator, then 
the key
+     * should be left out of the result so that the request will be retried.
+     *
+     * @param keys the set of keys from the associated request
+     * @param response the response received from the broker
+     *
+     * @return a result indicating which keys mapped successfully to a 
brokerId and
+     *         which encountered a fatal error
+     */
+    LookupResult<T> handleResponse(Set<T> keys, AbstractResponse response);
+
+    class LookupResult<K> {
+        public final Map<K, Integer> mappedKeys;
+        public final Map<K, Throwable> failedKeys;
+
+        public LookupResult(
+            Map<K, Throwable> failedKeys,
+            Map<K, Integer> mappedKeys
+        ) {
+            this.failedKeys = Collections.unmodifiableMap(failedKeys);
+            this.mappedKeys = Collections.unmodifiableMap(mappedKeys);
+        }
+    }
+
+    interface RequestScope {
+        default OptionalInt destinationBrokerId() {
+            return OptionalInt.empty();
+        };

Review comment:
       unnecessary `;`

##########
File path: 
clients/src/main/java/org/apache/kafka/clients/admin/DescribeProducersOptions.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.annotation.InterfaceStability;
+
+import java.util.Collection;
+import java.util.Objects;
+import java.util.OptionalInt;
+
+/**
+ * Options for {@link Admin#describeProducers(Collection)}.
+ *
+ * The API of this class is evolving, see {@link Admin} for details.
+ */
+@InterfaceStability.Evolving
+public class DescribeProducersOptions extends 
AbstractOptions<DescribeProducersOptions> {
+    private OptionalInt brokerId = OptionalInt.empty();
+
+    public DescribeProducersOptions setBrokerId(int brokerId) {
+        this.brokerId = OptionalInt.of(brokerId);
+        return this;
+    }
+
+    public OptionalInt brokerId() {
+        return brokerId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        DescribeProducersOptions that = (DescribeProducersOptions) o;
+        return Objects.equals(brokerId, that.brokerId);

Review comment:
       Should it compare `timeoutMs`?

##########
File path: 
clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiHandler.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 org.apache.kafka.common.requests.AbstractRequest;
+import org.apache.kafka.common.requests.AbstractResponse;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+public interface AdminApiHandler<K, V> {
+
+    /**
+     * Get a user-friendly name for the API this handler is implementing.
+     */
+    String apiName();
+
+    /**
+     * Initialize the set of keys required to handle this API and how the 
driver
+     * should map them to the broker that will handle the request for these 
keys.
+     *
+     * Two mapping types are supported:
+     *
+     * - Static mapping: when the brokerId is known ahead of time
+     * - Dynamic mapping: when the brokerId must be discovered dynamically
+     *
+     * @return the key mappings
+     */
+    KeyMappings<K> initializeKeys();
+
+    /**
+     * Build the fulfillment request. The set of keys are derived during the 
Lookup stage
+     * as the set of keys which all map to the same destination broker.
+     *
+     * @param brokerId the target brokerId for the request
+     * @param keys the set of keys that should be handled by this request
+     *
+     * @return a builder for the request containing the given keys
+     */
+    AbstractRequest.Builder<?> buildRequest(Integer brokerId, Set<K> keys);
+
+    /**
+     * Callback that is invoked when a Fulfillment request returns 
successfully.
+     * The handler should parse the response, check for errors, and return a
+     * result which indicates which keys (if any) have either been completed or
+     * failed with an unrecoverable error.
+     *
+     * It is also possible that the response indicates an incorrect target 
brokerId
+     * (e.g. in the case of a NotLeader error when the request is bound for a 
partition
+     * leader). In this case the key will be "unmapped" from the target 
brokerId
+     * and lookup will be retried.
+     *
+     * Note that keys which received a retriable error should be left out of 
the
+     * result. They will be retried automatically.
+     *
+     * @param brokerId the brokerId that the associated request was sent to
+     * @param keys the set of keys from the associated request
+     * @param response the response received from the broker
+     *
+     * @return result indicating key complation, failure, and unmapping
+     */
+    ApiResult<K, V> handleResponse(Integer brokerId, Set<K> keys, 
AbstractResponse response);
+
+    class KeyMappings<K> {
+        public final Optional<StaticKeyMapping<K>> staticMapping;
+        public final Optional<DynamicKeyMapping<K>> dynamicMapping;
+
+        public KeyMappings(
+            Optional<StaticKeyMapping<K>> staticMapping,
+            Optional<DynamicKeyMapping<K>> dynamicMapping
+        ) {
+            this.staticMapping = staticMapping;
+            this.dynamicMapping = dynamicMapping;
+        }
+    }
+
+    class StaticKeyMapping<K> {
+        public final Map<K, Integer> keys;
+
+        public StaticKeyMapping(Map<K, Integer> keys) {
+            this.keys = keys;
+        }
+    }
+
+    class DynamicKeyMapping<K> {
+        public final Set<K> keys;
+        public final AdminApiLookupStrategy<K> lookupStrategy;
+
+        public DynamicKeyMapping(Set<K> keys, AdminApiLookupStrategy<K> 
lookupStrategy) {
+            this.keys = keys;

Review comment:
       wrap it to unmodified collection?




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

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


Reply via email to