Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -357,10 +357,41 @@ public void testAcknowledgementCommitCallbackSuccessfulAcknowledgement() throws
TestUtils.waitForCondition(() -> shareConsumer.poll(Duration.ofMillis(2000)).count() == 1,
DEFAULT_MAX_WAIT_MS, 100L, () -> "Failed to consume records for share consumer");

TestUtils.waitForCondition(() -> {
shareConsumer.poll(Duration.ofMillis(500));
return partitionOffsetsMap.containsKey(tp);
}, DEFAULT_MAX_WAIT_MS, 100L, () -> "Failed to receive call to callback");
// The callback should be called before the return of the poll, even when there are no more records.
ConsumerRecords<byte[], byte[]> records = shareConsumer.poll(Duration.ofMillis(2000));
assertEquals(0, records.count());
assertTrue(partitionOffsetsMap.containsKey(tp));

// We expect no exception as the acknowledgement error code is null.
assertFalse(partitionExceptionMap.containsKey(tp));
verifyShareGroupStateTopicRecordsProduced();
}
}

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

Map<TopicPartition, Set<Long>> partitionOffsetsMap = new HashMap<>();
Map<TopicPartition, Exception> partitionExceptionMap = new HashMap<>();
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());

producer.send(record);
producer.flush();

shareConsumer.setAcknowledgementCommitCallback(new TestableAcknowledgementCommitCallback(partitionOffsetsMap, partitionExceptionMap));
shareConsumer.subscribe(Set.of(tp.topic()));

TestUtils.waitForCondition(() -> shareConsumer.poll(Duration.ofMillis(2000)).count() == 1,
DEFAULT_MAX_WAIT_MS, 100L, () -> "Failed to consume records for share consumer");

// The acknowledgement commit callback should be called before the commitSync returns
// once the records have been confirmed to have been acknowledged.
Map<TopicIdPartition, Optional<KafkaException>> result = shareConsumer.commitSync();
assertEquals(1, result.size());
assertTrue(partitionOffsetsMap.containsKey(tp));

// We expect no exception as the acknowledgement error code is null.
assertFalse(partitionExceptionMap.containsKey(tp));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -609,6 +609,15 @@ public synchronized ConsumerRecords<K, V> poll(final Duration timeout) {
throw e.cause();
} finally {
kafkaShareConsumerMetrics.recordPollEnd(timer.currentTimeMs());

// Handle any acknowledgements which completed while we were waiting, but do not throw
// the exception because the fetched records would then not be returned to the caller
try {
handleCompletedAcknowledgements(false);
} catch (Throwable t) {
log.warn("Exception thrown in acknowledgement commit callback", t);
}
Comment on lines +615 to +619
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I find this change make testShareGroupMaxSizeConfigExceeded become flaky.
The following code will fix this issue but seem to violate the comments.

// Handle any acknowledgements which completed while we were waiting, but do not throw
// the exception because the fetched records would then not be returned to the caller
try {
    handleCompletedAcknowledgements(false);
} catch (GroupMaxSizeReachedException e) {
    log.warn("Exception thrown in acknowledgement commit callback", e);
    throw e;
} catch (Throwable t) {
    log.warn("Exception thrown in acknowledgement commit callback", t);
}

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@TaiJuWu thanks for this find. The flaky behaviour and its root cause have already been logged in https://issues.apache.org/jira/browse/KAFKA-19840


release();
}
}
Expand Down Expand Up @@ -752,8 +761,11 @@ public Map<TopicIdPartition, Optional<KafkaException>> commitSync(final Duration
result.put(tip, Optional.of(exception));
}
});
return result;

// Handle any acknowledgements which completed while we were waiting
handleCompletedAcknowledgements(false);

return result;
} finally {
wakeupTrigger.clearTask();
}
Expand Down