diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index d5b1a4b956d..830f071c08a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -874,10 +874,6 @@ public class KafkaConsumer implements Consumer { public void unsubscribe() { acquire(); try { - // make sure the offsets of topic partitions the consumer is unsubscribing from - // are committed since there will be no following rebalance - this.coordinator.maybeAutoCommitOffsetsNow(); - log.debug("Unsubscribed all topics or patterns and assigned partitions"); this.subscriptions.unsubscribe(); this.coordinator.maybeLeaveGroup(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 0096e728381..b1c69625dcb 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -614,8 +614,7 @@ public class KafkaConsumerTest { * do not immediately change, and the latest consumed offsets of its to-be-revoked * partitions are properly committed (when auto-commit is enabled). * Upon unsubscribing from subscribed topics the consumer subscription and assignment - * are both updated right away and its consumed offsets are committed (if auto-commit - * is enabled). + * are both updated right away but its consumed offsets are not auto committed. */ @Test public void testSubscriptionChangesWithAutoCommitEnabled() { @@ -722,21 +721,12 @@ public class KafkaConsumerTest { assertTrue(consumer.assignment().size() == 2); assertTrue(consumer.assignment().contains(tp0) && consumer.assignment().contains(t3p0)); - // mock the offset commit response for to be revoked partitions - Map partitionOffsets2 = new HashMap<>(); - partitionOffsets2.put(tp0, 2L); - partitionOffsets2.put(t3p0, 100L); - commitReceived = prepareOffsetCommitResponse(client, coordinator, partitionOffsets2); - consumer.unsubscribe(); // verify that subscription and assignment are both cleared assertTrue(consumer.subscription().isEmpty()); assertTrue(consumer.assignment().isEmpty()); - // verify that the offset commits occurred as expected - assertTrue(commitReceived.get()); - consumer.close(); }