X-Git-Url: https://juplo.de/gitweb/?a=blobdiff_plain;f=src%2Fmain%2Fjava%2Fde%2Fjuplo%2Fkafka%2FApplicationRebalanceListener.java;h=cd9da64538cf0097d669fc26606120f72aba4d90;hb=e11c6152c721440d4a599a6f5fe0fe46f2283f31;hp=7256732b81f9c1649021ca19e54171ca0d53307a;hpb=de4f94c45fd0678777fecba4dbcb63f89e0ebafa;p=demos%2Fkafka%2Ftraining diff --git a/src/main/java/de/juplo/kafka/ApplicationRebalanceListener.java b/src/main/java/de/juplo/kafka/ApplicationRebalanceListener.java index 7256732..cd9da64 100644 --- a/src/main/java/de/juplo/kafka/ApplicationRebalanceListener.java +++ b/src/main/java/de/juplo/kafka/ApplicationRebalanceListener.java @@ -2,13 +2,13 @@ package de.juplo.kafka; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; +import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.common.TopicPartition; import java.time.Clock; import java.time.Duration; import java.time.Instant; -import java.util.Collection; -import java.util.Map; +import java.util.*; @RequiredArgsConstructor @@ -16,12 +16,18 @@ import java.util.Map; public class ApplicationRebalanceListener implements PollIntervalAwareConsumerRebalanceListener { private final ApplicationRecordHandler recordHandler; + private final AdderResults adderResults; private final StateRepository stateRepository; private final String id; + private final String topic; private final Clock clock; private final Duration commitInterval; + private final Consumer consumer; + + private final Set partitions = new HashSet<>(); private Instant lastCommit = Instant.EPOCH; + private boolean commitsEnabled = true; @Override public void onPartitionsAssigned(Collection partitions) @@ -29,12 +35,20 @@ public class ApplicationRebalanceListener implements PollIntervalAwareConsumerRe partitions.forEach(tp -> { Integer partition = tp.partition(); - log.info("{} - adding partition: {}", id, partition); + this.partitions.add(partition); StateDocument document = stateRepository .findById(Integer.toString(partition)) .orElse(new StateDocument(partition)); + log.info("{} - adding partition: {}, offset={}", id, partition, document.offset); + if (document.offset >= 0) + { + // Only seek, if a stored offset was found + // Otherwise: Use initial offset, generated by Kafka + consumer.seek(tp, document.offset); + } recordHandler.addPartition(partition, document.state); + adderResults.addPartition(partition, document.results); }); } @@ -44,18 +58,23 @@ public class ApplicationRebalanceListener implements PollIntervalAwareConsumerRe partitions.forEach(tp -> { Integer partition = tp.partition(); - log.info("{} - removing partition: {}", id, partition); - Map removed = recordHandler.removePartition(partition); - for (String key : removed.keySet()) + this.partitions.remove(partition); + Long offset = consumer.position(tp); + log.info( + "{} - removing partition: {}, offset of next message {})", + id, + partition, + offset); + if (commitsEnabled) + { + Map state = recordHandler.removePartition(partition); + Map> results = adderResults.removePartition(partition); + stateRepository.save(new StateDocument(partition, state, results, offset)); + } + else { - log.info( - "{} - Seen {} messages for partition={}|key={}", - id, - removed.get(key), - partition, - key); + log.info("Offset commits are disabled! Last commit: {}", lastCommit); } - stateRepository.save(new StateDocument(partition, removed)); }); } @@ -63,14 +82,34 @@ public class ApplicationRebalanceListener implements PollIntervalAwareConsumerRe @Override public void beforeNextPoll() { + if (!commitsEnabled) + { + log.info("Offset commits are disabled! Last commit: {}", lastCommit); + return; + } + if (lastCommit.plus(commitInterval).isBefore(clock.instant())) { - log.debug("Storing data, last commit: {}", lastCommit); - recordHandler.getState().forEach((partiton, adder) -> stateRepository.save( + log.debug("Storing data and offsets, last commit: {}", lastCommit); + partitions.forEach(partition -> stateRepository.save( new StateDocument( - partiton, - adder.getState()))); + partition, + recordHandler.getState(partition).getState(), + adderResults.getState(partition), + consumer.position(new TopicPartition(topic, partition))))); lastCommit = clock.instant(); } } + + @Override + public void enableCommits() + { + commitsEnabled = true; + } + + @Override + public void disableCommits() + { + commitsEnabled = false; + } }