rajinisivaram commented on a change in pull request #8683: URL: https://github.com/apache/kafka/pull/8683#discussion_r445008715
########## File path: clients/src/test/java/org/apache/kafka/clients/ClusterConnectionStatesTest.java ########## @@ -40,7 +40,12 @@ private final MockTime time = new MockTime(); private final long reconnectBackoffMs = 10 * 1000; private final long reconnectBackoffMax = 60 * 1000; - private final double reconnectBackoffJitter = 0.2; + private final long connectionSetupTimeoutMs = 10 * 1000; + private final long connectionSetupTimeoutMaxMs = 127 * 1000; + private final static int RECONNECT_BACKOFF_EXP_BASE = 2; + private final static double RECONNECT_BACKOFF_JITTER = 0.2; + private final static int CONNECTION_SETUP_TIMEOUT_EXP_BASE = 2; + private final static double CONNECTION_SETUP_TIMEOUT_JITTER = 0.2; Review comment: We can just make the constants in `ClusterConnectionStates` package-private and reuse them here? ########## File path: clients/src/main/java/org/apache/kafka/common/utils/ExponentialBackoff.java ########## @@ -0,0 +1,51 @@ +/* + * 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.utils; + +import java.util.concurrent.ThreadLocalRandom; + +/** + * An utility class for exponential backoff, timeout, etc... + * The formula is Term(n) = random(1 - jitter, 1 + jitter) * scaleFactor * (ratio) ^ n + * If scaleFactor is greater or equal than termMax, a constant term of will be provided + * This class is thread-safe + */ +public class ExponentialBackoff { + private final int ratio; + private final double expMax; + private final long scaleFactor; + private final double jitter; + + public ExponentialBackoff(long scaleFactor, int ratio, long termMax, double jitter) { Review comment: Sorry, missed this earlier. Yes, I prefer `initialInterval`, `multiplier`, and `maxInterval` too since they fit better with ExponentialBackoff. ---------------------------------------------------------------- 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