Skip to content

Commit

Permalink
GH-3166: BatchInterceptor issues with retries
Browse files Browse the repository at this point in the history
Fixes: #3166

* When retries are enabled, batch interceptor is not
  invoking the intercept methods for failures on retries
  and the possible eventual success method call.
  Addressing this issue.

* Addressing PR review

(cherry picked from commit c26f26e)
  • Loading branch information
sobychacko authored and spring-builds committed Mar 28, 2024
1 parent 9361d0f commit 7735794
Show file tree
Hide file tree
Showing 2 changed files with 93 additions and 16 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -2272,12 +2272,8 @@ private List<ConsumerRecord<K, V>> createRecordList(final ConsumerRecords<K, V>
@Nullable
private RuntimeException doInvokeBatchListener(final ConsumerRecords<K, V> records, // NOSONAR
List<ConsumerRecord<K, V>> recordList) {

Object sample = startMicrometerSample();
try {
invokeBatchOnMessage(records, recordList);
batchInterceptAfter(records, null);
successTimer(sample, null);
if (this.batchFailed) {
this.batchFailed = false;
if (this.commonErrorHandler != null) {
Expand All @@ -2290,9 +2286,6 @@ private RuntimeException doInvokeBatchListener(final ConsumerRecords<K, V> recor
}
}
catch (RuntimeException e) {
this.batchFailed = true;
failureTimer(sample, null);
batchInterceptAfter(records, e);
if (this.commonErrorHandler == null) {
throw e;
}
Expand Down Expand Up @@ -2446,15 +2439,26 @@ private void invokeBatchOnMessageWithRecordsOrList(final ConsumerRecords<K, V> r
recordList = createRecordList(records);
}
}
if (this.wantsFullRecords) {
this.batchListener.onMessage(records, // NOSONAR
this.isAnyManualAck
? new ConsumerBatchAcknowledgment(records, recordList)
: null,
this.consumer);
Object sample = startMicrometerSample();
try {
if (this.wantsFullRecords) {
this.batchListener.onMessage(records, // NOSONAR
this.isAnyManualAck
? new ConsumerBatchAcknowledgment(records, recordList)
: null,
this.consumer);
}
else {
doInvokeBatchOnMessage(records, recordList); // NOSONAR
}
batchInterceptAfter(records, null);
successTimer(sample, null);
}
else {
doInvokeBatchOnMessage(records, recordList); // NOSONAR
catch (RuntimeException e) {
this.batchFailed = true;
failureTimer(sample, null);
batchInterceptAfter(records, e);
throw e;
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4021,7 +4021,6 @@ public ConsumerRecords<Integer, String> intercept(ConsumerRecords<Integer, Strin
inOrder.verify(interceptor).setupThreadState(eq(consumer));
inOrder.verify(consumer).poll(Duration.ofMillis(ContainerProperties.DEFAULT_POLL_TIMEOUT));
inOrder.verify(interceptor).intercept(any(), eq(consumer));
inOrder.verify(interceptor).success(any(), eq(consumer));
inOrder.verify(consumer).commitSync(eq(Map.of(new TopicPartition("foo", 0), new OffsetAndMetadata(2L))),
any(Duration.class));
container.stop();
Expand Down Expand Up @@ -4242,6 +4241,80 @@ public void clearThreadState(Consumer<?, ?> consumer) {
container.stop();
}

@Test
@SuppressWarnings("unchecked")
public void invokeBatchInterceptorSuccessFailureOnRetry() throws Exception {
ConsumerFactory<Integer, String> cf = mock(ConsumerFactory.class);
Consumer<Integer, String> consumer = mock(Consumer.class);
given(cf.createConsumer(eq("grp"), eq("clientId"), isNull(), any())).willReturn(consumer);
ConsumerRecord<Integer, String> firstRecord = new ConsumerRecord<>("test-topic", 0, 0L, 1, "data-1");
ConsumerRecord<Integer, String> secondRecord = new ConsumerRecord<>("test-topic", 0, 1L, 1, "data-2");
Map<TopicPartition, List<ConsumerRecord<Integer, String>>> records = new HashMap<>();
records.put(new TopicPartition("test-topic", 0), List.of(firstRecord, secondRecord));
ConsumerRecords<Integer, String> consumerRecords = new ConsumerRecords<>(records);
AtomicInteger invocation = new AtomicInteger(0);
given(consumer.poll(any(Duration.class))).willAnswer(i -> {
if (invocation.getAndIncrement() == 0) {
return consumerRecords;
}
else {
// Subsequent polls after the first one returns empty records.
return new ConsumerRecords<Integer, String>(Map.of());
}
});
TopicPartitionOffset[] topicPartition = new TopicPartitionOffset[] {
new TopicPartitionOffset("test-topic", 0) };

CountDownLatch latch = new CountDownLatch(4); // 3 failures, 1 success
BatchMessageListener<Integer, String> batchMessageListener = spy(
new BatchMessageListener<Integer, String>() { // Cannot be lambda: Mockito doesn't mock final classes

@Override
public void onMessage(List<ConsumerRecord<Integer, String>> data) {
latch.countDown();
if (latch.getCount() > 0) {
throw new IllegalArgumentException("Failed record");
}
}

});

ContainerProperties containerProps = new ContainerProperties(topicPartition);
containerProps.setGroupId("grp");
containerProps.setAckMode(ContainerProperties.AckMode.BATCH);
containerProps.setMissingTopicsFatal(false);
containerProps.setMessageListener(batchMessageListener);
containerProps.setClientId("clientId");

BatchInterceptor<Integer, String> batchInterceptor = spy(new BatchInterceptor<Integer, String>() {

@Override
public ConsumerRecords<Integer, String> intercept(ConsumerRecords<Integer, String> records,
Consumer<Integer, String> consumer) {
return records;
}

});

KafkaMessageListenerContainer<Integer, String> container =
new KafkaMessageListenerContainer<>(cf, containerProps);
container.setCommonErrorHandler(new DefaultErrorHandler(new FixedBackOff(0, 3)));
container.setBatchInterceptor(batchInterceptor);
container.start();
assertThat(latch.await(10, TimeUnit.SECONDS)).isTrue();

InOrder inOrder = inOrder(batchInterceptor, batchMessageListener, consumer);
for (int i = 0; i < 3; i++) {
inOrder.verify(batchInterceptor).intercept(eq(consumerRecords), eq(consumer));
inOrder.verify(batchMessageListener).onMessage(eq(List.of(firstRecord, secondRecord)));
inOrder.verify(batchInterceptor).failure(eq(consumerRecords), any(), eq(consumer));
}
inOrder.verify(batchInterceptor).intercept(eq(consumerRecords), eq(consumer));
inOrder.verify(batchMessageListener).onMessage(eq(List.of(firstRecord, secondRecord)));
inOrder.verify(batchInterceptor).success(eq(consumerRecords), eq(consumer));
container.stop();
}

@Test
public void testOffsetAndMetadataWithoutProvider() throws InterruptedException {
testOffsetAndMetadata(null, new OffsetAndMetadata(1));
Expand Down

0 comments on commit 7735794

Please sign in to comment.