dajac commented on a change in pull request #10743: URL: https://github.com/apache/kafka/pull/10743#discussion_r659603553
########## File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategy.java ########## @@ -17,84 +17,158 @@ package org.apache.kafka.clients.admin.internals; import org.apache.kafka.common.errors.GroupAuthorizationException; +import org.apache.kafka.common.errors.InvalidGroupIdException; import org.apache.kafka.common.errors.TransactionalIdAuthorizationException; import org.apache.kafka.common.message.FindCoordinatorRequestData; +import org.apache.kafka.common.message.FindCoordinatorResponseData.Coordinator; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.AbstractResponse; import org.apache.kafka.common.requests.FindCoordinatorRequest; +import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType; import org.apache.kafka.common.requests.FindCoordinatorResponse; 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.Objects; import java.util.Set; +import java.util.stream.Collectors; public class CoordinatorStrategy implements AdminApiLookupStrategy<CoordinatorKey> { + + private static final ApiRequestScope BATCH_REQUEST_SCOPE = new ApiRequestScope() { }; + private final Logger log; + private final FindCoordinatorRequest.CoordinatorType type; + private Set<CoordinatorKey> unrepresentableKeys = Collections.emptySet(); + + boolean batch = true; public CoordinatorStrategy( + FindCoordinatorRequest.CoordinatorType type, LogContext logContext ) { + this.type = type; this.log = logContext.logger(CoordinatorStrategy.class); } @Override public ApiRequestScope lookupScope(CoordinatorKey key) { - // The `FindCoordinator` API does not support batched lookups, so we use a - // separate lookup context for each coordinator key we need to lookup - return new LookupRequestScope(key); + if (batch) { + return BATCH_REQUEST_SCOPE; + } else { + // If the `FindCoordinator` API does not support batched lookups, we use a + // separate lookup context for each coordinator key we need to lookup + return new LookupRequestScope(key); + } } @Override public FindCoordinatorRequest.Builder buildRequest(Set<CoordinatorKey> keys) { - CoordinatorKey key = requireSingleton(keys); - return new FindCoordinatorRequest.Builder( - new FindCoordinatorRequestData() - .setKey(key.idValue) - .setKeyType(key.type.id()) - ); + unrepresentableKeys = keys.stream().filter(k -> k == null || !isRepresentableKey(k.idValue)).collect(Collectors.toSet()); + Set<CoordinatorKey> representableKeys = keys.stream().filter(k -> k != null && isRepresentableKey(k.idValue)).collect(Collectors.toSet()); + if (batch) { + ensureSameType(representableKeys); + FindCoordinatorRequestData data = new FindCoordinatorRequestData() + .setKeyType(type.id()) + .setCoordinatorKeys(representableKeys.stream().map(k -> k.idValue).collect(Collectors.toList())); + return new FindCoordinatorRequest.Builder(data); + } else { + CoordinatorKey key = requireSingletonAndType(representableKeys); + return new FindCoordinatorRequest.Builder( + new FindCoordinatorRequestData() + .setKey(key.idValue) + .setKeyType(key.type.id()) + ); + } } @Override public LookupResult<CoordinatorKey> handleResponse( Set<CoordinatorKey> keys, AbstractResponse abstractResponse ) { - CoordinatorKey key = requireSingleton(keys); + Map<CoordinatorKey, Integer> mappedKeys = new HashMap<>(); + Map<CoordinatorKey, Throwable> failedKeys = new HashMap<>(); + + for (CoordinatorKey key : unrepresentableKeys) { + failedKeys.put(key, new InvalidGroupIdException("The given group id '" + + key.idValue + "' cannot be represented in a request.")); Review comment: nit: Indentation seems off here. ########## File path: clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiDriver.java ########## @@ -250,6 +252,13 @@ public void onFailure( .filter(future.lookupKeys()::contains) .collect(Collectors.toSet()); retryLookup(keysToUnmap); + + } else if (t instanceof NoBatchedFindCoordinatorsException) { Review comment: It feels a bit weird to handle a special case like this one in the driver. It is probably OK for the time being but we don't want to add more custom cases like this in the driver, I think. I wonder if we could delegate the decision to the handler. We could add an `handleUnsupportedVersionException` method to the `AdminApiHandler` for this purpose to delegate the decision. That method could basically return the keys to unmap and the keys to complete with the exception. An alternative would be to rely on the the `handleUnsupportedVersionException` method in `Call`. The driver could also implement it and still delegate the decision to the handler. The advantage of using this method is that downgrade would not be counted as failures and thus does not count for the retries. Have you considered something like this? ########## File path: clients/src/main/resources/common/message/FindCoordinatorRequest.json ########## @@ -23,12 +23,16 @@ // Version 2 is the same as version 1. // // Version 3 is the first flexible version. - "validVersions": "0-3", + // + // Version 4 adds CoordinatorKeys Review comment: nit: Could we add the KIP number here? I would also explain a bit more the change. The comment you have in the response is much better for instance. ########## File path: clients/src/main/resources/common/message/FindCoordinatorResponse.json ########## @@ -22,20 +22,33 @@ // Starting in version 2, on quota violation, brokers send out responses before throttling. // // Version 3 is the first flexible version. - "validVersions": "0-3", + // + // Version 4 adds support for batching via Coordinators Review comment: nit: Could we add the KIP number here? ########## File path: clients/src/main/java/org/apache/kafka/common/errors/NoBatchedFindCoordinatorsException.java ########## @@ -0,0 +1,33 @@ +/* + * 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.common.errors; + +/** + * Indicates that it is not possible to lookup coordinators in batches with FindCoordinator. Instead + * coordinators must be looked up one by one. + */ +public class NoBatchedFindCoordinatorsException extends UnsupportedVersionException { Review comment: We need to discuss this further. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org