TaiJuWu commented on code in PR #19822: URL: https://github.com/apache/kafka/pull/19822#discussion_r2160657018
########## clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ConsumerBounceTest.java: ########## @@ -0,0 +1,816 @@ +/* + * 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; + +import kafka.server.KafkaBroker; + +import org.apache.kafka.clients.ClientsTestUtils; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.common.errors.GroupMaxSizeReachedException; +import org.apache.kafka.common.message.FindCoordinatorRequestData; +import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.requests.FindCoordinatorRequest; +import org.apache.kafka.common.requests.FindCoordinatorResponse; +import org.apache.kafka.common.test.ClusterInstance; +import org.apache.kafka.common.test.TestUtils; +import org.apache.kafka.common.test.api.ClusterConfigProperty; +import org.apache.kafka.common.test.api.ClusterTest; +import org.apache.kafka.common.test.api.ClusterTestDefaults; +import org.apache.kafka.common.test.api.Type; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; +import org.apache.kafka.server.IntegrationTestUtils; +import org.apache.kafka.server.config.KRaftConfigs; +import org.apache.kafka.server.config.ReplicationConfigs; +import org.apache.kafka.server.config.ServerConfigs; +import org.apache.kafka.server.config.ServerLogConfigs; +import org.apache.kafka.server.util.ShutdownableThread; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.slf4j.Logger; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_PROTOCOL_CONFIG; +import static org.apache.kafka.common.test.TestUtils.SEEDED_RANDOM; +import static org.apache.kafka.common.test.TestUtils.randomSelect; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Integration tests for the consumer that cover basic usage as well as server failures + */ +@ClusterTestDefaults( + types = {Type.KRAFT}, + brokers = ConsumerBounceTest.BROKER_COUNT, + serverProperties = { + @ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "3"), // don't want to lose offset + @ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"), + @ClusterConfigProperty(key = GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, value = "10"), // set small enough session timeout + @ClusterConfigProperty(key = GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, value = "0"), + + // Tests will run for CONSUMER and CLASSIC group protocol, so set the group max size property + // required for each. + @ClusterConfigProperty(key = GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SIZE_CONFIG, value = ConsumerBounceTest.MAX_GROUP_SIZE), + @ClusterConfigProperty(key = GroupCoordinatorConfig.GROUP_MAX_SIZE_CONFIG, value = ConsumerBounceTest.MAX_GROUP_SIZE), + + @ClusterConfigProperty(key = ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, value = "false"), + @ClusterConfigProperty(key = ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, value = "100"), + @ClusterConfigProperty(key = ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, value = "false"), + @ClusterConfigProperty(key = TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, value = "true"), + @ClusterConfigProperty(key = TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, value = "1000"), + @ClusterConfigProperty(key = ReplicationConfigs.UNCLEAN_LEADER_ELECTION_INTERVAL_MS_CONFIG, value = "50"), + + @ClusterConfigProperty(key = KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_CONFIG, value = "50"), + @ClusterConfigProperty(key = KRaftConfigs.BROKER_SESSION_TIMEOUT_MS_CONFIG, value = "300") + } +) +public class ConsumerBounceTest { + + private final Logger logger = new LogContext("ConsumerBounceTest").logger(this.getClass()); + + public static final int BROKER_COUNT = 3; + public static final String MAX_GROUP_SIZE = "5"; + + private final Optional<Long> gracefulCloseTimeMs = Optional.of(1000L); + private final ScheduledExecutorService executor = Executors.newScheduledThreadPool(2); + private final String topic = "topic"; + private final int partition = 0; + private final int numPartitions = 3; + private final short numReplica = 3; + private final TopicPartition topicPartition = new TopicPartition(topic, partition); + + private final ClusterInstance clusterInstance; + + private final List<Consumer<byte[], byte[]>> consumers = new ArrayList<>(); + private final List<ConsumerAssignmentPoller> consumerPollers = new ArrayList<>(); + + ConsumerBounceTest(ClusterInstance clusterInstance) { + this.clusterInstance = clusterInstance; + } + + @BeforeEach + void setUp() throws InterruptedException { + consumerPollers.clear(); + consumers.clear(); + clusterInstance.createTopic(topic, numPartitions, numReplica); + } + + @AfterEach + void tearDown() throws InterruptedException { + consumerPollers.forEach(poller -> { + try { + poller.shutdown(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }); + executor.shutdownNow(); + // Wait for any active tasks to terminate to ensure consumer is not closed while being used from another thread + assertTrue(executor.awaitTermination(5000, TimeUnit.MILLISECONDS), "Executor did not terminate"); + consumers.forEach(Consumer::close); + } + + @ClusterTest + public void testClassicConsumerConsumptionWithBrokerFailures() throws Exception { + consumeWithBrokerFailures(10, GroupProtocol.CLASSIC); + } + + @ClusterTest + public void testAsyncConsumerConsumptionWithBrokerFailures() throws Exception { + consumeWithBrokerFailures(10, GroupProtocol.CONSUMER); + } + + /** + * 1. Produce a bunch of messages + * 2. Then consume the messages while killing and restarting brokers at random + */ + private void consumeWithBrokerFailures(int numIters, GroupProtocol groupProtocol) throws InterruptedException { + int numRecords = 1000; + ClientsTestUtils.sendRecords(clusterInstance, topicPartition, numRecords); + + AtomicInteger consumed = new AtomicInteger(0); + try (Consumer<byte[], byte[]> consumer = clusterInstance.consumer(Map.of(GROUP_PROTOCOL_CONFIG, groupProtocol.name))) { + + consumer.subscribe(List.of(topic)); + + BounceBrokerScheduler scheduler = new BounceBrokerScheduler(numIters, clusterInstance); + try { + scheduler.start(); + + while (scheduler.isRunning()) { + ConsumerRecords<byte[], byte[]> records = consumer.poll(Duration.ofMillis(100)); + + records.forEach(record -> { + assertEquals(consumed.get(), record.offset()); + consumed.incrementAndGet(); + }); + + if (!records.isEmpty()) { + consumer.commitSync(); + + long currentPosition = consumer.position(topicPartition); + long committedOffset = consumer.committed(Set.of(topicPartition)).get(topicPartition).offset(); + assertEquals(currentPosition, committedOffset); + + if (currentPosition == numRecords) { + consumer.seekToBeginning(List.of()); + consumed.set(0); + } + } + } + } finally { + scheduler.shutdown(); + } + } + } + + @ClusterTest + public void testClassicConsumerSeekAndCommitWithBrokerFailures() throws InterruptedException { + seekAndCommitWithBrokerFailures(5, GroupProtocol.CLASSIC); + } + + @ClusterTest + public void testAsyncConsumerSeekAndCommitWithBrokerFailures() throws InterruptedException { + seekAndCommitWithBrokerFailures(5, GroupProtocol.CONSUMER); + } + + private void seekAndCommitWithBrokerFailures(int numIters, GroupProtocol groupProtocol) throws InterruptedException { + int numRecords = 1000; + ClientsTestUtils.sendRecords(clusterInstance, topicPartition, numRecords); + + try (Consumer<byte[], byte[]> consumer = clusterInstance.consumer(Map.of(GROUP_PROTOCOL_CONFIG, groupProtocol.name))) { + consumer.assign(List.of(topicPartition)); + consumer.seek(topicPartition, 0); + + TestUtils.waitForCondition(() -> clusterInstance.brokers().values().stream().allMatch(broker -> + broker.replicaManager().localLog(topicPartition).get().highWatermark() == numRecords + ), 30000, "Failed to update high watermark for followers after timeout."); + + BounceBrokerScheduler scheduler = new BounceBrokerScheduler(numIters, clusterInstance); + try { + scheduler.start(); + + while (scheduler.isRunning()) { + int coin = SEEDED_RANDOM.nextInt(0, 3); + + if (coin == 0) { + logger.info("Seeking to end of log."); + consumer.seekToEnd(List.of()); + assertEquals(numRecords, consumer.position(topicPartition)); + } else if (coin == 1) { + int pos = SEEDED_RANDOM.nextInt(numRecords); + logger.info("Seeking to {}", pos); + consumer.seek(topicPartition, pos); + assertEquals(pos, consumer.position(topicPartition)); + } else { + logger.info("Committing offset."); + consumer.commitSync(); + assertEquals(consumer.position(topicPartition), consumer.committed(Set.of(topicPartition)).get(topicPartition).offset()); + } + } + } finally { + scheduler.shutdown(); + } + } + } + + @ClusterTest + public void testClassicSubscribeWhenTopicUnavailable() throws InterruptedException { + testSubscribeWhenTopicUnavailable(GroupProtocol.CLASSIC); + } + + @ClusterTest + public void testAsyncSubscribeWhenTopicUnavailable() throws InterruptedException { + testSubscribeWhenTopicUnavailable(GroupProtocol.CONSUMER); + } + + private void testSubscribeWhenTopicUnavailable(GroupProtocol groupProtocol) throws InterruptedException { + String newTopic = "new-topic"; + TopicPartition newTopicPartition = new TopicPartition(newTopic, 0); + int numRecords = 1000; + + Consumer<byte[], byte[]> consumer = clusterInstance.consumer(Map.of(GROUP_PROTOCOL_CONFIG, groupProtocol.name, "max.poll.interval.ms", 6000, "metadata.max.age.ms", 100)); + consumers.add(consumer); + consumer.subscribe(List.of(newTopic)); + consumer.poll(Duration.ZERO); + // Schedule topic creation after 2 seconds + executor.schedule(() -> assertDoesNotThrow(() -> clusterInstance.createTopic(newTopic, numPartitions, numReplica)), + 2, TimeUnit.SECONDS); + + // Start first poller + ConsumerAssignmentPoller poller = new ConsumerAssignmentPoller(consumer, List.of(newTopic)); + consumerPollers.add(poller); + poller.start(); + ClientsTestUtils.sendRecords(clusterInstance, newTopicPartition, numRecords); + receiveExactRecords(poller, numRecords, 60000L); + poller.shutdown(); + + // Simulate broker failure and recovery + clusterInstance.brokers().keySet().forEach(clusterInstance::shutdownBroker); + Thread.sleep(500); + clusterInstance.brokers().keySet().forEach(clusterInstance::startBroker); + + // Start second poller after recovery + ConsumerAssignmentPoller poller2 = new ConsumerAssignmentPoller(consumer, List.of(newTopic)); + consumerPollers.add(poller2); + poller2.start(); + + ClientsTestUtils.sendRecords(clusterInstance, newTopicPartition, numRecords); + receiveExactRecords(poller2, numRecords, 60000L); + } + + + @ClusterTest + public void testClassicClose() throws Exception { + testClose(GroupProtocol.CLASSIC); + } + + @ClusterTest + public void testAsyncClose() throws Exception { + testClose(GroupProtocol.CONSUMER); + } + + private void testClose(GroupProtocol groupProtocol) throws Exception { + int numRecords = 10; + ClientsTestUtils.sendRecords(clusterInstance, topicPartition, numRecords); + + checkCloseGoodPath(numRecords, "group1"); + checkCloseWithCoordinatorFailure(numRecords, "group2", "group3"); + checkCloseWithClusterFailure(numRecords, "group4", "group5", groupProtocol); + } + + /** + * Consumer is closed while cluster is healthy. Consumer should complete pending offset commits + * and leave group. New consumer instance should be able to join group and start consuming from + * last committed offset. + */ + private void checkCloseGoodPath(int numRecords, String groupId) throws InterruptedException { + Consumer<byte[], byte[]> consumer = createConsumerAndReceive(groupId, false, numRecords); + assertDoesNotThrow(() -> submitCloseAndValidate(consumer, Long.MAX_VALUE, Optional.empty(), gracefulCloseTimeMs).get()); + checkClosedState(groupId, numRecords); + } + + /** + * Consumer closed while coordinator is unavailable. Close of consumers using group + * management should complete after commit attempt even though commits fail due to rebalance. + * Close of consumers using manual assignment should complete with successful commits since a + * broker is available. + */ + private void checkCloseWithCoordinatorFailure(int numRecords, String dynamicGroup, String manualGroup) throws Exception { + Consumer<byte[], byte[]> dynamicConsumer = createConsumerAndReceive(dynamicGroup, false, numRecords); + Consumer<byte[], byte[]> manualConsumer = createConsumerAndReceive(manualGroup, true, numRecords); + + findCoordinators(List.of(dynamicGroup, manualGroup)).forEach(clusterInstance::shutdownBroker); Review Comment: We can't use `admin` to findCoordinator because the `group3` is not created on broker side. From my understanding, `consumer.assign` won't go through group manager so `group3` should not be created. Admin will throw `GroupIdNotFoundException` But when we send RPC directly, we can find the coordinator for `group3`. Im try finding the difference. You can run cfc0ee82907a34c217a5e00d8d3bb45b561c7cc8 to observe this behaviour. -- 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