Rückbau der Deaktivierung der Commits, um den Code anzugleichen
[demos/kafka/training] / src / main / java / de / juplo / kafka / ApplicationRebalanceListener.java
index 7256732..3653197 100644 (file)
@@ -2,26 +2,24 @@ 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
 @Slf4j
-public class ApplicationRebalanceListener implements PollIntervalAwareConsumerRebalanceListener
+public class ApplicationRebalanceListener implements RebalanceListener
 {
   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 String topic;
+  private final Consumer consumer;
 
-  private Instant lastCommit = Instant.EPOCH;
+  private final Set<Integer> partitions = new HashSet<>();
 
   @Override
   public void onPartitionsAssigned(Collection<TopicPartition> partitions)
@@ -30,11 +28,33 @@ public class ApplicationRebalanceListener implements PollIntervalAwareConsumerRe
     {
       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));
+      if (document.offset >= 0)
+      {
+        // Only seek, if a stored offset was found
+        // Otherwise: Use initial offset, generated by Kafka
+        consumer.seek(tp, document.offset);
+        log.info(
+            "{} - Seeking to offset {} for partition {}",
+            id,
+            document.offset,
+            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,17 +65,25 @@ public class ApplicationRebalanceListener implements PollIntervalAwareConsumerRe
     {
       Integer partition = tp.partition();
       log.info("{} - removing partition: {}", id, partition);
-      Map<String, Long> removed = recordHandler.removePartition(partition);
-      for (String key : removed.keySet())
+      this.partitions.remove(partition);
+      Map<String, AdderResult> state = recordHandler.removePartition(partition);
+      Long offset = consumer.position(tp);
+      log.info(
+          "{} - offset of next unseen message for partition {} is {}",
+          id,
+          partition,
+          offset);
+      for (String user : state.keySet())
       {
         log.info(
-            "{} - Seen {} messages for partition={}|key={}",
+            "{} - Saved state for partition={}|user={}: {}",
             id,
-            removed.get(key),
             partition,
-            key);
+            user,
+            state.get(user));
       }
-      stateRepository.save(new StateDocument(partition, removed));
+      Map<String, List<AdderResult>> results = adderResults.removePartition(partition);
+      stateRepository.save(new StateDocument(partition, state, results, offset));
     });
   }
 
@@ -63,14 +91,29 @@ public class ApplicationRebalanceListener implements PollIntervalAwareConsumerRe
   @Override
   public void beforeNextPoll()
   {
-    if (lastCommit.plus(commitInterval).isBefore(clock.instant()))
-    {
-      log.debug("Storing data, last commit: {}", lastCommit);
-      recordHandler.getState().forEach((partiton, adder) -> stateRepository.save(
-          new StateDocument(
-              partiton,
-              adder.getState())));
-      lastCommit = clock.instant();
-    }
+    partitions
+      .stream()
+      .forEach(partition ->
+      {
+        log.info("{} - persisting state & offset for partition: {}", id, partition);
+        Map<String, AdderResult> state = recordHandler.getState(partition).getState();
+        Long offset = consumer.position(new TopicPartition(topic, partition));
+        log.info(
+          "{} - offset of next unseen message for partition {} is {}",
+          id,
+          partition,
+          offset);
+        for (String user : state.keySet())
+        {
+          log.info(
+            "{} - Saved state for partition={}|user={}: {}",
+            id,
+            partition,
+            user,
+            state.get(user));
+        }
+        Map<String, List<AdderResult>> results = adderResults.getState(partition);
+        stateRepository.save(new StateDocument(partition, state, results, offset));
+      });
   }
 }