Skip to content

Commit

Permalink
KAFKA-18829: Added check before converting to IMPLICIT mode (#18964)
Browse files Browse the repository at this point in the history
Reviewers: Andrew Schofield <aschofield@confluent.io>
  • Loading branch information
ShivsundarR authored Feb 19, 2025
1 parent 6aab304 commit 3603c8f
Show file tree
Hide file tree
Showing 2 changed files with 40 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1018,8 +1018,8 @@ private void acknowledgeBatchIfImplicitAcknowledgement(boolean calledOnPoll) {
if (acknowledgementMode == AcknowledgementMode.UNKNOWN) {
// The first call to poll(Duration) moves into PENDING
acknowledgementMode = AcknowledgementMode.PENDING;
} else if (acknowledgementMode == AcknowledgementMode.PENDING) {
// The second call to poll(Duration) if PENDING moves into IMPLICIT
} else if (acknowledgementMode == AcknowledgementMode.PENDING && !currentFetch.isEmpty()) {
// If there are records to acknowledge and PENDING, moves into IMPLICIT
acknowledgementMode = AcknowledgementMode.IMPLICIT;
}
} else {
Expand Down
38 changes: 38 additions & 0 deletions core/src/test/java/kafka/test/api/ShareConsumerTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -688,6 +688,44 @@ public void testExplicitAcknowledgementCommitAsync() throws InterruptedException
}
}

@ClusterTest
public void testImplicitModeNotTriggeredByPollWhenNoAcksToSend() throws InterruptedException {
setup();
alterShareAutoOffsetReset("group1", "earliest");
try (Producer<byte[], byte[]> producer = createProducer();
ShareConsumer<byte[], byte[]> shareConsumer = createShareConsumer("group1")) {

shareConsumer.subscribe(Collections.singleton(tp.topic()));

Map<TopicPartition, Set<Long>> partitionOffsetsMap1 = new HashMap<>();
Map<TopicPartition, Exception> partitionExceptionMap1 = new HashMap<>();
shareConsumer.setAcknowledgementCommitCallback(new TestableAcknowledgementCommitCallback(partitionOffsetsMap1, partitionExceptionMap1));

// The acknowledgement mode moves to PENDING from UNKNOWN.
ConsumerRecords<byte[], byte[]> records = shareConsumer.poll(Duration.ofMillis(5000));
assertEquals(0, records.count());
shareConsumer.commitAsync();

ProducerRecord<byte[], byte[]> record1 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
producer.send(record1);
producer.flush();

// The acknowledgement mode remains in PENDING because no records were returned.
records = shareConsumer.poll(Duration.ofMillis(5000));
assertEquals(1, records.count());

// The acknowledgement mode now moves to EXPLICIT.
shareConsumer.acknowledge(records.iterator().next());
shareConsumer.commitAsync();

TestUtils.waitForCondition(() -> {
shareConsumer.poll(Duration.ofMillis(500));
return partitionExceptionMap1.containsKey(tp);
}, 30000, 100L, () -> "Didn't receive call to callback");
verifyShareGroupStateTopicRecordsProduced();
}
}

@ClusterTest
public void testExplicitAcknowledgementCommitAsyncPartialBatch() {
setup();
Expand Down

0 comments on commit 3603c8f

Please sign in to comment.