lianetm commented on code in PR #14364: URL: https://github.com/apache/kafka/pull/14364#discussion_r1326463429
########## 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 Review Comment: The subscription state already has it, and it is a component that is all over, so I would try to keep the regex in that single place -- 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