Wordcount-Implementierung mit Kafka-Boardmitteln und MongoDB als Storage
[demos/kafka/training] / src / main / java / de / juplo / kafka / EndlessConsumer.java
index 2310ccd..01f9057 100644 (file)
@@ -1,83 +1,98 @@
 package de.juplo.kafka;
 
+import lombok.RequiredArgsConstructor;
 import lombok.extern.slf4j.Slf4j;
-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.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Properties;
+import java.time.Instant;
+import java.util.*;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Pattern;
 
 
 @Slf4j
-public class EndlessConsumer implements Runnable
+@RequiredArgsConstructor
+public class EndlessConsumer implements ConsumerRebalanceListener, Runnable
 {
+  final static Pattern PATTERN = Pattern.compile("\\W+");
+
+
   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<String, String> consumer;
 
   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, Map<String, Long>>> seen = new HashMap<>();
 
-  private Map<Integer, Map<String, Integer>> seen;
 
+  @Override
+  public void onPartitionsRevoked(Collection<TopicPartition> partitions)
+  {
+    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, Map<String, Long>> removed = seen.remove(partition);
+      repository.save(new StatisticsDocument(partition, removed, consumer.position(tp)));
+    });
+  }
 
-  public EndlessConsumer(
-      ExecutorService executor,
-      String bootstrapServer,
-      String groupId,
-      String clientId,
-      String topic,
-      String autoOffsetReset)
+  @Override
+  public void onPartitionsAssigned(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 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);
+    });
   }
 
+
   @Override
   public void run()
   {
     try
     {
-      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());
-
-      this.consumer = new KafkaConsumer<>(props);
-
       log.info("{} - Subscribing to topic {}", id, topic);
-      consumer.subscribe(Arrays.asList(topic));
+      consumer.subscribe(Arrays.asList(topic), this);
 
-      seen = new HashMap<>();
+      Instant lastCommit = clock.instant();
 
       while (true)
       {
@@ -88,7 +103,6 @@ public class EndlessConsumer implements Runnable
         log.info("{} - Received {} messages", id, records.count());
         for (ConsumerRecord<String, String> record : records)
         {
-          consumed++;
           log.info(
               "{} - {}: {}/{} - {}={}",
               id,
@@ -99,28 +113,64 @@ public class EndlessConsumer implements Runnable
               record.value()
           );
 
-          Integer partition = record.partition();
-          String key = record.key() == null ? "NULL" : record.key();
-
-          if (!seen.containsKey(partition))
-            seen.put(partition, new HashMap<>());
-
-          Map<String, Integer> byKey = seen.get(partition);
+          consumed++;
 
-          if (!byKey.containsKey(key))
-            byKey.put(key, 0);
+          Integer partition = record.partition();
+          String user = record.key();
+          Map<String, Map<String, Long>> users = seen.get(partition);
+
+          Map<String, Long> words = users.get(user);
+          if (words == null)
+          {
+            words = new HashMap<>();
+            users.put(user, words);
+          }
+
+          for (String word : PATTERN.split(record.value()))
+          {
+            Long num = words.get(word);
+            if (num == null)
+            {
+              num = 1l;
+            }
+            else
+            {
+              num++;
+            }
+            words.put(word, num);
+          }
+        }
 
-          int seenByKey = byKey.get(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);
@@ -128,33 +178,10 @@ public class EndlessConsumer implements Runnable
     }
     finally
     {
-      log.info("{} - Closing the KafkaConsumer", id);
-      consumer.close();
-
-      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;
-
       log.info("{} - Consumer-Thread exiting", id);
     }
   }
 
-  public Map<Integer, Map<String, Integer>> getSeen()
-  {
-    return seen;
-  }
-
   private void shutdown()
   {
     shutdown(null);
@@ -165,9 +192,25 @@ public class EndlessConsumer implements Runnable
     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
     {
@@ -175,6 +218,11 @@ public class EndlessConsumer implements Runnable
     }
   }
 
+  public Map<Integer, Map<String, Map<String, Long>>> getSeen()
+  {
+    return seen;
+  }
+
   public void start()
   {
     lock.lock();
@@ -194,7 +242,7 @@ public class EndlessConsumer implements Runnable
     }
   }
 
-  public synchronized void stop() throws ExecutionException, InterruptedException
+  public synchronized void stop() throws InterruptedException
   {
     lock.lock();
     try
@@ -217,22 +265,7 @@ public class EndlessConsumer implements Runnable
   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()