dajac commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r596869352
########## 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: Have you decided to put this interface here because it is mainly used by the `AdminApiLookupStrategy`? Intuitively, I would have placed it in the driver because it is used by few other classes as well. ########## File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java ########## @@ -0,0 +1,474 @@ +/* + * 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 -> { + mapping.keys.forEach((key, brokerId) -> { + 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); + } + + /** + * 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) { Review comment: I suggest to add a comment explaining why it is safe to rely on `AdminApiDriver.BrokerScope` to dispatch. At first, I did not understand this until I realized that `AdminApiDriver.BrokerScope` is actually private and can be instantiated only by the `AdminApiDriver`. Do I get this right? ########## File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersHandler.java ########## @@ -0,0 +1,212 @@ +/* + * 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.DescribeProducersOptions; +import org.apache.kafka.clients.admin.DescribeProducersResult.PartitionProducerState; +import org.apache.kafka.clients.admin.ProducerState; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.InvalidTopicException; +import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.common.message.DescribeProducersRequestData; +import org.apache.kafka.common.message.DescribeProducersResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.ApiError; +import org.apache.kafka.common.requests.DescribeProducersRequest; +import org.apache.kafka.common.requests.DescribeProducersResponse; +import org.apache.kafka.common.utils.CollectionUtils; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.OptionalLong; +import java.util.Set; +import java.util.stream.Collectors; + +public class DescribeProducersHandler implements AdminApiHandler<TopicPartition, PartitionProducerState> { + private final LogContext logContext; + private final Logger log; + private final DescribeProducersOptions options; + private final Set<TopicPartition> topicPartitions; + + public DescribeProducersHandler( + Set<TopicPartition> topicPartitions, + DescribeProducersOptions options, + LogContext logContext + ) { + this.topicPartitions = Collections.unmodifiableSet(topicPartitions); + this.options = options; + this.log = logContext.logger(DescribeProducersHandler.class); + this.logContext = logContext; + } + + @Override + public String apiName() { + return "describeProducers"; + } + + @Override + public KeyMappings<TopicPartition> initializeKeys() { + if (options.brokerId().isPresent()) { + // If the options indicate a specific broker, then we can skip the lookup step + int destinationBrokerId = options.brokerId().getAsInt(); + Map<TopicPartition, Integer> staticMappedPartitions = + Utils.initializeMap(topicPartitions, () -> destinationBrokerId); + return new KeyMappings<>( + Optional.of(new StaticKeyMapping<>(staticMappedPartitions)), + Optional.empty() + ); + } else { + PartitionLeaderStrategy lookupStrategy = new PartitionLeaderStrategy(logContext); + return new KeyMappings<>( + Optional.empty(), + Optional.of(new DynamicKeyMapping<>(topicPartitions, lookupStrategy)) + ); + } + } + + @Override + public DescribeProducersRequest.Builder buildRequest( + Integer brokerId, + Set<TopicPartition> topicPartitions + ) { + DescribeProducersRequestData request = new DescribeProducersRequestData(); + DescribeProducersRequest.Builder builder = new DescribeProducersRequest.Builder(request); + + CollectionUtils.groupPartitionsByTopic( + topicPartitions, + builder::addTopic, + (topicRequest, partitionId) -> topicRequest.partitionIndexes().add(partitionId) + ); + + return builder; + } + + private void handlePartitionError( + TopicPartition topicPartition, + ApiError apiError, + Map<TopicPartition, Throwable> failed, + List<TopicPartition> unmapped + ) { + switch (apiError.error()) { + case NOT_LEADER_OR_FOLLOWER: + if (options.brokerId().isPresent()) { + // Typically these errors are retriable, but if the user specified the brokerId + // explicitly, then they are fatal. + int brokerId = options.brokerId().getAsInt(); + log.error("Not leader error in `DescribeProducers` response for partition {} " + + "for brokerId {} set in options", topicPartition, brokerId, apiError.exception()); Review comment: nit: Indentation of this line seems odd in comparison to the others. ---------------------------------------------------------------- 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