Skip to content

Commit

Permalink
SeekToCurrentErrorHandler and Recovery
Browse files Browse the repository at this point in the history
Don't throw an exception if the record was skipped by invoking the recoverer.

- Adds noise to the log since the record was recovered in some way.
  • Loading branch information
garyrussell authored and artembilan committed Aug 28, 2018
1 parent 9997ffd commit e16ae4f
Show file tree
Hide file tree
Showing 3 changed files with 47 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -78,8 +78,9 @@ public SeekToCurrentErrorHandler(@Nullable BiConsumer<ConsumerRecord<?, ?>, Exce
@Override
public void handle(Exception thrownException, List<ConsumerRecord<?, ?>> records,
Consumer<?, ?> consumer, MessageListenerContainer container) {
SeekUtils.doSeeks(records, consumer, thrownException, true, this.failureTracker::skip, logger);
throw new KafkaException("Seek to current after exception", thrownException);
if (!SeekUtils.doSeeks(records, consumer, thrownException, true, this.failureTracker::skip, logger)) {
throw new KafkaException("Seek to current after exception", thrownException);
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,25 +53,39 @@ private SeekUtils() {
* @param recoverable true if skipping the first record is allowed.
* @param skipper function to determine whether or not to skip seeking the first.
* @param logger a {@link Log} for seek errors.
* @return true if the failed record was skipped.
*/
public static void doSeeks(List<ConsumerRecord<?, ?>> records, Consumer<?, ?> consumer, Exception exception,
public static boolean doSeeks(List<ConsumerRecord<?, ?>> records, Consumer<?, ?> consumer, Exception exception,
boolean recoverable, BiPredicate<ConsumerRecord<?, ?>, Exception> skipper, Log logger) {
Map<TopicPartition, Long> partitions = new LinkedHashMap<>();
AtomicBoolean first = new AtomicBoolean(true);
AtomicBoolean skipped = new AtomicBoolean();
records.forEach(record -> {
if (!recoverable || !first.get() || !skipper.test(record, exception)) {
partitions.computeIfAbsent(new TopicPartition(record.topic(), record.partition()), offset -> record.offset());
if (recoverable && first.get()) {
skipped.set(skipper.test(record, exception));
if (skipped.get() && logger.isDebugEnabled()) {
logger.debug("Skipping seek of: " + record);
}
}
if (!recoverable || !first.get() || !skipped.get()) {
partitions.computeIfAbsent(new TopicPartition(record.topic(), record.partition()),
offset -> record.offset());
}
first.set(false);
});
boolean tracing = logger.isTraceEnabled();
partitions.forEach((topicPartition, offset) -> {
try {
if (tracing) {
logger.trace("Seeking: " + topicPartition + " to: " + offset);
}
consumer.seek(topicPartition, offset);
}
catch (Exception e) {
logger.error("Failed to seek " + topicPartition + " to " + offset, e);
}
});
return skipped.get();
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,14 @@
package org.springframework.kafka.listener;

import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.fail;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;

import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
Expand All @@ -33,6 +38,7 @@
import org.junit.ClassRule;
import org.junit.Test;

import org.springframework.kafka.KafkaException;
import org.springframework.kafka.core.DefaultKafkaConsumerFactory;
import org.springframework.kafka.core.DefaultKafkaProducerFactory;
import org.springframework.kafka.core.KafkaTemplate;
Expand Down Expand Up @@ -121,4 +127,25 @@ public void accept(ConsumerRecord<?, ?> record, Exception exception) {
verify(errorHandler).clearThreadState();
}

@Test
public void seekToCurrentErrorHandlerRecovers() {
SeekToCurrentErrorHandler eh = new SeekToCurrentErrorHandler((r, e) -> { }, 2);
List<ConsumerRecord<?, ?>> records = new ArrayList<>();
records.add(new ConsumerRecord<>("foo", 0, 0, null, "foo"));
records.add(new ConsumerRecord<>("foo", 0, 1, null, "bar"));
Consumer<?, ?> consumer = mock(Consumer.class);
try {
eh.handle(new RuntimeException(), records, consumer, null);
fail("Expected exception");
}
catch (KafkaException e) {
// NOSONAR
}
verify(consumer).seek(new TopicPartition("foo", 0), 0L);
verifyNoMoreInteractions(consumer);
eh.handle(new RuntimeException(), records, consumer, null);
verify(consumer).seek(new TopicPartition("foo", 0), 1L);
verifyNoMoreInteractions(consumer);
}

}

0 comments on commit e16ae4f

Please sign in to comment.