frankvicky commented on code in PR #19914: URL: https://github.com/apache/kafka/pull/19914#discussion_r2203213180
########## clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java: ########## @@ -452,6 +452,37 @@ private void testPositionAndCommit(GroupProtocol groupProtocol) throws Interrupt } } + @ClusterTest + public void testClearBrokerBeforeConsumerCloses() throws InterruptedException { + // This is testing when closing the consumer but commit request has already been sent. + // During the closing, the consumer won't find the coordinator anymore. + + // Create offsets topic to ensure coordinator is available during close + cluster.createTopic(Topic.GROUP_METADATA_TOPIC_NAME, Integer.parseInt(OFFSETS_TOPIC_PARTITIONS), Short.parseShort(OFFSETS_TOPIC_REPLICATION)); + + try (Producer<byte[], byte[]> producer = cluster.producer(Map.of(ProducerConfig.ACKS_CONFIG, "all")); + var consumer = createConsumer(GroupProtocol.CONSUMER, false) + ) { + sendRecords(producer, tp, 3, System.currentTimeMillis()); + sendRecords(producer, tp1, 3, System.currentTimeMillis()); + consumer.assign(List.of(tp, tp1)); + + // Try without looking up the coordinator first + var cb = new CountConsumerCommitCallback(); + + // Close the coordinator before committing because otherwise the commit will fail to find the coordinator. + Set<Integer> brokerIds = cluster.brokerIds(); + brokerIds.forEach(cluster::shutdownBroker); Review Comment: ```suggestion cluster.brokerIds().forEach(cluster::shutdownBroker); ``` ########## clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java: ########## @@ -452,6 +452,37 @@ private void testPositionAndCommit(GroupProtocol groupProtocol) throws Interrupt } } + @ClusterTest + public void testClearBrokerBeforeConsumerCloses() throws InterruptedException { + // This is testing when closing the consumer but commit request has already been sent. + // During the closing, the consumer won't find the coordinator anymore. + + // Create offsets topic to ensure coordinator is available during close + cluster.createTopic(Topic.GROUP_METADATA_TOPIC_NAME, Integer.parseInt(OFFSETS_TOPIC_PARTITIONS), Short.parseShort(OFFSETS_TOPIC_REPLICATION)); Review Comment: We don't need this since the top-level annotation has already created it. ########## clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java: ########## @@ -452,6 +452,37 @@ private void testPositionAndCommit(GroupProtocol groupProtocol) throws Interrupt } } + @ClusterTest + public void testClearBrokerBeforeConsumerCloses() throws InterruptedException { + // This is testing when closing the consumer but commit request has already been sent. + // During the closing, the consumer won't find the coordinator anymore. Review Comment: Please change it to the method-level javadoc. ########## clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java: ########## @@ -452,6 +452,37 @@ private void testPositionAndCommit(GroupProtocol groupProtocol) throws Interrupt } } + @ClusterTest + public void testClearBrokerBeforeConsumerCloses() throws InterruptedException { Review Comment: This test name is not accurate. ########## clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java: ########## @@ -452,6 +452,37 @@ private void testPositionAndCommit(GroupProtocol groupProtocol) throws Interrupt } } + @ClusterTest + public void testClearBrokerBeforeConsumerCloses() throws InterruptedException { + // This is testing when closing the consumer but commit request has already been sent. + // During the closing, the consumer won't find the coordinator anymore. + + // Create offsets topic to ensure coordinator is available during close + cluster.createTopic(Topic.GROUP_METADATA_TOPIC_NAME, Integer.parseInt(OFFSETS_TOPIC_PARTITIONS), Short.parseShort(OFFSETS_TOPIC_REPLICATION)); + + try (Producer<byte[], byte[]> producer = cluster.producer(Map.of(ProducerConfig.ACKS_CONFIG, "all")); + var consumer = createConsumer(GroupProtocol.CONSUMER, false) + ) { + sendRecords(producer, tp, 3, System.currentTimeMillis()); + sendRecords(producer, tp1, 3, System.currentTimeMillis()); + consumer.assign(List.of(tp, tp1)); + + // Try without looking up the coordinator first + var cb = new CountConsumerCommitCallback(); + + // Close the coordinator before committing because otherwise the commit will fail to find the coordinator. + Set<Integer> brokerIds = cluster.brokerIds(); + brokerIds.forEach(cluster::shutdownBroker); + + consumer.commitAsync(Map.of(tp, new OffsetAndMetadata(1L)), cb); + consumer.commitAsync(Map.of(tp1, new OffsetAndMetadata(1L)), cb); + + consumer.close(); + assertTrue(cb.lastError.get() instanceof CommitFailedException); Review Comment: ```suggestion assertTrue(cb.lastError.isPresent()); assertInstanceOf(CommitFailedException.class, cb.lastError.get()); ``` ########## clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/PlaintextConsumerCommitTest.java: ########## @@ -452,6 +452,37 @@ private void testPositionAndCommit(GroupProtocol groupProtocol) throws Interrupt } } + @ClusterTest + public void testClearBrokerBeforeConsumerCloses() throws InterruptedException { + // This is testing when closing the consumer but commit request has already been sent. + // During the closing, the consumer won't find the coordinator anymore. + + // Create offsets topic to ensure coordinator is available during close + cluster.createTopic(Topic.GROUP_METADATA_TOPIC_NAME, Integer.parseInt(OFFSETS_TOPIC_PARTITIONS), Short.parseShort(OFFSETS_TOPIC_REPLICATION)); + + try (Producer<byte[], byte[]> producer = cluster.producer(Map.of(ProducerConfig.ACKS_CONFIG, "all")); + var consumer = createConsumer(GroupProtocol.CONSUMER, false) + ) { + sendRecords(producer, tp, 3, System.currentTimeMillis()); + sendRecords(producer, tp1, 3, System.currentTimeMillis()); + consumer.assign(List.of(tp, tp1)); + + // Try without looking up the coordinator first + var cb = new CountConsumerCommitCallback(); Review Comment: nit: ```suggestion var callback = new CountConsumerCommitCallback(); ``` -- 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