Merge der überarbeiteten Compose-Konfiguration ('stored-state')
[demos/kafka/training] / src / main / java / de / juplo / kafka / EndlessConsumer.java
index 063a09e..c79b1e0 100644 (file)
@@ -1,44 +1,49 @@
 package de.juplo.kafka;
 
 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.common.TopicPartition;
 import org.apache.kafka.common.errors.WakeupException;
 import org.apache.kafka.common.serialization.StringDeserializer;
 
 import javax.annotation.PreDestroy;
 import java.time.Duration;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
+import java.util.*;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
 
 
 @Slf4j
 public class EndlessConsumer implements Runnable
 {
   private final ExecutorService executor;
+  private final PartitionStatisticsRepository repository;
   private final String bootstrapServer;
   private final String groupId;
   private final String id;
   private final String topic;
   private final String autoOffsetReset;
 
-  private AtomicBoolean running = new AtomicBoolean();
+  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 Future<?> future = null;
 
-  private Map<Integer, Map<String, Integer>> seen;
+
+  private final Map<Integer, Map<String, Integer>> seen = new HashMap<>();
 
 
   public EndlessConsumer(
       ExecutorService executor,
+      PartitionStatisticsRepository repository,
       String bootstrapServer,
       String groupId,
       String clientId,
@@ -46,6 +51,7 @@ public class EndlessConsumer implements Runnable
       String autoOffsetReset)
   {
     this.executor = executor;
+    this.repository = repository;
     this.bootstrapServer = bootstrapServer;
     this.groupId = groupId;
     this.id = clientId;
@@ -62,16 +68,52 @@ public class EndlessConsumer implements Runnable
       props.put("bootstrap.servers", bootstrapServer);
       props.put("group.id", groupId);
       props.put("client.id", id);
+      props.put("enable.auto.commit", false);
       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());
 
       this.consumer = new KafkaConsumer<>(props);
 
       log.info("{} - Subscribing to topic {}", id, topic);
-      consumer.subscribe(Arrays.asList(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, Integer> removed = seen.remove(tp.partition());
+            for (String key : removed.keySet())
+            {
+              log.info(
+                  "{} - Seen {} messages for partition={}|key={}",
+                  id,
+                  removed.get(key),
+                  tp.partition(),
+                  key);
+            }
+            repository.save(new StatisticsDocument(tp.partition(), removed, consumer.position(tp)));
+          });
+        }
 
-      seen = new HashMap<>();
+        @Override
+        public void onPartitionsAssigned(Collection<TopicPartition> partitions)
+        {
+          partitions.forEach(tp ->
+          {
+            log.info("{} - adding partition: {}", id, tp);
+            StatisticsDocument document =
+                repository
+                    .findById(Integer.toString(tp.partition()))
+                    .orElse(new StatisticsDocument(tp.partition()));
+            consumer.seek(tp, document.offset);
+            seen.put(tp.partition(), document.statistics);
+          });
+        }
+      });
 
       while (true)
       {
@@ -94,11 +136,7 @@ public class EndlessConsumer implements Runnable
           );
 
           Integer partition = record.partition();
-          String key = record.key();
-
-          if (!seen.containsKey(partition))
-            seen.put(partition, new HashMap<>());
-
+          String key = record.key() == null ? "NULL" : record.key();
           Map<String, Integer> byKey = seen.get(partition);
 
           if (!byKey.containsKey(key))
@@ -108,38 +146,49 @@ public class EndlessConsumer implements Runnable
           seenByKey++;
           byKey.put(key, seenByKey);
         }
+
+        seen.forEach((partiton, statistics) -> repository.save(
+            new StatisticsDocument(
+                partiton,
+                statistics,
+                consumer.position(new TopicPartition(topic, partiton)))));
       }
     }
     catch(WakeupException e)
     {
       log.info("{} - RIIING!", id);
+      shutdown();
     }
     catch(Exception e)
     {
       log.error("{} - Unexpected error: {}", id, e.toString(), e);
-      running.set(false); // Mark the instance as not running
+      shutdown(e);
     }
     finally
     {
       log.info("{} - Closing the KafkaConsumer", id);
       consumer.close();
+      log.info("{} - Consumer-Thread exiting", id);
+    }
+  }
 
-      for (Integer partition : seen.keySet())
-      {
-        Map<String, Integer> byKey = seen.get(partition);
-        for (String key : byKey.keySet())
-        {
-          log.info(
-              "{} - Seen {} messages for partition={}|key={}",
-              id,
-              byKey.get(key),
-              partition,
-              key);
-        }
-      }
-      seen = null;
+  private void shutdown()
+  {
+    shutdown(null);
+  }
 
-      log.info("{} - Consumer-Thread exiting", id);
+  private void shutdown(Exception e)
+  {
+    lock.lock();
+    try
+    {
+      running = false;
+      exception = e;
+      condition.signal();
+    }
+    finally
+    {
+      lock.unlock();
     }
   }
 
@@ -148,26 +197,42 @@ public class EndlessConsumer implements Runnable
     return seen;
   }
 
-  public synchronized void start()
+  public void start()
   {
-    boolean stateChanged = running.compareAndSet(false, true);
-    if (!stateChanged)
-      throw new RuntimeException("Consumer instance " + id + " is already running!");
+    lock.lock();
+    try
+    {
+      if (running)
+        throw new IllegalStateException("Consumer instance " + id + " is already running!");
 
-    log.info("{} - Starting - consumed {} messages before", id, consumed);
-    future = executor.submit(this);
+      log.info("{} - Starting - consumed {} messages before", id, consumed);
+      running = true;
+      exception = null;
+      executor.submit(this);
+    }
+    finally
+    {
+      lock.unlock();
+    }
   }
 
   public synchronized void stop() throws ExecutionException, InterruptedException
   {
-    boolean stateChanged = running.compareAndSet(true, false);
-    if (!stateChanged)
-      throw new RuntimeException("Consumer instance " + id + " is not running!");
-
-    log.info("{} - Stopping", id);
-    consumer.wakeup();
-    future.get();
-    log.info("{} - Stopped - consumed {} messages so far", id, consumed);
+    lock.lock();
+    try
+    {
+      if (!running)
+        throw new IllegalStateException("Consumer instance " + id + " is not running!");
+
+      log.info("{} - Stopping", id);
+      consumer.wakeup();
+      condition.await();
+      log.info("{} - Stopped - consumed {} messages so far", id, consumed);
+    }
+    finally
+    {
+      lock.unlock();
+    }
   }
 
   @PreDestroy
@@ -182,9 +247,42 @@ public class EndlessConsumer implements Runnable
     {
       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);
     }
   }
+
+  public boolean running()
+  {
+    lock.lock();
+    try
+    {
+      return running;
+    }
+    finally
+    {
+      lock.unlock();
+    }
+  }
+
+  public Optional<Exception> exitStatus()
+  {
+    lock.lock();
+    try
+    {
+      if (running)
+        throw new IllegalStateException("No exit-status available: Consumer instance " + id + " is running!");
+
+      return Optional.ofNullable(exception);
+    }
+    finally
+    {
+      lock.unlock();
+    }
+  }
 }