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


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AssignorSelection.java:
##########
@@ -79,6 +80,9 @@ public int hashCode() {
 
     @Override
     public String toString() {
-        return String.format("Assignor selection {type:%s, name:%s}", type, 
serverAssignor);
+        return "AssignorSelection{" +

Review Comment:
   nit: I think that we usually use `()` instead of `{}` in to Strings.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.Collections;
+
+public class HeartbeatRequestManager implements RequestManager {
+    private final Time time;

Review Comment:
   Should we add javadoc to attributes, classes and methods? I think that we 
usually do it for all the new java code these days.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.Collections;
+
+public class HeartbeatRequestManager implements RequestManager {
+    private final Time time;
+    private final Logger logger;
+
+    private final int rebalanceTimeoutMs;
+
+    private final CoordinatorRequestManager coordinatorRequestManager;
+    private final SubscriptionState subscriptions;
+    private final HeartbeatRequestState heartbeatRequestState;
+    private final MembershipManager membershipManager;
+    private final ErrorEventHandler nonRetriableErrorHandler;
+
+    public HeartbeatRequestManager(
+        final Time time,
+        final LogContext logContext,
+        final ConsumerConfig config,
+        final CoordinatorRequestManager coordinatorRequestManager,
+        final SubscriptionState subscriptions,
+        final MembershipManager membershipManager,
+        final ErrorEventHandler nonRetriableErrorHandler) {
+        this.coordinatorRequestManager = coordinatorRequestManager;
+        this.time = time;
+        this.logger = logContext.logger(HeartbeatRequestManager.class);
+        this.subscriptions = subscriptions;
+        this.membershipManager = membershipManager;
+        this.nonRetriableErrorHandler = nonRetriableErrorHandler;
+        this.rebalanceTimeoutMs = 
config.getInt(CommonClientConfigs.MAX_POLL_INTERVAL_MS_CONFIG);
+
+        long heartbeatIntervalMs = 
config.getInt(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG);
+        long retryBackoffMs = 
config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
+        long retryBackoffMaxMs = 
config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG);
+        this.heartbeatRequestState = new HeartbeatRequestState(logContext, 
time, heartbeatIntervalMs, retryBackoffMs,
+            retryBackoffMaxMs, rebalanceTimeoutMs);
+    }
+
+    // Visible for testing
+    HeartbeatRequestManager(
+        final Time time,
+        final LogContext logContext,
+        final ConsumerConfig config,
+        final CoordinatorRequestManager coordinatorRequestManager,
+        final SubscriptionState subscriptions,
+        final MembershipManager membershipManager,
+        final HeartbeatRequestState heartbeatRequestState,
+        final ErrorEventHandler nonRetriableErrorHandler) {
+        this.time = time;
+        this.logger = logContext.logger(this.getClass());
+        this.subscriptions = subscriptions;
+        this.rebalanceTimeoutMs = 
config.getInt(CommonClientConfigs.MAX_POLL_INTERVAL_MS_CONFIG);
+        this.coordinatorRequestManager = coordinatorRequestManager;
+        this.heartbeatRequestState = heartbeatRequestState;
+        this.membershipManager = membershipManager;
+        this.nonRetriableErrorHandler = nonRetriableErrorHandler;
+    }
+
+    @Override
+    public NetworkClientDelegate.PollResult poll(long currentTimeMs) {
+        if (!coordinatorRequestManager.coordinator().isPresent() || 
membershipManager.notInGroup()) {
+            return new NetworkClientDelegate.PollResult(
+                Long.MAX_VALUE, Collections.emptyList());
+        }
+
+        if (!heartbeatRequestState.canSendRequest(currentTimeMs)) {
+            return new NetworkClientDelegate.PollResult(
+                heartbeatRequestState.nextHeartbeatMs(currentTimeMs),
+                Collections.emptyList());
+        }
+        this.heartbeatRequestState.onSendAttempt(currentTimeMs);
+        NetworkClientDelegate.UnsentRequest request = makeHeartbeatRequest();
+        // return Long.MAX_VALUE because we will update the timer when the 
response is received
+        return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, 
Collections.singletonList(request));
+    }
+
+    private NetworkClientDelegate.UnsentRequest makeHeartbeatRequest() {
+        ConsumerGroupHeartbeatRequestData data = new 
ConsumerGroupHeartbeatRequestData()
+            .setGroupId(membershipManager.groupId())
+            .setMemberEpoch(membershipManager.memberEpoch())
+            .setMemberId(membershipManager.memberId())
+            .setRebalanceTimeoutMs(rebalanceTimeoutMs);
+
+        membershipManager.groupInstanceId().ifPresent(data::setInstanceId);
+
+        if (this.subscriptions.hasPatternSubscription()) {
+            // We haven't discsussed how Regex is stored in the consumer. We 
could do it in the subscriptionState
+            // , in the memberStateManager, or here.
+            // data.setSubscribedTopicRegex(regex)
+        } else {
+            data.setSubscribedTopicNames(new 
ArrayList<>(this.subscriptions.subscription()));
+        }
+
+        if 
(this.membershipManager.assignorSelection().serverAssignor().isPresent()) {

Review Comment:
   nit: We could use `ifPresent`.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java:
##########
@@ -59,4 +59,5 @@ public interface MembershipManager {
      * current assignment.
      */
     void updateAssignment(ConsumerGroupHeartbeatResponseData.Assignment 
assignment);
+    boolean notInGroup();

Review Comment:
   nit: Let's add javadoc to be consistent with the other methods.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.Collections;
+
+public class HeartbeatRequestManager implements RequestManager {
+    private final Time time;
+    private final Logger logger;
+
+    private final int rebalanceTimeoutMs;
+
+    private final CoordinatorRequestManager coordinatorRequestManager;
+    private final SubscriptionState subscriptions;
+    private final HeartbeatRequestState heartbeatRequestState;
+    private final MembershipManager membershipManager;
+    private final ErrorEventHandler nonRetriableErrorHandler;
+
+    public HeartbeatRequestManager(
+        final Time time,
+        final LogContext logContext,
+        final ConsumerConfig config,
+        final CoordinatorRequestManager coordinatorRequestManager,
+        final SubscriptionState subscriptions,
+        final MembershipManager membershipManager,
+        final ErrorEventHandler nonRetriableErrorHandler) {
+        this.coordinatorRequestManager = coordinatorRequestManager;
+        this.time = time;
+        this.logger = logContext.logger(HeartbeatRequestManager.class);
+        this.subscriptions = subscriptions;
+        this.membershipManager = membershipManager;
+        this.nonRetriableErrorHandler = nonRetriableErrorHandler;
+        this.rebalanceTimeoutMs = 
config.getInt(CommonClientConfigs.MAX_POLL_INTERVAL_MS_CONFIG);
+
+        long heartbeatIntervalMs = 
config.getInt(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG);
+        long retryBackoffMs = 
config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
+        long retryBackoffMaxMs = 
config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG);
+        this.heartbeatRequestState = new HeartbeatRequestState(logContext, 
time, heartbeatIntervalMs, retryBackoffMs,
+            retryBackoffMaxMs, rebalanceTimeoutMs);
+    }
+
+    // Visible for testing
+    HeartbeatRequestManager(
+        final Time time,
+        final LogContext logContext,
+        final ConsumerConfig config,
+        final CoordinatorRequestManager coordinatorRequestManager,
+        final SubscriptionState subscriptions,
+        final MembershipManager membershipManager,
+        final HeartbeatRequestState heartbeatRequestState,
+        final ErrorEventHandler nonRetriableErrorHandler) {
+        this.time = time;
+        this.logger = logContext.logger(this.getClass());
+        this.subscriptions = subscriptions;
+        this.rebalanceTimeoutMs = 
config.getInt(CommonClientConfigs.MAX_POLL_INTERVAL_MS_CONFIG);
+        this.coordinatorRequestManager = coordinatorRequestManager;
+        this.heartbeatRequestState = heartbeatRequestState;
+        this.membershipManager = membershipManager;
+        this.nonRetriableErrorHandler = nonRetriableErrorHandler;
+    }
+
+    @Override
+    public NetworkClientDelegate.PollResult poll(long currentTimeMs) {
+        if (!coordinatorRequestManager.coordinator().isPresent() || 
membershipManager.notInGroup()) {

Review Comment:
   I am not sure to follow the `membershipManager.notInGroup()` part here. If 
we are not in the group, shouldn't we heartbeat to join (or rejoin) it?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.Collections;
+
+public class HeartbeatRequestManager implements RequestManager {
+    private final Time time;
+    private final Logger logger;
+
+    private final int rebalanceTimeoutMs;
+
+    private final CoordinatorRequestManager coordinatorRequestManager;
+    private final SubscriptionState subscriptions;
+    private final HeartbeatRequestState heartbeatRequestState;
+    private final MembershipManager membershipManager;
+    private final ErrorEventHandler nonRetriableErrorHandler;
+
+    public HeartbeatRequestManager(
+        final Time time,
+        final LogContext logContext,
+        final ConsumerConfig config,
+        final CoordinatorRequestManager coordinatorRequestManager,
+        final SubscriptionState subscriptions,
+        final MembershipManager membershipManager,
+        final ErrorEventHandler nonRetriableErrorHandler) {
+        this.coordinatorRequestManager = coordinatorRequestManager;
+        this.time = time;
+        this.logger = logContext.logger(HeartbeatRequestManager.class);
+        this.subscriptions = subscriptions;
+        this.membershipManager = membershipManager;
+        this.nonRetriableErrorHandler = nonRetriableErrorHandler;
+        this.rebalanceTimeoutMs = 
config.getInt(CommonClientConfigs.MAX_POLL_INTERVAL_MS_CONFIG);
+
+        long heartbeatIntervalMs = 
config.getInt(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG);

Review Comment:
   This is incorrect. The heartbeat interval comes is provided in the response.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.Collections;
+
+public class HeartbeatRequestManager implements RequestManager {
+    private final Time time;
+    private final Logger logger;
+
+    private final int rebalanceTimeoutMs;
+
+    private final CoordinatorRequestManager coordinatorRequestManager;
+    private final SubscriptionState subscriptions;
+    private final HeartbeatRequestState heartbeatRequestState;
+    private final MembershipManager membershipManager;
+    private final ErrorEventHandler nonRetriableErrorHandler;
+
+    public HeartbeatRequestManager(
+        final Time time,
+        final LogContext logContext,
+        final ConsumerConfig config,
+        final CoordinatorRequestManager coordinatorRequestManager,
+        final SubscriptionState subscriptions,
+        final MembershipManager membershipManager,
+        final ErrorEventHandler nonRetriableErrorHandler) {
+        this.coordinatorRequestManager = coordinatorRequestManager;
+        this.time = time;
+        this.logger = logContext.logger(HeartbeatRequestManager.class);
+        this.subscriptions = subscriptions;
+        this.membershipManager = membershipManager;
+        this.nonRetriableErrorHandler = nonRetriableErrorHandler;
+        this.rebalanceTimeoutMs = 
config.getInt(CommonClientConfigs.MAX_POLL_INTERVAL_MS_CONFIG);
+
+        long heartbeatIntervalMs = 
config.getInt(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG);
+        long retryBackoffMs = 
config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
+        long retryBackoffMaxMs = 
config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG);
+        this.heartbeatRequestState = new HeartbeatRequestState(logContext, 
time, heartbeatIntervalMs, retryBackoffMs,
+            retryBackoffMaxMs, rebalanceTimeoutMs);
+    }
+
+    // Visible for testing
+    HeartbeatRequestManager(
+        final Time time,
+        final LogContext logContext,
+        final ConsumerConfig config,
+        final CoordinatorRequestManager coordinatorRequestManager,
+        final SubscriptionState subscriptions,
+        final MembershipManager membershipManager,
+        final HeartbeatRequestState heartbeatRequestState,
+        final ErrorEventHandler nonRetriableErrorHandler) {
+        this.time = time;
+        this.logger = logContext.logger(this.getClass());
+        this.subscriptions = subscriptions;
+        this.rebalanceTimeoutMs = 
config.getInt(CommonClientConfigs.MAX_POLL_INTERVAL_MS_CONFIG);
+        this.coordinatorRequestManager = coordinatorRequestManager;
+        this.heartbeatRequestState = heartbeatRequestState;
+        this.membershipManager = membershipManager;
+        this.nonRetriableErrorHandler = nonRetriableErrorHandler;
+    }
+
+    @Override
+    public NetworkClientDelegate.PollResult poll(long currentTimeMs) {
+        if (!coordinatorRequestManager.coordinator().isPresent() || 
membershipManager.notInGroup()) {
+            return new NetworkClientDelegate.PollResult(
+                Long.MAX_VALUE, Collections.emptyList());
+        }
+
+        if (!heartbeatRequestState.canSendRequest(currentTimeMs)) {
+            return new NetworkClientDelegate.PollResult(
+                heartbeatRequestState.nextHeartbeatMs(currentTimeMs),
+                Collections.emptyList());
+        }
+        this.heartbeatRequestState.onSendAttempt(currentTimeMs);
+        NetworkClientDelegate.UnsentRequest request = makeHeartbeatRequest();
+        // return Long.MAX_VALUE because we will update the timer when the 
response is received
+        return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, 
Collections.singletonList(request));
+    }
+
+    private NetworkClientDelegate.UnsentRequest makeHeartbeatRequest() {
+        ConsumerGroupHeartbeatRequestData data = new 
ConsumerGroupHeartbeatRequestData()
+            .setGroupId(membershipManager.groupId())
+            .setMemberEpoch(membershipManager.memberEpoch())
+            .setMemberId(membershipManager.memberId())
+            .setRebalanceTimeoutMs(rebalanceTimeoutMs);
+
+        membershipManager.groupInstanceId().ifPresent(data::setInstanceId);
+
+        if (this.subscriptions.hasPatternSubscription()) {
+            // We haven't discsussed how Regex is stored in the consumer. We 
could do it in the subscriptionState
+            // , in the memberStateManager, or here.
+            // data.setSubscribedTopicRegex(regex)
+        } else {
+            data.setSubscribedTopicNames(new 
ArrayList<>(this.subscriptions.subscription()));
+        }
+
+        if 
(this.membershipManager.assignorSelection().serverAssignor().isPresent()) {
+            
data.setServerAssignor(this.membershipManager.assignorSelection().serverAssignor().get());
+        }
+
+        NetworkClientDelegate.UnsentRequest request = new 
NetworkClientDelegate.UnsentRequest(
+            new ConsumerGroupHeartbeatRequest.Builder(data),
+            coordinatorRequestManager.coordinator());
+
+        request.future().whenComplete((response, exception) -> {
+            final long currentTimeMs = time.milliseconds();
+            if (exception == null) {
+                onSuccess((ConsumerGroupHeartbeatResponse) 
response.responseBody(), currentTimeMs);
+            } else {
+                onFailure(exception, currentTimeMs);
+            }
+        });
+        return request;
+    }
+
+    private void onFailure(final Throwable exception, final long 
responseTimeMs) {
+        this.heartbeatRequestState.onFailedAttempt(responseTimeMs);
+        logger.debug("failed sending heartbeat due to {}", 
exception.getMessage());
+    }
+
+    private void onSuccess(final ConsumerGroupHeartbeatResponse response, long 
currentTimeMs) {
+        if (response.data().errorCode() == Errors.NONE.code()) {
+            this.heartbeatRequestState.onSuccessfulAttempt(currentTimeMs);
+            this.heartbeatRequestState.reset();
+            try {
+                membershipManager.updateState(response.data());
+            } catch (KafkaException e) {
+                logger.error("Received unexpected error in heartbeat response: 
{}", e.getMessage());
+            }
+            return;
+        }
+
+        onError(response, currentTimeMs);
+    }
+
+    private void onError(final ConsumerGroupHeartbeatResponse response,
+                         final long currentTimeMs) {
+
+        this.heartbeatRequestState.onFailedAttempt(currentTimeMs);
+        short errorCode = response.data().errorCode();
+        if (errorCode == Errors.NOT_COORDINATOR.code() || errorCode == 
Errors.COORDINATOR_NOT_AVAILABLE.code()) {
+            logger.info("GroupHeartbeatRequest failed: coordinator is either 
not started or not valid. Retrying in " +
+                    "{}ms: {}",
+                heartbeatRequestState.remainingBackoffMs(currentTimeMs),
+                response.data().errorMessage());
+            
coordinatorRequestManager.markCoordinatorUnknown(response.data().errorMessage(),
 currentTimeMs);
+        } else if (errorCode == Errors.COORDINATOR_LOAD_IN_PROGRESS.code()) {
+            // retry
+            logger.info("GroupHeartbeatRequest failed: Coordinator {} is 
loading. Retrying in {}ms: {}",
+                coordinatorRequestManager.coordinator(),
+                heartbeatRequestState.remainingBackoffMs(currentTimeMs),
+                response.data().errorMessage());
+        } else if (errorCode == Errors.GROUP_AUTHORIZATION_FAILED.code()) {
+            GroupAuthorizationException error = 
GroupAuthorizationException.forGroupId(membershipManager.groupId());
+            logger.error("GroupHeartbeatRequest failed due to group 
authorization failure: {}", error.getMessage());
+            nonRetriableErrorHandler.handle(error);
+        } else if (errorCode == Errors.INVALID_REQUEST.code()) {
+            logger.error("GroupHeartbeatRequest failed due to fatal error: 
{}", response.data().errorMessage());
+            
nonRetriableErrorHandler.handle(Errors.INVALID_REQUEST.exception());
+        } else if (errorCode == Errors.GROUP_MAX_SIZE_REACHED.code()) {
+            logger.error("GroupHeartbeatRequest failed due to the max group 
size limit: {}",
+                response.data().errorMessage());
+            
nonRetriableErrorHandler.handle(Errors.GROUP_MAX_SIZE_REACHED.exception());
+        } else if (errorCode == Errors.UNSUPPORTED_ASSIGNOR.code()) {
+            logger.error("GroupHeartbeatRequest failed due to unsupported 
assignor {}: {}",
+                membershipManager.assignorSelection(), 
response.data().errorMessage());
+            
nonRetriableErrorHandler.handle(Errors.UNSUPPORTED_ASSIGNOR.exception());
+        } else if (errorCode == Errors.UNRELEASED_INSTANCE_ID.code()) {
+            logger.error("GroupHeartbeatRequest failed due to the instance id 
{} was not released: {}",
+                membershipManager.groupInstanceId().orElse("null"),
+                response.data().errorMessage());
+            
nonRetriableErrorHandler.handle(Errors.UNRELEASED_INSTANCE_ID.exception());
+        }
+        membershipManager.updateState(response.data());

Review Comment:
   When we transition to Failed in updateState, I think that it analogous to a 
non retriable error. Is our plan to capture all the non-retriable errors before 
we reach this? We also do some error handling in updateState. The 
responsibilities are not clear here.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.Collections;
+
+public class HeartbeatRequestManager implements RequestManager {
+    private final Time time;
+    private final Logger logger;
+
+    private final int rebalanceTimeoutMs;
+
+    private final CoordinatorRequestManager coordinatorRequestManager;
+    private final SubscriptionState subscriptions;
+    private final HeartbeatRequestState heartbeatRequestState;
+    private final MembershipManager membershipManager;
+    private final ErrorEventHandler nonRetriableErrorHandler;
+
+    public HeartbeatRequestManager(
+        final Time time,
+        final LogContext logContext,
+        final ConsumerConfig config,
+        final CoordinatorRequestManager coordinatorRequestManager,
+        final SubscriptionState subscriptions,
+        final MembershipManager membershipManager,
+        final ErrorEventHandler nonRetriableErrorHandler) {
+        this.coordinatorRequestManager = coordinatorRequestManager;
+        this.time = time;
+        this.logger = logContext.logger(HeartbeatRequestManager.class);
+        this.subscriptions = subscriptions;
+        this.membershipManager = membershipManager;
+        this.nonRetriableErrorHandler = nonRetriableErrorHandler;
+        this.rebalanceTimeoutMs = 
config.getInt(CommonClientConfigs.MAX_POLL_INTERVAL_MS_CONFIG);
+
+        long heartbeatIntervalMs = 
config.getInt(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG);
+        long retryBackoffMs = 
config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
+        long retryBackoffMaxMs = 
config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG);
+        this.heartbeatRequestState = new HeartbeatRequestState(logContext, 
time, heartbeatIntervalMs, retryBackoffMs,
+            retryBackoffMaxMs, rebalanceTimeoutMs);
+    }
+
+    // Visible for testing
+    HeartbeatRequestManager(
+        final Time time,
+        final LogContext logContext,
+        final ConsumerConfig config,
+        final CoordinatorRequestManager coordinatorRequestManager,
+        final SubscriptionState subscriptions,
+        final MembershipManager membershipManager,
+        final HeartbeatRequestState heartbeatRequestState,
+        final ErrorEventHandler nonRetriableErrorHandler) {
+        this.time = time;
+        this.logger = logContext.logger(this.getClass());
+        this.subscriptions = subscriptions;
+        this.rebalanceTimeoutMs = 
config.getInt(CommonClientConfigs.MAX_POLL_INTERVAL_MS_CONFIG);
+        this.coordinatorRequestManager = coordinatorRequestManager;
+        this.heartbeatRequestState = heartbeatRequestState;
+        this.membershipManager = membershipManager;
+        this.nonRetriableErrorHandler = nonRetriableErrorHandler;
+    }
+
+    @Override
+    public NetworkClientDelegate.PollResult poll(long currentTimeMs) {
+        if (!coordinatorRequestManager.coordinator().isPresent() || 
membershipManager.notInGroup()) {
+            return new NetworkClientDelegate.PollResult(
+                Long.MAX_VALUE, Collections.emptyList());
+        }
+
+        if (!heartbeatRequestState.canSendRequest(currentTimeMs)) {
+            return new NetworkClientDelegate.PollResult(
+                heartbeatRequestState.nextHeartbeatMs(currentTimeMs),
+                Collections.emptyList());
+        }
+        this.heartbeatRequestState.onSendAttempt(currentTimeMs);
+        NetworkClientDelegate.UnsentRequest request = makeHeartbeatRequest();
+        // return Long.MAX_VALUE because we will update the timer when the 
response is received
+        return new NetworkClientDelegate.PollResult(Long.MAX_VALUE, 
Collections.singletonList(request));
+    }
+
+    private NetworkClientDelegate.UnsentRequest makeHeartbeatRequest() {
+        ConsumerGroupHeartbeatRequestData data = new 
ConsumerGroupHeartbeatRequestData()
+            .setGroupId(membershipManager.groupId())
+            .setMemberEpoch(membershipManager.memberEpoch())
+            .setMemberId(membershipManager.memberId())
+            .setRebalanceTimeoutMs(rebalanceTimeoutMs);
+
+        membershipManager.groupInstanceId().ifPresent(data::setInstanceId);
+
+        if (this.subscriptions.hasPatternSubscription()) {
+            // We haven't discsussed how Regex is stored in the consumer. We 
could do it in the subscriptionState
+            // , in the memberStateManager, or here.
+            // data.setSubscribedTopicRegex(regex)
+        } else {
+            data.setSubscribedTopicNames(new 
ArrayList<>(this.subscriptions.subscription()));
+        }
+
+        if 
(this.membershipManager.assignorSelection().serverAssignor().isPresent()) {
+            
data.setServerAssignor(this.membershipManager.assignorSelection().serverAssignor().get());
+        }
+
+        NetworkClientDelegate.UnsentRequest request = new 
NetworkClientDelegate.UnsentRequest(
+            new ConsumerGroupHeartbeatRequest.Builder(data),
+            coordinatorRequestManager.coordinator());
+
+        request.future().whenComplete((response, exception) -> {
+            final long currentTimeMs = time.milliseconds();
+            if (exception == null) {
+                onSuccess((ConsumerGroupHeartbeatResponse) 
response.responseBody(), currentTimeMs);
+            } else {
+                onFailure(exception, currentTimeMs);
+            }
+        });
+        return request;
+    }
+
+    private void onFailure(final Throwable exception, final long 
responseTimeMs) {
+        this.heartbeatRequestState.onFailedAttempt(responseTimeMs);
+        logger.debug("failed sending heartbeat due to {}", 
exception.getMessage());
+    }
+
+    private void onSuccess(final ConsumerGroupHeartbeatResponse response, long 
currentTimeMs) {
+        if (response.data().errorCode() == Errors.NONE.code()) {
+            this.heartbeatRequestState.onSuccessfulAttempt(currentTimeMs);
+            this.heartbeatRequestState.reset();
+            try {
+                membershipManager.updateState(response.data());
+            } catch (KafkaException e) {
+                logger.error("Received unexpected error in heartbeat response: 
{}", e.getMessage());
+            }
+            return;
+        }
+
+        onError(response, currentTimeMs);
+    }
+
+    private void onError(final ConsumerGroupHeartbeatResponse response,
+                         final long currentTimeMs) {
+

Review Comment:
   nit: This empty line could be removed.



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java:
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.consumer.ConsumerConfig;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ConsumerGroupHeartbeatResponse;
+import org.apache.kafka.common.requests.RequestHeader;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.common.utils.Time;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Optional;
+import java.util.Properties;
+
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.RETRY_BACKOFF_MS_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class HeartbeatRequestManagerTest {
+
+    private final int heartbeatInterval = 1000;
+    private final long retryBackoffMaxMs = 3000;
+    private final long retryBackoffMs = 100;
+    private final String groupId = "group-id";
+
+    private Time mockTime;
+    private LogContext mockLogContext;
+    private CoordinatorRequestManager mockCoordinatorRequestManager;
+    private SubscriptionState mockSubscriptionState;
+    private HeartbeatRequestManager heartbeatRequestManager;
+    private MembershipManager mockMembershipManager;
+    private HeartbeatRequestManager.HeartbeatRequestState 
heartbeatRequestState;
+    private ConsumerConfig config;
+
+    private String memberId = "member-id";
+    private int memberEpoch = 1;
+    private ConsumerGroupHeartbeatResponseData.Assignment memberAssignment = 
mockAssignment();
+    private ErrorEventHandler errorEventHandler;
+
+    private ConsumerGroupHeartbeatResponseData.Assignment mockAssignment() {
+        return new ConsumerGroupHeartbeatResponseData.Assignment()
+            .setAssignedTopicPartitions(Arrays.asList(
+                new ConsumerGroupHeartbeatResponseData.TopicPartitions()
+                    .setTopicId(Uuid.randomUuid())
+                    .setPartitions(Arrays.asList(0, 1, 2)),
+                new ConsumerGroupHeartbeatResponseData.TopicPartitions()
+                    .setTopicId(Uuid.randomUuid())
+                    .setPartitions(Arrays.asList(3, 4, 5))
+            ));
+    }
+
+    @BeforeEach
+    public void setUp() {
+        mockTime = new MockTime();
+        mockLogContext = new LogContext();
+        Properties properties = new Properties();
+        properties.put(BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
+        properties.put(KEY_DESERIALIZER_CLASS_CONFIG, 
StringDeserializer.class);
+        properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, 
StringDeserializer.class);
+        properties.put(RETRY_BACKOFF_MS_CONFIG, "100");
+        properties.put(HEARTBEAT_INTERVAL_MS_CONFIG, heartbeatInterval);
+        config = new ConsumerConfig(properties);
+        mockCoordinatorRequestManager = mock(CoordinatorRequestManager.class);
+        
when(mockCoordinatorRequestManager.coordinator()).thenReturn(Optional.of(new 
Node(1, "localhost", 9999)));
+        mockSubscriptionState = mock(SubscriptionState.class);
+        mockMembershipManager = spy(new MembershipManagerImpl(groupId));
+        heartbeatRequestState = 
mock(HeartbeatRequestManager.HeartbeatRequestState.class);
+        errorEventHandler = mock(ErrorEventHandler.class);
+        heartbeatRequestManager = new HeartbeatRequestManager(
+            mockTime,
+            mockLogContext,
+            config,
+            mockCoordinatorRequestManager,
+            mockSubscriptionState,
+            mockMembershipManager,
+            heartbeatRequestState,
+            errorEventHandler);
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testPoll_sendHeartbeatOnMemberState(boolean notInGroup) {

Review Comment:
   nit: The naming does not respect our conventions here. We should use camel 
case.



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