X-Git-Url: https://juplo.de/gitweb/?a=blobdiff_plain;f=src%2Fmain%2Fjava%2Fde%2Fjuplo%2Fkafka%2FApplicationRebalanceListener.java;h=d31929519e61a4a6fc18f84e4590a7295f757c4f;hb=384ed293b9d6d71f68031f92bd7bf60670a7ba4b;hp=6776c0d6218c24f59e14ba356c52927014261730;hpb=14ddf90be0adbeb8ab34b516b9b158071ae491e4;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 6776c0d..d319295 100644 --- a/src/main/java/de/juplo/kafka/ApplicationRebalanceListener.java +++ b/src/main/java/de/juplo/kafka/ApplicationRebalanceListener.java @@ -2,28 +2,25 @@ 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.*; @RequiredArgsConstructor @Slf4j -public class ApplicationRebalanceListener implements PollIntervalAwareConsumerRebalanceListener +public class ApplicationRebalanceListener implements CommittingConsumerRebalanceListener { private final ApplicationRecordHandler recordHandler; private final AdderResults adderResults; private final StateRepository stateRepository; private final String id; - 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) @@ -38,6 +35,15 @@ public class ApplicationRebalanceListener implements PollIntervalAwareConsumerRe .findById(Integer.toString(partition)) .orElse(new StateDocument(partition)); recordHandler.addPartition(partition, document.state); + for (String user : document.state.keySet()) + { + log.info( + "{} - Restored state for partition={}|user={}: {}", + id, + partition, + user, + document.state.get(user)); + } adderResults.addPartition(partition, document.results); }); } @@ -45,6 +51,19 @@ public class ApplicationRebalanceListener implements PollIntervalAwareConsumerRe @Override public void onPartitionsRevoked(Collection partitions) { + if (commitsEnabled) + { + log.info("{} - Commiting offsets for all previously assigned partitions", id); + try + { + consumer.commitSync(); + } + catch (Exception e) + { + log.warn("{} - Could not commit offsets in onPartitionsRevoked():", id, e); + } + } + partitions.forEach(tp -> { Integer partition = tp.partition(); @@ -54,7 +73,7 @@ public class ApplicationRebalanceListener implements PollIntervalAwareConsumerRe for (String user : state.keySet()) { log.info( - "{} - Calculations for partition={}|user={}: {}", + "{} - Saved state for partition={}|user={}: {}", id, partition, user, @@ -65,19 +84,15 @@ public class ApplicationRebalanceListener implements PollIntervalAwareConsumerRe }); } + @Override + public void enableCommits() + { + commitsEnabled = true; + } @Override - public void beforeNextPoll() + public void disableCommits() { - if (lastCommit.plus(commitInterval).isBefore(clock.instant())) - { - log.debug("Storing data, last commit: {}", lastCommit); - partitions.forEach(partition -> stateRepository.save( - new StateDocument( - partition, - recordHandler.getState(partition).getState(), - adderResults.getState(partition)))); - lastCommit = clock.instant(); - } + commitsEnabled = false; } }