hachikuji commented on code in PR #12862: URL: https://github.com/apache/kafka/pull/12862#discussion_r1035413023
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/CoordinatorManager.java: ########## @@ -0,0 +1,245 @@ +/* + * 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.consumer.internals; + +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorBackgroundEvent; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.errors.GroupAuthorizationException; +import org.apache.kafka.common.errors.RetriableException; +import org.apache.kafka.common.message.FindCoordinatorRequestData; +import org.apache.kafka.common.message.FindCoordinatorResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.requests.FindCoordinatorRequest; +import org.apache.kafka.common.requests.FindCoordinatorResponse; +import org.apache.kafka.common.utils.ExponentialBackoff; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.Time; +import org.slf4j.Logger; + +import java.util.List; +import java.util.Optional; +import java.util.concurrent.BlockingQueue; + +public class CoordinatorManager { + final static int RECONNECT_BACKOFF_EXP_BASE = 2; + final static double RECONNECT_BACKOFF_JITTER = 0.0; + private final Logger log; + private final Time time; + private final long requestTimeoutMs; + private Node coordinator; + private final BlockingQueue<BackgroundEvent> backgroundEventQueue; + private final ExponentialBackoff exponentialBackoff; + private long lastTimeOfConnectionMs = -1L; // starting logging a warning only after unable to connect for a while + private final CoordinatorRequestState coordinatorRequestState; + + private final long rebalanceTimeoutMs; + private final Optional<String> groupId; + + public CoordinatorManager(final Time time, + final LogContext logContext, + final ConsumerConfig config, + final BlockingQueue<BackgroundEvent> backgroundEventQueue, + final Optional<String> groupId, + final long rebalanceTimeoutMs) { + this.time = time; + this.log = logContext.logger(this.getClass()); + this.backgroundEventQueue = backgroundEventQueue; + this.exponentialBackoff = new ExponentialBackoff( + config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG), + RECONNECT_BACKOFF_EXP_BASE, + config.getLong(CommonClientConfigs.RECONNECT_BACKOFF_MAX_MS_CONFIG), + RECONNECT_BACKOFF_JITTER); + this.coordinatorRequestState = new CoordinatorRequestState(); + this.groupId = groupId; + this.rebalanceTimeoutMs = rebalanceTimeoutMs; + this.requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG); + } + + /** + * Returns a non-empty UnsentRequest we need send a FindCoordinatorRequest. These conditions are: + * 1. The request has not been sent + * 2. If the previous request failed, and the retryBackoff has expired + * @return Optional UnsentRequest. Empty if we are not allowed to send a request. + */ + public Optional<NetworkClientDelegate.UnsentRequest> tryFindCoordinator() { + if (coordinatorRequestState.lastSentMs == -1) { + // no request has been sent + return Optional.of( + new NetworkClientDelegate.UnsentRequest( + this.time.timer(requestTimeoutMs), + getFindCoordinatorRequest(), + new FindCoordinatorRequestHandler())); + } + + if (coordinatorRequestState.lastReceivedMs == -1 || + coordinatorRequestState.lastReceivedMs < coordinatorRequestState.lastSentMs) { + // there is an inflight request + return Optional.empty(); + } + + if (!coordinatorRequestState.requestBackoffExpired()) { + // retryBackoff + return Optional.empty(); + } + + return Optional.of( + new NetworkClientDelegate.UnsentRequest( + this.time.timer(requestTimeoutMs), + getFindCoordinatorRequest(), + new FindCoordinatorRequestHandler())); + } + + /** + * Mark the current coordinator null and return the old coordinator. Return an empty Optional + * if the current coordinator is unknown. + * @param cause why the coordinator is marked unknown + * @return Optional coordinator node that can be null. + */ + protected Optional<Node> markCoordinatorUnknown(String cause) { + Node oldCoordinator = this.coordinator; + if (this.coordinator != null) { + log.info("Group coordinator {} is unavailable or invalid due to cause: {}. " + + "Rediscovery will be attempted.", this.coordinator, cause); + this.coordinator = null; + lastTimeOfConnectionMs = time.milliseconds(); + } else { + long durationOfOngoingDisconnect = time.milliseconds() - lastTimeOfConnectionMs; Review Comment: We need to be a little careful with measurements like this since `milliseconds()` is not guaranteed to be monotonic. No harm here I guess if the value is less than 0, but perhaps we should use `Math.max(0, XX)` at least to make the log message less confusing. -- 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