You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/11/29 22:27:04 UTC

[GitHub] [kafka] philipnee commented on a diff in pull request #12862: Consumer refator find coordinator

philipnee commented on code in PR #12862:
URL: https://github.com/apache/kafka/pull/12862#discussion_r1026994564


##########
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.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 ExponentialBackoff exponentialBackoff;
+    private long lastTimeOfConnectionMs = -1L; // starting logging a warning only after unable to connect for a while
+    private CoordinatorRequestState coordinatorRequestState;
+
+    private long rebalanceTimeoutMs;
+    private 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) {
+        // TODO: We should decouple the KafkaClient from this class
+        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);
+    }
+
+    public Optional<NetworkClientUtils.UnsentRequest> tryFindCoordinator() {
+        if (coordinatorRequestState.lastSentMs == -1) {
+            // no request has been sent
+            return Optional.of(
+                    new NetworkClientUtils.UnsentRequest(
+                            this.time.timer(requestTimeoutMs), // TODO: What is the correct timer to use here

Review Comment:
   Here we need a timeout to expire requests that sits in the Unsent queue (See NetworkClientUtils) Not sure if requestTimeoutMs is the right timeout to use here. 



##########
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.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;

Review Comment:
   i think we don't need jitter for the exp. backoff here.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org