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


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * 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.common.KafkaException;
+import org.apache.kafka.common.errors.GroupAuthorizationException;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest;
+import org.apache.kafka.common.requests.ConsumerGroupHeartbeatResponse;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Timer;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * <p>Manages the request creation and response handling for the heartbeat. 
The module creates a
+ * {@link ConsumerGroupHeartbeatRequest} using the state stored in the {@link 
MembershipManager} and enqueue it to
+ * the network queue to be sent out. Once the response is received, the module 
will update the state in the
+ * {@link MembershipManager} and handle any errors.</p>
+ *
+ * <p>The manager only emits heartbeat when the member is in a group, tries to 
join or rejoin a group.
+ * If the member does not have groupId configured, got kicked out of the 
group, or encountering fatal exceptions, the
+ * heartbeat will not be sent.</p>
+ *
+ * <p>If the coordinator not is not found, we will skip sending the heartbeat 
and try to find a coordinator first.</p>
+ *
+ * <p>If the heartbeat failed due to retriable errors, such as, 
TimeoutException. The subsequent attempt will be
+ * backoff exponentially.</p>
+ *
+ * <p>If the member completes the assignment changes, i.e. revocation and 
assignment, a heartbeat request will be
+ * sent in the next event loop.</p>
+ *
+ * <p>See {@link HeartbeatRequestState} for more details.</p>
+ */
+public class HeartbeatRequestManager implements RequestManager {
+    private final Logger logger;
+    private final Set<Errors> fatalErrors = new HashSet<>(Arrays.asList(
+        Errors.GROUP_AUTHORIZATION_FAILED,
+        Errors.INVALID_REQUEST,
+        Errors.GROUP_MAX_SIZE_REACHED,
+        Errors.UNSUPPORTED_ASSIGNOR,
+        Errors.UNRELEASED_INSTANCE_ID));
+
+    private final int rebalanceTimeoutMs;

Review Comment:
   Added some comments directly above the var - Is this enough or you actually 
want this to be presented in the "javaDoc" of the class section?



-- 
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

Reply via email to