[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r598954950 ## File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java ## @@ -0,0 +1,461 @@ +/* + * 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.Keys; +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 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 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 { +private final Logger log; +private final long retryBackoffMs; +private final long deadlineMs; +private final AdminApiHandler handler; +private final Keys keys; +private final Map> futures; + +private final BiMultimap lookupMap = new BiMultimap<>(); +private final BiMultimap fulfillmentMap = new BiMultimap<>(); +private final Map requestStates = new HashMap<>(); + +public AdminApiDriver( +AdminApiHandler handler, +long deadlineMs, +long retryBackoffMs, +LogContext logContext +) { +this.handler = handler; +this.deadlineMs = deadlineMs; +this.ret
[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r597978853 ## 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 { + +/** + * 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 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 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 handleResponse(Integer brokerId, Set keys, AbstractResponse response); + +class KeyMappings { +public final Optional> staticMapping; Review comment: That's fair. I think I added the separate types because i wanted to express the fact that the `LookupStrategy` was only required for dynamic keys. Maybe that is overkill though and we could do the validation dynamically. -- 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
[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r597946534 ## File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java ## @@ -0,0 +1,473 @@ +/* + * 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.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 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 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 { +private final Logger log; +private final long retryBackoffMs; +private final long deadlineMs; +private final AdminApiHandler handler; +private final Optional> staticMapping; +private final Optional> dynamicMapping; +private final Map> futures; + +private final BiMultimap lookupMap = new BiMultimap<>(); +private final BiMultimap fulfillmentMap = new BiMultimap<>(); +private final Map requestStates = new HashMap<>(); + +public
[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r597193539 ## 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 { + +/** + * 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 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 handleResponse(Set keys, AbstractResponse response); + +class LookupResult { +public final Map mappedKeys; +public final Map failedKeys; + +public LookupResult( +Map failedKeys, +Map mappedKeys +) { +this.failedKeys = Collections.unmodifiableMap(failedKeys); +this.mappedKeys = Collections.unmodifiableMap(mappedKeys); +} +} + +interface RequestScope { +default OptionalInt destinationBrokerId() { +return OptionalInt.empty(); +} +} Review comment: I decided to pull this up to the top level. Let me know if that seems ok. -- 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
[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r597064897 ## 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 { + +/** + * 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 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 handleResponse(Set keys, AbstractResponse response); + +class LookupResult { +public final Map mappedKeys; +public final Map failedKeys; + +public LookupResult( +Map failedKeys, +Map mappedKeys +) { +this.failedKeys = Collections.unmodifiableMap(failedKeys); +this.mappedKeys = Collections.unmodifiableMap(mappedKeys); +} +} + +interface RequestScope { +default OptionalInt destinationBrokerId() { +return OptionalInt.empty(); +} +} Review comment: Let me think about it. I wasn't entirely happy with this location either. -- 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
[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r597061628 ## 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 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 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 { +private final Logger log; +private final long retryBackoffMs; +private final long deadlineMs; +private final AdminApiHandler handler; +private final Optional> staticMapping; +private final Optional> dynamicMapping; +private final Map> futures; + +private final BiMultimap lookupMap = new BiMultimap<>(); +private final BiMultimap fulfillmentMap = new B
[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r596289179 ## File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java ## @@ -0,0 +1,476 @@ +/* + * 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 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 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 { +private final Logger log; +private final long retryBackoffMs; +private final long deadlineMs; +private final AdminApiHandler handler; +private final Optional> staticMapping; +private final Optional> dynamicMapping; +private final Map> futures; + +private final BiMultimap lookupMap = new BiMultimap<>(); +private final BiMultimap fulfillmentMap = new B
[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r591903496 ## File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/PartitionLeaderStrategy.java ## @@ -0,0 +1,185 @@ +/* + * 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.TopicPartition; +import org.apache.kafka.common.errors.InvalidTopicException; +import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.common.message.MetadataRequestData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.utils.LogContext; +import org.slf4j.Logger; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Base driver implementation for APIs which target partition leaders. + */ +public class PartitionLeaderStrategy implements AdminApiLookupStrategy { +private static final RequestScope SINGLE_REQUEST_SCOPE = new RequestScope() { +}; + +private final Logger log; + +public PartitionLeaderStrategy(LogContext logContext) { +this.log = logContext.logger(PartitionLeaderStrategy.class); +} + +@Override +public RequestScope lookupScope(TopicPartition key) { +// Metadata requests can group topic partitions arbitrarily, so they can all share +// the same request context +return SINGLE_REQUEST_SCOPE; +} + +@Override +public MetadataRequest.Builder buildRequest(Set partitions) { +MetadataRequestData request = new MetadataRequestData(); +request.setAllowAutoTopicCreation(false); +Set topics = partitions.stream().map(TopicPartition::topic).collect(Collectors.toSet()); +for (String topic : topics) { +request.topics().add(new MetadataRequestData.MetadataRequestTopic().setName(topic)); +} +return new MetadataRequest.Builder(request); +} + +private void handleTopicError( +String topic, +Errors topicError, +Set requestPartitions, +Map failed +) { +switch (topicError) { +case UNKNOWN_TOPIC_OR_PARTITION: +case LEADER_NOT_AVAILABLE: +case BROKER_NOT_AVAILABLE: +log.debug("Metadata request for topic {} returned topic-level error {}. Will retry", +topic, topicError); +break; + +case TOPIC_AUTHORIZATION_FAILED: +log.error("Received authorization failure for topic {} in `Metadata` response", topic, +topicError.exception()); +failAllPartitionsForTopic(topic, requestPartitions, failed, tp -> new TopicAuthorizationException( +"Failed to fetch metadata for partition " + tp + " due to topic authorization failure", +Collections.singleton(topic))); +break; + +case INVALID_TOPIC_EXCEPTION: +log.error("Received invalid topic error for topic {} in `Metadata` response", topic, +topicError.exception()); +failAllPartitionsForTopic(topic, requestPartitions, failed, tp -> new InvalidTopicException( +"Failed to fetch metadata for partition " + tp + " due to invalid topic `" + topic + "`", +Collections.singleton(topic))); +break; + +default: +log.error("Received unexpected error for topic {} in `Metadata` response", topic, +topicError.exception()); +failAllPartitionsForTopic(topic, requestPartitions, failed, tp -> topicError.exception( +"Failed to fetch metadata for partition " + tp + " due to unexpected error for topic `" + topic + "`")); +} +} + +private void failAllPa
[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r590942924 ## 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 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 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 { +private final Logger log; +private final long retryBackoffMs; +private final long deadlineMs; +private final AdminApiHandler handler; +private final Optional> staticMapping; +private final Optional> dynamicMapping; +private final Map> futures; + +private final BiMultimap lookupMap = new BiMultimap<>(); +private final BiMultimap fulfillmentMap = new B
[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r590941485 ## File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersHandler.java ## @@ -0,0 +1,213 @@ +/* + * 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 { +private final LogContext logContext; +private final Logger log; +private final DescribeProducersOptions options; +private final Set topicPartitions; + +public DescribeProducersHandler( +Set 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 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 staticMappedPartitions = +Utils.initializeMap(topicPartitions, () -> destinationBrokerId); +return new KeyMappings<>( +Optional.of(new StaticKeyMapping<>(staticMappedPartitions)), +Optional.empty() +); +} else { +PartitionLeaderStrategy lookupStrategy = +new PartitionLeaderStrategy(topicPartitions, logContext); +return new KeyMappings<>( +Optional.empty(), +Optional.of(new DynamicKeyMapping<>(topicPartitions, lookupStrategy)) +); +} +} + +@Override +public DescribeProducersRequest.Builder buildRequest( +Integer brokerId, +Set 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 failed, +List unmapped +) { +switch (apiError.error()) { +case NOT_LEADER_OR_FOLLOWER: +if (options.brokerId().isPresent()) { +// Typically these errors are retriable, but if the user sp
[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r590938521 ## File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/PartitionLeaderStrategy.java ## @@ -0,0 +1,190 @@ +/* + * 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.TopicPartition; +import org.apache.kafka.common.errors.InvalidTopicException; +import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.common.message.MetadataRequestData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.utils.LogContext; +import org.slf4j.Logger; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Base driver implementation for APIs which target partition leaders. + */ +public class PartitionLeaderStrategy implements AdminApiLookupStrategy { +private static final RequestScope SINGLE_REQUEST_SCOPE = new RequestScope() { +}; + +protected final Set topicPartitions; +private final Logger log; + +public PartitionLeaderStrategy( +Set topicPartitions, +LogContext logContext +) { +this.topicPartitions = Collections.unmodifiableSet(topicPartitions); +this.log = logContext.logger(PartitionLeaderStrategy.class); +} + +@Override +public RequestScope lookupScope(TopicPartition key) { +// Metadata requests can group topic partitions arbitrarily, so they can all share +// the same request context +return SINGLE_REQUEST_SCOPE; +} + +@Override +public MetadataRequest.Builder buildRequest(Set partitions) { +MetadataRequestData request = new MetadataRequestData(); +request.setAllowAutoTopicCreation(false); +Set topics = partitions.stream().map(TopicPartition::topic).collect(Collectors.toSet()); +for (String topic : topics) { +request.topics().add(new MetadataRequestData.MetadataRequestTopic().setName(topic)); +} +return new MetadataRequest.Builder(request); +} + +private void handleTopicError( +String topic, +Errors topicError, +Set requestPartitions, +Map failed +) { +switch (topicError) { +case UNKNOWN_TOPIC_OR_PARTITION: +case LEADER_NOT_AVAILABLE: +case BROKER_NOT_AVAILABLE: +log.debug("Metadata request for topic {} returned topic-level error {}. Will retry", +topic, topicError); +break; + +case TOPIC_AUTHORIZATION_FAILED: +log.error("Received authorization failure for topic {} in `Metadata` response", topic, +topicError.exception()); +failAllPartitionsForTopic(topic, requestPartitions, failed, tp -> new TopicAuthorizationException( +"Failed to fetch metadata for partition " + tp + " due to topic authorization failure", +Collections.singleton(topic))); +break; + +case INVALID_TOPIC_EXCEPTION: +log.error("Received invalid topic error for topic {} in `Metadata` response", topic, +topicError.exception()); +failAllPartitionsForTopic(topic, requestPartitions, failed, tp -> new InvalidTopicException( +"Failed to fetch metadata for partition " + tp + " due to invalid topic `" + topic + "`", +Collections.singleton(topic))); +break; + +default: +log.error("Received unexpected error for topic {} in `Metadata` response", topic, +topicError.exception()); +failAllPartitionsForTopic(topic, requestPartitions, failed, tp -> topicError.exception( +
[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r590934574 ## 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 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 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 { +private final Logger log; +private final long retryBackoffMs; +private final long deadlineMs; +private final AdminApiHandler handler; +private final Optional> staticMapping; +private final Optional> dynamicMapping; +private final Map> futures; + +private final BiMultimap lookupMap = new BiMultimap<>(); +private final BiMultimap fulfillmentMap = new B
[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r590928976 ## 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 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 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 { +private final Logger log; +private final long retryBackoffMs; +private final long deadlineMs; +private final AdminApiHandler handler; +private final Optional> staticMapping; +private final Optional> dynamicMapping; +private final Map> futures; + +private final BiMultimap lookupMap = new BiMultimap<>(); +private final BiMultimap fulfillmentMap = new B
[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r590928976 ## 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 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 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 { +private final Logger log; +private final long retryBackoffMs; +private final long deadlineMs; +private final AdminApiHandler handler; +private final Optional> staticMapping; +private final Optional> dynamicMapping; +private final Map> futures; + +private final BiMultimap lookupMap = new BiMultimap<>(); +private final BiMultimap fulfillmentMap = new B
[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r590038851 ## 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> futures; + +DescribeProducersResult(Map> futures) { +this.futures = futures; +} + +public KafkaFuture partitionResult(final TopicPartition partition) { +KafkaFuture future = futures.get(partition); +if (future == null) { +throw new IllegalArgumentException("Topic partition " + partition + +" was not included in the request"); +} +return future; +} + +public KafkaFuture> all() { +return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])) +.thenApply(nil -> { +Map results = new HashMap<>(futures.size()); +for (Map.Entry> 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 activeProducers; + +public PartitionProducerState(List activeProducers) { Review comment: I need it public because the handler is in `admin.internals`. This might be a pattern that has to give way if we want to externalize some of the admin logic rather than packing it all into `KafkaAdminClient`. I'll try to consider some other options. I am not sure I would call it "useless" by the way. It is helpful when you need to mock the admin client. Of course users can always mock `PartitionProducerState` as well, though it gets tedious for property classes. 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
[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r590027837 ## 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 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 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 { +private final Logger log; +private final long retryBackoffMs; +private final long deadlineMs; +private final AdminApiHandler handler; +private final Optional> staticMapping; +private final Optional> dynamicMapping; +private final Map> futures; + +private final BiMultimap lookupMap = new BiMultimap<>(); +private final BiMultimap fulfillmentMap = new B
[GitHub] [kafka] hachikuji commented on a change in pull request #10275: KAFKA-12434; Admin support for `DescribeProducers` API
hachikuji commented on a change in pull request #10275: URL: https://github.com/apache/kafka/pull/10275#discussion_r590025229 ## 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 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 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 { +private final Logger log; +private final long retryBackoffMs; +private final long deadlineMs; +private final AdminApiHandler handler; +private final Optional> staticMapping; +private final Optional> dynamicMapping; +private final Map> futures; + +private final BiMultimap lookupMap = new BiMultimap<>(); +private final BiMultimap fulfillmentMap = new B