package de.juplo.kafka;
+import lombok.RequiredArgsConstructor;
import lombok.extern.slf4j.Slf4j;
-import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.*;
import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.RecordDeserializationException;
import org.apache.kafka.common.errors.WakeupException;
-import org.apache.kafka.common.serialization.StringDeserializer;
import javax.annotation.PreDestroy;
+import java.time.Clock;
import java.time.Duration;
+import java.time.Instant;
import java.util.*;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
@Slf4j
-public class EndlessConsumer implements Runnable
+@RequiredArgsConstructor
+public class EndlessConsumer<K, V> implements ConsumerRebalanceListener, Runnable
{
private final ExecutorService executor;
- private final String bootstrapServer;
- private final String groupId;
+ private final PartitionStatisticsRepository repository;
private final String id;
private final String topic;
- private final String autoOffsetReset;
+ private final Clock clock;
+ private final Duration commitInterval;
+ private final Consumer<K, V> consumer;
+ private final java.util.function.Consumer<ConsumerRecord<K, V>> handler;
private final Lock lock = new ReentrantLock();
private final Condition condition = lock.newCondition();
private boolean running = false;
private Exception exception;
private long consumed = 0;
- private KafkaConsumer<String, String> consumer = null;
-
private final Map<Integer, Map<String, Long>> seen = new HashMap<>();
- public EndlessConsumer(
- ExecutorService executor,
- String bootstrapServer,
- String groupId,
- String clientId,
- String topic,
- String autoOffsetReset)
+ @Override
+ public void onPartitionsRevoked(Collection<TopicPartition> partitions)
{
- this.executor = executor;
- this.bootstrapServer = bootstrapServer;
- this.groupId = groupId;
- this.id = clientId;
- this.topic = topic;
- this.autoOffsetReset = autoOffsetReset;
+ partitions.forEach(tp ->
+ {
+ Integer partition = tp.partition();
+ Long newOffset = consumer.position(tp);
+ log.info(
+ "{} - removing partition: {}, offset of next message {})",
+ id,
+ partition,
+ newOffset);
+ Map<String, Long> removed = seen.remove(partition);
+ for (String key : removed.keySet())
+ {
+ log.info(
+ "{} - Seen {} messages for partition={}|key={}",
+ id,
+ removed.get(key),
+ partition,
+ key);
+ }
+ repository.save(new StatisticsDocument(partition, removed, consumer.position(tp)));
+ });
}
@Override
- public void run()
+ public void onPartitionsAssigned(Collection<TopicPartition> partitions)
{
- try
+ partitions.forEach(tp ->
{
- Properties props = new Properties();
- props.put("bootstrap.servers", bootstrapServer);
- props.put("group.id", groupId);
- props.put("client.id", id);
- props.put("auto.offset.reset", autoOffsetReset);
- props.put("metadata.max.age.ms", "1000");
- props.put("key.deserializer", StringDeserializer.class.getName());
- props.put("value.deserializer", StringDeserializer.class.getName());
+ Integer partition = tp.partition();
+ Long offset = consumer.position(tp);
+ log.info("{} - adding partition: {}, offset={}", id, partition, offset);
+ StatisticsDocument document =
+ repository
+ .findById(Integer.toString(partition))
+ .orElse(new StatisticsDocument(partition));
+ if (document.offset >= 0)
+ {
+ // Only seek, if a stored offset was found
+ // Otherwise: Use initial offset, generated by Kafka
+ consumer.seek(tp, document.offset);
+ }
+ seen.put(partition, document.statistics);
+ });
+ }
- this.consumer = new KafkaConsumer<>(props);
+ @Override
+ public void run()
+ {
+ try
+ {
log.info("{} - Subscribing to topic {}", id, topic);
- consumer.subscribe(Arrays.asList(topic), new ConsumerRebalanceListener()
- {
- @Override
- public void onPartitionsRevoked(Collection<TopicPartition> partitions)
- {
- partitions.forEach(tp ->
- {
- log.info("{} - removing partition: {}", id, tp);
- Map<String, Long> removed = seen.remove(tp.partition());
- for (String key : removed.keySet())
- {
- log.info(
- "{} - Seen {} messages for partition={}|key={}",
- id,
- removed.get(key),
- tp.partition(),
- key);
- }
- });
- }
+ consumer.subscribe(Arrays.asList(topic), this);
- @Override
- public void onPartitionsAssigned(Collection<TopicPartition> partitions)
- {
- partitions.forEach(tp ->
- {
- log.info("{} - adding partition: {}", id, tp);
- seen.put(tp.partition(), new HashMap<>());
- });
- }
- });
+ Instant lastCommit = clock.instant();
while (true)
{
- ConsumerRecords<String, String> records =
+ ConsumerRecords<K, V> records =
consumer.poll(Duration.ofSeconds(1));
// Do something with the data...
log.info("{} - Received {} messages", id, records.count());
- for (ConsumerRecord<String, String> record : records)
+ for (ConsumerRecord<K, V> record : records)
{
- consumed++;
log.info(
"{} - {}: {}/{} - {}={}",
id,
record.value()
);
+ handler.accept(record);
+
+ consumed++;
+
Integer partition = record.partition();
- String key = record.key() == null ? "NULL" : record.key();
+ String key = record.key() == null ? "NULL" : record.key().toString();
Map<String, Long> byKey = seen.get(partition);
if (!byKey.containsKey(key))
seenByKey++;
byKey.put(key, seenByKey);
}
+
+ if (lastCommit.plus(commitInterval).isBefore(clock.instant()))
+ {
+ log.debug("Storing data and offsets, last commit: {}", lastCommit);
+ seen.forEach((partiton, statistics) -> repository.save(
+ new StatisticsDocument(
+ partiton,
+ statistics,
+ consumer.position(new TopicPartition(topic, partiton)))));
+ lastCommit = clock.instant();
+ }
}
}
catch(WakeupException e)
{
- log.info("{} - RIIING!", id);
+ log.info("{} - RIIING! Request to stop consumption - commiting current offsets!", id);
shutdown();
}
+ catch(RecordDeserializationException e)
+ {
+ TopicPartition tp = e.topicPartition();
+ long offset = e.offset();
+ log.error(
+ "{} - Could not deserialize message on topic {} with offset={}: {}",
+ id,
+ tp,
+ offset,
+ e.getCause().toString());
+
+ shutdown(e);
+ }
catch(Exception e)
{
log.error("{} - Unexpected error: {}", id, e.toString(), e);
}
finally
{
- log.info("{} - Closing the KafkaConsumer", id);
- consumer.close();
log.info("{} - Consumer-Thread exiting", id);
}
}
lock.lock();
try
{
- running = false;
- exception = e;
- condition.signal();
+ try
+ {
+ log.info("{} - Unsubscribing from topic {}", id, topic);
+ consumer.unsubscribe();
+ }
+ catch (Exception ue)
+ {
+ log.error(
+ "{} - Error while unsubscribing from topic {}: {}",
+ id,
+ topic,
+ ue.toString());
+ }
+ finally
+ {
+ running = false;
+ exception = e;
+ condition.signal();
+ }
}
finally
{
}
}
- public synchronized void stop() throws ExecutionException, InterruptedException
+ public synchronized void stop() throws InterruptedException
{
lock.lock();
try
public void destroy() throws ExecutionException, InterruptedException
{
log.info("{} - Destroy!", id);
- try
- {
- stop();
- }
- catch (IllegalStateException e)
- {
- log.info("{} - Was already stopped", id);
- }
- catch (Exception e)
- {
- log.error("{} - Unexpected exception while trying to stop the consumer", id, e);
- }
- finally
- {
- log.info("{}: Consumed {} messages in total, exiting!", id, consumed);
- }
+ log.info("{}: Consumed {} messages in total, exiting!", id, consumed);
}
public boolean running()