dajac commented on code in PR #14364: URL: https://github.com/apache/kafka/pull/14364#discussion_r1347449738
########## clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java: ########## @@ -0,0 +1,364 @@ +/* + * 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.errors.TimeoutException; +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.ConsumerGroupHeartbeatRequest; +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.apache.kafka.common.utils.annotation.ApiKeyVersionsSource; +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.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.junit.jupiter.api.Assertions.assertTrue; +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.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class HeartbeatRequestManagerTest { + + private static final int HEARTBEAT_INTERVAL_MS = 1000; + private static final long RETRY_BACKOFF_MAX_MS = 3000; + private static final long RETRY_BACKOFF_MS = 100; + private static final String GROUP_ID = "group-id"; + + private Time time; + private LogContext logContext; + private CoordinatorRequestManager coordinatorRequestManager; + private SubscriptionState subscriptionState; + private HeartbeatRequestManager heartbeatRequestManager; + private MembershipManager membershipManager; + private HeartbeatRequestManager.HeartbeatRequestState heartbeatRequestState; + private ConsumerConfig config; + + private String memberId = "member-id"; + private int memberEpoch = 1; + private ErrorEventHandler errorEventHandler; + + @BeforeEach + public void setUp() { + time = new MockTime(); + logContext = 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"); + config = new ConsumerConfig(properties); + coordinatorRequestManager = mock(CoordinatorRequestManager.class); + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); + subscriptionState = mock(SubscriptionState.class); + membershipManager = spy(new MembershipManagerImpl(GROUP_ID)); + heartbeatRequestState = mock(HeartbeatRequestManager.HeartbeatRequestState.class); + errorEventHandler = mock(ErrorEventHandler.class); + heartbeatRequestManager = createManager(); + } + + @Test + public void testHeartbeatOnStartup() { + // The initial heartbeatInterval is set to 0 + heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( + logContext, + time, + 0, + RETRY_BACKOFF_MS, + RETRY_BACKOFF_MAX_MS, + 0); + heartbeatRequestManager = createManager(); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(1, result.unsentRequests.size()); + + // Ensure we do not resend the request without the first request being completed + NetworkClientDelegate.PollResult result2 = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(0, result2.unsentRequests.size()); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testSendHeartbeatOnMemberState(final boolean shouldSendHeartbeat) { + // Mocking notInGroup + when(membershipManager.shouldSendHeartbeat()).thenReturn(shouldSendHeartbeat); + when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); + + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + + if (shouldSendHeartbeat) { + assertEquals(1, result.unsentRequests.size()); + assertEquals(0, result.timeUntilNextPollMs); + } else { + assertEquals(0, result.unsentRequests.size()); + assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); + + } + } + + @ParameterizedTest + @MethodSource("stateProvider") + public void testTimerNotDue(final MemberState state) { + heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( + logContext, + time, + HEARTBEAT_INTERVAL_MS, + RETRY_BACKOFF_MS, + RETRY_BACKOFF_MAX_MS); + heartbeatRequestManager = createManager(); + + when(membershipManager.state()).thenReturn(state); + time.sleep(100); // time elapsed < heartbeatInterval, no heartbeat should be sent + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(0, result.unsentRequests.size()); + + if (membershipManager.shouldSendHeartbeat()) { + assertEquals(HEARTBEAT_INTERVAL_MS - 100, result.timeUntilNextPollMs); + } else { + assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); + } + } + + @Test + public void testBackoffOnHeartbeatTimeout() { + heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( + logContext, + time, + 0, + RETRY_BACKOFF_MS, + RETRY_BACKOFF_MAX_MS, + 0); + heartbeatRequestManager = createManager(); + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); + when(membershipManager.shouldSendHeartbeat()).thenReturn(true); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(1, result.unsentRequests.size()); + result.unsentRequests.get(0).future().completeExceptionally(new TimeoutException("timeout")); + + // assure the manager will backoff on timeout + time.sleep(RETRY_BACKOFF_MS - 1); + result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(0, result.unsentRequests.size()); + + time.sleep(1); + result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(1, result.unsentRequests.size()); + } + + @Test + public void testNoCoordinator() { + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.empty()); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + + assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); + assertEquals(0, result.unsentRequests.size()); + } + + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT) + public void testValidateConsumerGroupHeartbeatRequest(final short version) { + heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( + logContext, + time, + 0, + RETRY_BACKOFF_MS, + RETRY_BACKOFF_MAX_MS, + 0); + heartbeatRequestManager = createManager(); + // Update membershipManager's memberId and memberEpoch + ConsumerGroupHeartbeatResponse result = + new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(memberEpoch)); + membershipManager.updateState(result.data()); + + // Create a ConsumerHeartbeatRequest and verify the payload + NetworkClientDelegate.PollResult pollResult = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(1, pollResult.unsentRequests.size()); + NetworkClientDelegate.UnsentRequest request = pollResult.unsentRequests.get(0); + assertTrue(request.requestBuilder() instanceof ConsumerGroupHeartbeatRequest.Builder); + + ConsumerGroupHeartbeatRequest heartbeatRequest = + (ConsumerGroupHeartbeatRequest) request.requestBuilder().build(version); + assertEquals(GROUP_ID, heartbeatRequest.data().groupId()); + assertEquals(memberId, heartbeatRequest.data().memberId()); + assertEquals(memberEpoch, heartbeatRequest.data().memberEpoch()); Review Comment: Should we verify all the fields? We also set others [here](https://github.com/apache/kafka/pull/14364/files#diff-0eb94945bb75c3b1bda527708a48af59eafafc5bd9f85136fff0d0c405590497R166). ########## clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java: ########## @@ -0,0 +1,364 @@ +/* + * 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.errors.TimeoutException; +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.ConsumerGroupHeartbeatRequest; +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.apache.kafka.common.utils.annotation.ApiKeyVersionsSource; +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.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.junit.jupiter.api.Assertions.assertTrue; +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.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class HeartbeatRequestManagerTest { + + private static final int HEARTBEAT_INTERVAL_MS = 1000; + private static final long RETRY_BACKOFF_MAX_MS = 3000; + private static final long RETRY_BACKOFF_MS = 100; + private static final String GROUP_ID = "group-id"; + + private Time time; + private LogContext logContext; + private CoordinatorRequestManager coordinatorRequestManager; + private SubscriptionState subscriptionState; + private HeartbeatRequestManager heartbeatRequestManager; + private MembershipManager membershipManager; + private HeartbeatRequestManager.HeartbeatRequestState heartbeatRequestState; + private ConsumerConfig config; + + private String memberId = "member-id"; + private int memberEpoch = 1; + private ErrorEventHandler errorEventHandler; + + @BeforeEach + public void setUp() { + time = new MockTime(); + logContext = 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"); + config = new ConsumerConfig(properties); + coordinatorRequestManager = mock(CoordinatorRequestManager.class); + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); + subscriptionState = mock(SubscriptionState.class); + membershipManager = spy(new MembershipManagerImpl(GROUP_ID)); + heartbeatRequestState = mock(HeartbeatRequestManager.HeartbeatRequestState.class); + errorEventHandler = mock(ErrorEventHandler.class); + heartbeatRequestManager = createManager(); + } + + @Test + public void testHeartbeatOnStartup() { + // The initial heartbeatInterval is set to 0 + heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( + logContext, + time, + 0, + RETRY_BACKOFF_MS, + RETRY_BACKOFF_MAX_MS, + 0); + heartbeatRequestManager = createManager(); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(1, result.unsentRequests.size()); + + // Ensure we do not resend the request without the first request being completed + NetworkClientDelegate.PollResult result2 = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(0, result2.unsentRequests.size()); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testSendHeartbeatOnMemberState(final boolean shouldSendHeartbeat) { + // Mocking notInGroup + when(membershipManager.shouldSendHeartbeat()).thenReturn(shouldSendHeartbeat); + when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); + + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + + if (shouldSendHeartbeat) { + assertEquals(1, result.unsentRequests.size()); + assertEquals(0, result.timeUntilNextPollMs); + } else { + assertEquals(0, result.unsentRequests.size()); + assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); + + } + } + + @ParameterizedTest + @MethodSource("stateProvider") + public void testTimerNotDue(final MemberState state) { + heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( + logContext, + time, + HEARTBEAT_INTERVAL_MS, + RETRY_BACKOFF_MS, + RETRY_BACKOFF_MAX_MS); + heartbeatRequestManager = createManager(); + + when(membershipManager.state()).thenReturn(state); + time.sleep(100); // time elapsed < heartbeatInterval, no heartbeat should be sent + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(0, result.unsentRequests.size()); + + if (membershipManager.shouldSendHeartbeat()) { + assertEquals(HEARTBEAT_INTERVAL_MS - 100, result.timeUntilNextPollMs); + } else { + assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); + } + } + + @Test + public void testBackoffOnHeartbeatTimeout() { + heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( + logContext, + time, + 0, + RETRY_BACKOFF_MS, + RETRY_BACKOFF_MAX_MS, + 0); + heartbeatRequestManager = createManager(); + when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); + when(membershipManager.shouldSendHeartbeat()).thenReturn(true); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(time.milliseconds()); + assertEquals(1, result.unsentRequests.size()); + result.unsentRequests.get(0).future().completeExceptionally(new TimeoutException("timeout")); + + // assure the manager will backoff on timeout Review Comment: nit: Assure... ########## clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java: ########## @@ -0,0 +1,346 @@ +/* + * 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.errors.TimeoutException; +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.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.apache.kafka.clients.consumer.internals.MemberState.FAILED; +import static org.apache.kafka.clients.consumer.internals.MemberState.STABLE; +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.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class HeartbeatRequestManagerTest { + + private final int heartbeatIntervalMs = 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 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"); + 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( + mockLogContext, + mockTime, + config, + mockCoordinatorRequestManager, + mockSubscriptionState, + mockMembershipManager, + heartbeatRequestState, + errorEventHandler); + } + + @Test + public void testHeartbeatOnStartup() { + // The initial heartbeatInterval is set to 0 + heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( + mockLogContext, + mockTime, + 0, + retryBackoffMs, + retryBackoffMaxMs, + 0); + heartbeatRequestManager = createManager(); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(mockTime.milliseconds()); + assertEquals(1, result.unsentRequests.size()); + + // Ensure we do not resend the request without the first request being completed + NetworkClientDelegate.PollResult result2 = heartbeatRequestManager.poll(mockTime.milliseconds()); + assertEquals(0, result2.unsentRequests.size()); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testSendHeartbeatOnMemberState(boolean shouldSendHeartbeat) { + // Mocking notInGroup + when(mockMembershipManager.shouldSendHeartbeat()).thenReturn(shouldSendHeartbeat); + when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); + + NetworkClientDelegate.PollResult result; + result = heartbeatRequestManager.poll(mockTime.milliseconds()); + + if (shouldSendHeartbeat) { + assertEquals(1, result.unsentRequests.size()); + assertEquals(0, result.timeUntilNextPollMs); + } else { + assertEquals(0, result.unsentRequests.size()); + assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); + + } + } + + @ParameterizedTest + @MethodSource("stateProvider") + public void testTimerNotDue(final MemberState state) { + this.heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( + mockLogContext, + mockTime, + heartbeatIntervalMs, + retryBackoffMs, + retryBackoffMaxMs); + heartbeatRequestManager = createManager(); + + when(mockMembershipManager.state()).thenReturn(state); + mockTime.sleep(100); // time elapsed < heartbeatInterval, no heartbeat should be sent + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(mockTime.milliseconds()); + assertEquals(0, result.unsentRequests.size()); + + if (mockMembershipManager.shouldSendHeartbeat()) { + assertEquals(heartbeatIntervalMs - 100, result.timeUntilNextPollMs); + } else { + assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); + } + } + + @Test + public void testBackoffOnHeartbeatTimeout() { + heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( + mockLogContext, + mockTime, + 0, + retryBackoffMs, + retryBackoffMaxMs, + 0); + heartbeatRequestManager = createManager(); + when(mockCoordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); + when(mockMembershipManager.shouldSendHeartbeat()).thenReturn(true); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(mockTime.milliseconds()); + assertEquals(1, result.unsentRequests.size()); + result.unsentRequests.get(0).future().completeExceptionally(new TimeoutException("timeout")); Review Comment: Hum... I don't follow. My point was that we don't have any tests verifying that we actually update the state machine when a fatal __exception__ is received [here](https://github.com/apache/kafka/pull/14364/files#diff-0eb94945bb75c3b1bda527708a48af59eafafc5bd9f85136fff0d0c405590497R209). Or did I miss it? ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java: ########## @@ -0,0 +1,384 @@ +/* + * 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.errors.GroupAuthorizationException; +import org.apache.kafka.common.errors.RetriableException; +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; + +/** + * <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 will try to send a heartbeat when the member is in {@link MemberState#STABLE}, + * {@link MemberState#UNJOINED}, or {@link MemberState#RECONCILING}. Which mean the member is either in a stable + * group, is trying to join a group, or is in the process of reconciling the assignment changes.</p> + * + * <p>If the member got kick out of a group, it will try to give up the current assignment by invoking {@code + * OnPartitionsLost} because reattempting to join again with a zero epoch.</p> + * + * <p>If the member does not have groupId configured or encountering fatal exceptions, a 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>When the member completes the assignment reconciliation, the {@link HeartbeatRequestState} will be reset so + * that a heartbeat 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 Time time; + + /** + * Time that the group coordinator will wait on member to revoke its partitions. This is provided by the group + * coordinator in the heartbeat + */ + private final int rebalanceTimeoutMs; + + /** + * CoordinatorRequestManager manages the connection to the group coordinator + */ + private final CoordinatorRequestManager coordinatorRequestManager; + + /** + * SubscriptionState tracks the topic, partition and offset of the member + */ + private final SubscriptionState subscriptions; + + /** + * HeartbeatRequestState manages heartbeat request timing and retries + */ + private final HeartbeatRequestState heartbeatRequestState; + + /** + * MembershipManager manages member's essential attributes like epoch and id, and its rebalance state + */ + private final MembershipManager membershipManager; + + /** + * ErrorEventHandler allows the background thread to propagate errors back to the user + */ + 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(getClass()); + this.subscriptions = subscriptions; + this.membershipManager = membershipManager; + this.nonRetriableErrorHandler = nonRetriableErrorHandler; + this.rebalanceTimeoutMs = config.getInt(CommonClientConfigs.MAX_POLL_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, 0, retryBackoffMs, + retryBackoffMaxMs, rebalanceTimeoutMs); + } + + // Visible for testing + HeartbeatRequestManager( + final LogContext logContext, + final Time time, + final ConsumerConfig config, + final CoordinatorRequestManager coordinatorRequestManager, + final SubscriptionState subscriptions, + final MembershipManager membershipManager, + final HeartbeatRequestState heartbeatRequestState, + final ErrorEventHandler nonRetriableErrorHandler) { + this.logger = logContext.logger(this.getClass()); + this.time = time; + 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; + } + + /** + * Determines the maximum wait time until the next poll based on the member's state, and creates a heartbeat + * request. + * <ol> + * <li>If the member is without a coordinator or is in a failed state, the timer is set to Long.MAX_VALUE, as there's no need to send a heartbeat.</li> + * <li>If the member cannot send a heartbeat due to either exponential backoff, it will return the remaining time left on the backoff timer.</li> + * <li>If the member's heartbeat timer has not expired, It will return the remaining time left on the + * heartbeat timer.</li> + * <li>If the member can send a heartbeat, the timer is set to the current heartbeat interval.</li> + * </ol> + */ + @Override + public NetworkClientDelegate.PollResult poll(long currentTimeMs) { + if (!coordinatorRequestManager.coordinator().isPresent() || !membershipManager.shouldSendHeartbeat()) { + return new NetworkClientDelegate.PollResult( + Long.MAX_VALUE, Collections.emptyList()); + } + + // TODO: We will need to send a heartbeat response after partitions being revoke. This needs to be + // implemented either with or after the partition reconciliation logic. + if (!heartbeatRequestState.canSendRequest(currentTimeMs)) { + return new NetworkClientDelegate.PollResult( + heartbeatRequestState.nextHeartbeatMs(currentTimeMs), + Collections.emptyList()); + } + this.heartbeatRequestState.onSendAttempt(currentTimeMs); + NetworkClientDelegate.UnsentRequest request = makeHeartbeatRequest(); + return new NetworkClientDelegate.PollResult(heartbeatRequestState.heartbeatIntervalMs, Collections.singletonList(request)); + } + + private NetworkClientDelegate.UnsentRequest makeHeartbeatRequest() { + // TODO: We only need to send the rebalanceTimeoutMs field once unless the first request failed. + ConsumerGroupHeartbeatRequestData data = new ConsumerGroupHeartbeatRequestData() + .setGroupId(membershipManager.groupId()) + .setMemberEpoch(membershipManager.memberEpoch()) + .setMemberId(membershipManager.memberId()) + .setRebalanceTimeoutMs(rebalanceTimeoutMs); + + membershipManager.groupInstanceId().ifPresent(data::setInstanceId); + + if (this.subscriptions.hasPatternSubscription()) { + // TODO: Pass the string to the GC if server side regex is used. + } else { + data.setSubscribedTopicNames(new ArrayList<>(this.subscriptions.subscription())); + } + + this.membershipManager.assignorSelection().serverAssignor().ifPresent(data::setServerAssignor); + + NetworkClientDelegate.UnsentRequest request = new NetworkClientDelegate.UnsentRequest( + new ConsumerGroupHeartbeatRequest.Builder(data), + coordinatorRequestManager.coordinator()); + request.future().whenComplete((response, exception) -> { + if (response != null) { + onResponse((ConsumerGroupHeartbeatResponse) response.responseBody(), response.receivedTimeMs()); + } else { + // TODO: Currently, we lack a good way to propage the response time from the network client to the + // request handler. We will need to store the response time in the handler to make it accessible. + onFailure(exception, time.milliseconds()); + } + }); + return request; + } + + private void onFailure(final Throwable exception, final long responseTimeMs) { + this.heartbeatRequestState.onFailedAttempt(responseTimeMs); + logger.warn("Failed to send heartbeat to coordinator node {} due to error: {}", + coordinatorRequestManager.coordinator(), exception.getMessage()); + if (exception instanceof RetriableException) { + logger.debug("Retrying heartbeat request in {}ms due to {}", + heartbeatRequestState.remainingBackoffMs(responseTimeMs), + exception.getMessage()); + } else { + logger.error("Heartbeat request failed due to fatal error", exception); + membershipManager.transitionToFailed(); + nonRetriableErrorHandler.handle(exception); + } + } + + private void onResponse(final ConsumerGroupHeartbeatResponse response, long currentTimeMs) { + if (Errors.forCode(response.data().errorCode()) == Errors.NONE) { + this.heartbeatRequestState.updateHeartbeatIntervalMs(response.data().heartbeatIntervalMs()); + this.heartbeatRequestState.onSuccessfulAttempt(currentTimeMs); + this.heartbeatRequestState.resetTimer(); + this.membershipManager.updateState(response.data()); + return; + } + onErrorResponse(response, currentTimeMs); + } + + private void onErrorResponse(final ConsumerGroupHeartbeatResponse response, + final long currentTimeMs) { + Errors error = Errors.forCode(response.data().errorCode()); + String errorMessage = response.data().errorMessage(); + // TODO: upon encountering a fatal/fenced error, trigger onPartitionLost logic to give up the current + // assignments. + switch (error) { + case NOT_COORDINATOR: + // the manager should retry immediately when the coordinator node becomes available again + String message = String.format("GroupHeartbeatRequest failed because the group coordinator %s is incorrect. " + + "Will attempt to find the coordinator again and retry", + coordinatorRequestManager.coordinator()); + logInfo(message, response, currentTimeMs); + coordinatorRequestManager.markCoordinatorUnknown(errorMessage, currentTimeMs); + break; + case COORDINATOR_NOT_AVAILABLE: Review Comment: nit: empty line. ########## clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java: ########## @@ -0,0 +1,346 @@ +/* + * 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.errors.TimeoutException; +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.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.apache.kafka.clients.consumer.internals.MemberState.FAILED; +import static org.apache.kafka.clients.consumer.internals.MemberState.STABLE; +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.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class HeartbeatRequestManagerTest { + + private final int heartbeatIntervalMs = 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 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"); + 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( + mockLogContext, + mockTime, + config, + mockCoordinatorRequestManager, + mockSubscriptionState, + mockMembershipManager, + heartbeatRequestState, + errorEventHandler); + } + + @Test + public void testHeartbeatOnStartup() { + // The initial heartbeatInterval is set to 0 + heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( + mockLogContext, + mockTime, + 0, + retryBackoffMs, + retryBackoffMaxMs, + 0); + heartbeatRequestManager = createManager(); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(mockTime.milliseconds()); + assertEquals(1, result.unsentRequests.size()); + + // Ensure we do not resend the request without the first request being completed + NetworkClientDelegate.PollResult result2 = heartbeatRequestManager.poll(mockTime.milliseconds()); + assertEquals(0, result2.unsentRequests.size()); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testSendHeartbeatOnMemberState(boolean shouldSendHeartbeat) { + // Mocking notInGroup + when(mockMembershipManager.shouldSendHeartbeat()).thenReturn(shouldSendHeartbeat); + when(heartbeatRequestState.canSendRequest(anyLong())).thenReturn(true); + + NetworkClientDelegate.PollResult result; + result = heartbeatRequestManager.poll(mockTime.milliseconds()); + + if (shouldSendHeartbeat) { + assertEquals(1, result.unsentRequests.size()); + assertEquals(0, result.timeUntilNextPollMs); + } else { + assertEquals(0, result.unsentRequests.size()); + assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); + + } + } + + @ParameterizedTest + @MethodSource("stateProvider") + public void testTimerNotDue(final MemberState state) { + this.heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( + mockLogContext, + mockTime, + heartbeatIntervalMs, + retryBackoffMs, + retryBackoffMaxMs); + heartbeatRequestManager = createManager(); + + when(mockMembershipManager.state()).thenReturn(state); + mockTime.sleep(100); // time elapsed < heartbeatInterval, no heartbeat should be sent + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(mockTime.milliseconds()); + assertEquals(0, result.unsentRequests.size()); + + if (mockMembershipManager.shouldSendHeartbeat()) { + assertEquals(heartbeatIntervalMs - 100, result.timeUntilNextPollMs); + } else { + assertEquals(Long.MAX_VALUE, result.timeUntilNextPollMs); + } + } + + @Test + public void testBackoffOnHeartbeatTimeout() { + heartbeatRequestState = new HeartbeatRequestManager.HeartbeatRequestState( + mockLogContext, + mockTime, + 0, + retryBackoffMs, + retryBackoffMaxMs, + 0); + heartbeatRequestManager = createManager(); + when(mockCoordinatorRequestManager.coordinator()).thenReturn(Optional.of(new Node(1, "localhost", 9999))); + when(mockMembershipManager.shouldSendHeartbeat()).thenReturn(true); + NetworkClientDelegate.PollResult result = heartbeatRequestManager.poll(mockTime.milliseconds()); + assertEquals(1, result.unsentRequests.size()); + result.unsentRequests.get(0).future().completeExceptionally(new TimeoutException("timeout")); + + // assure the manager will backoff on timeout + mockTime.sleep(retryBackoffMs - 1); + result = heartbeatRequestManager.poll(mockTime.milliseconds()); + assertEquals(0, result.unsentRequests.size()); + + mockTime.sleep(1); + result = heartbeatRequestManager.poll(mockTime.milliseconds()); + assertEquals(1, result.unsentRequests.size()); + } + Review Comment: Yep, thanks. ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java: ########## @@ -0,0 +1,384 @@ +/* + * 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.errors.GroupAuthorizationException; +import org.apache.kafka.common.errors.RetriableException; +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; + +/** + * <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 will try to send a heartbeat when the member is in {@link MemberState#STABLE}, + * {@link MemberState#UNJOINED}, or {@link MemberState#RECONCILING}. Which mean the member is either in a stable + * group, is trying to join a group, or is in the process of reconciling the assignment changes.</p> + * + * <p>If the member got kick out of a group, it will try to give up the current assignment by invoking {@code + * OnPartitionsLost} because reattempting to join again with a zero epoch.</p> + * + * <p>If the member does not have groupId configured or encountering fatal exceptions, a 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>When the member completes the assignment reconciliation, the {@link HeartbeatRequestState} will be reset so + * that a heartbeat 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 Time time; + + /** + * Time that the group coordinator will wait on member to revoke its partitions. This is provided by the group + * coordinator in the heartbeat + */ + private final int rebalanceTimeoutMs; + + /** + * CoordinatorRequestManager manages the connection to the group coordinator + */ + private final CoordinatorRequestManager coordinatorRequestManager; + + /** + * SubscriptionState tracks the topic, partition and offset of the member + */ + private final SubscriptionState subscriptions; + + /** + * HeartbeatRequestState manages heartbeat request timing and retries + */ + private final HeartbeatRequestState heartbeatRequestState; + + /** + * MembershipManager manages member's essential attributes like epoch and id, and its rebalance state + */ + private final MembershipManager membershipManager; + + /** + * ErrorEventHandler allows the background thread to propagate errors back to the user + */ + 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(getClass()); + this.subscriptions = subscriptions; + this.membershipManager = membershipManager; + this.nonRetriableErrorHandler = nonRetriableErrorHandler; + this.rebalanceTimeoutMs = config.getInt(CommonClientConfigs.MAX_POLL_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, 0, retryBackoffMs, + retryBackoffMaxMs, rebalanceTimeoutMs); + } + + // Visible for testing + HeartbeatRequestManager( + final LogContext logContext, + final Time time, + final ConsumerConfig config, + final CoordinatorRequestManager coordinatorRequestManager, + final SubscriptionState subscriptions, + final MembershipManager membershipManager, + final HeartbeatRequestState heartbeatRequestState, + final ErrorEventHandler nonRetriableErrorHandler) { + this.logger = logContext.logger(this.getClass()); + this.time = time; + 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; + } + + /** + * Determines the maximum wait time until the next poll based on the member's state, and creates a heartbeat + * request. + * <ol> + * <li>If the member is without a coordinator or is in a failed state, the timer is set to Long.MAX_VALUE, as there's no need to send a heartbeat.</li> + * <li>If the member cannot send a heartbeat due to either exponential backoff, it will return the remaining time left on the backoff timer.</li> + * <li>If the member's heartbeat timer has not expired, It will return the remaining time left on the + * heartbeat timer.</li> + * <li>If the member can send a heartbeat, the timer is set to the current heartbeat interval.</li> + * </ol> + */ + @Override + public NetworkClientDelegate.PollResult poll(long currentTimeMs) { + if (!coordinatorRequestManager.coordinator().isPresent() || !membershipManager.shouldSendHeartbeat()) { + return new NetworkClientDelegate.PollResult( + Long.MAX_VALUE, Collections.emptyList()); + } + + // TODO: We will need to send a heartbeat response after partitions being revoke. This needs to be + // implemented either with or after the partition reconciliation logic. + if (!heartbeatRequestState.canSendRequest(currentTimeMs)) { + return new NetworkClientDelegate.PollResult( + heartbeatRequestState.nextHeartbeatMs(currentTimeMs), + Collections.emptyList()); + } + this.heartbeatRequestState.onSendAttempt(currentTimeMs); + NetworkClientDelegate.UnsentRequest request = makeHeartbeatRequest(); + return new NetworkClientDelegate.PollResult(heartbeatRequestState.heartbeatIntervalMs, Collections.singletonList(request)); + } + + private NetworkClientDelegate.UnsentRequest makeHeartbeatRequest() { + // TODO: We only need to send the rebalanceTimeoutMs field once unless the first request failed. + ConsumerGroupHeartbeatRequestData data = new ConsumerGroupHeartbeatRequestData() + .setGroupId(membershipManager.groupId()) + .setMemberEpoch(membershipManager.memberEpoch()) + .setMemberId(membershipManager.memberId()) + .setRebalanceTimeoutMs(rebalanceTimeoutMs); + + membershipManager.groupInstanceId().ifPresent(data::setInstanceId); + + if (this.subscriptions.hasPatternSubscription()) { + // TODO: Pass the string to the GC if server side regex is used. + } else { + data.setSubscribedTopicNames(new ArrayList<>(this.subscriptions.subscription())); + } + + this.membershipManager.assignorSelection().serverAssignor().ifPresent(data::setServerAssignor); + + NetworkClientDelegate.UnsentRequest request = new NetworkClientDelegate.UnsentRequest( + new ConsumerGroupHeartbeatRequest.Builder(data), + coordinatorRequestManager.coordinator()); + request.future().whenComplete((response, exception) -> { + if (response != null) { + onResponse((ConsumerGroupHeartbeatResponse) response.responseBody(), response.receivedTimeMs()); + } else { + // TODO: Currently, we lack a good way to propage the response time from the network client to the + // request handler. We will need to store the response time in the handler to make it accessible. + onFailure(exception, time.milliseconds()); + } + }); + return request; + } + + private void onFailure(final Throwable exception, final long responseTimeMs) { + this.heartbeatRequestState.onFailedAttempt(responseTimeMs); + logger.warn("Failed to send heartbeat to coordinator node {} due to error: {}", + coordinatorRequestManager.coordinator(), exception.getMessage()); + if (exception instanceof RetriableException) { + logger.debug("Retrying heartbeat request in {}ms due to {}", + heartbeatRequestState.remainingBackoffMs(responseTimeMs), + exception.getMessage()); + } else { + logger.error("Heartbeat request failed due to fatal error", exception); + membershipManager.transitionToFailed(); + nonRetriableErrorHandler.handle(exception); + } + } + + private void onResponse(final ConsumerGroupHeartbeatResponse response, long currentTimeMs) { + if (Errors.forCode(response.data().errorCode()) == Errors.NONE) { + this.heartbeatRequestState.updateHeartbeatIntervalMs(response.data().heartbeatIntervalMs()); + this.heartbeatRequestState.onSuccessfulAttempt(currentTimeMs); + this.heartbeatRequestState.resetTimer(); + this.membershipManager.updateState(response.data()); + return; + } + onErrorResponse(response, currentTimeMs); + } + + private void onErrorResponse(final ConsumerGroupHeartbeatResponse response, + final long currentTimeMs) { + Errors error = Errors.forCode(response.data().errorCode()); + String errorMessage = response.data().errorMessage(); + // TODO: upon encountering a fatal/fenced error, trigger onPartitionLost logic to give up the current + // assignments. + switch (error) { + case NOT_COORDINATOR: + // the manager should retry immediately when the coordinator node becomes available again + String message = String.format("GroupHeartbeatRequest failed because the group coordinator %s is incorrect. " + + "Will attempt to find the coordinator again and retry", + coordinatorRequestManager.coordinator()); + logInfo(message, response, currentTimeMs); + coordinatorRequestManager.markCoordinatorUnknown(errorMessage, currentTimeMs); + break; + case COORDINATOR_NOT_AVAILABLE: + message = String.format("GroupHeartbeatRequest failed because the group coordinator %s is not available. " + + "Will attempt to find the coordinator again and retry", + coordinatorRequestManager.coordinator()); + logInfo(message, response, currentTimeMs); + coordinatorRequestManager.markCoordinatorUnknown(errorMessage, currentTimeMs); + break; + + case COORDINATOR_LOAD_IN_PROGRESS: + // the manager will backoff and retry + message = String.format("GroupHeartbeatRequest failed because the group coordinator %s is still loading." + + "Will retry", + coordinatorRequestManager.coordinator()); + logInfo(message, response, currentTimeMs); + heartbeatRequestState.onFailedAttempt(currentTimeMs); + break; + + case GROUP_AUTHORIZATION_FAILED: + GroupAuthorizationException exception = + GroupAuthorizationException.forGroupId(membershipManager.groupId()); + logger.error("GroupHeartbeatRequest failed due to group authorization failure: {}", exception.getMessage()); + handleFatalFailure(error.exception(exception.getMessage())); + break; + + case UNRELEASED_INSTANCE_ID: + logger.error("GroupHeartbeatRequest failed due to the instance id {} was not released: {}", + membershipManager.groupInstanceId().orElse("null"), errorMessage); + handleFatalFailure(Errors.UNRELEASED_INSTANCE_ID.exception(errorMessage)); + break; + + case INVALID_REQUEST: + case GROUP_MAX_SIZE_REACHED: + case UNSUPPORTED_ASSIGNOR: + case UNSUPPORTED_VERSION: + logger.error("GroupHeartbeatRequest failed due to error: {}", error); + handleFatalFailure(error.exception(errorMessage)); + break; + + case FENCED_MEMBER_EPOCH: + message = String.format("GroupHeartbeatRequest failed because member epoch %s is invalid. " + + "Will abandon all partitions and rejoin the group", + membershipManager.memberId(), membershipManager.memberEpoch()); + logInfo(message, response, currentTimeMs); + membershipManager.transitionToFenced(); + break; + case UNKNOWN_MEMBER_ID: Review Comment: nit: empty line. -- 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