NEU
[demos/kafka/chat] / src / main / java / de / juplo / kafka / chat / backend / persistence / kafka / KafkaChatHomeService.java
index 5133d1a..b4dccbd 100644 (file)
@@ -19,6 +19,7 @@ import reactor.core.publisher.Mono;
 import java.time.*;
 import java.util.*;
 import java.util.concurrent.ExecutorService;
+import java.util.stream.IntStream;
 
 
 @Slf4j
@@ -29,8 +30,10 @@ public class KafkaChatHomeService implements ChatHomeService, Runnable, Consumer
   private final Producer<String, MessageTo> producer;
   private final String topic;
   private final ZoneId zoneId;
-  // private final long[] offsets; Erst mal immer alles neu einlesen
+  private final int numShards;
   private final boolean[] isShardOwned;
+  private final long[] currentOffset;
+  private final long[] nextOffset;
   private final Map<UUID, ChatRoom>[] chatRoomMaps;
   private final KafkaLikeShardingStrategy shardingStrategy;
 
@@ -52,12 +55,10 @@ public class KafkaChatHomeService implements ChatHomeService, Runnable, Consumer
     this.producer = producer;
     this.topic = topic;
     this.zoneId = zoneId;
-    // this.offsets = new long[numShards];
-    // for (int i=0; i< numShards; i++)
-    // {
-    //   this.offsets[i] = 0l;
-    // }
+    this.numShards = numShards;
     this.isShardOwned = new boolean[numShards];
+    this.currentOffset = new long[numShards];
+    this.nextOffset = new long[numShards];
     this.chatRoomMaps = new Map[numShards];
     this.shardingStrategy = new KafkaLikeShardingStrategy(numShards);
   }
@@ -70,23 +71,17 @@ public class KafkaChatHomeService implements ChatHomeService, Runnable, Consumer
 
     consumer.endOffsets(partitions).forEach((topicPartition, currentOffset) ->
     {
-      if (!topicPartition.topic().equals(topic))
-      {
-        log.warn("Ignoring partition from unwanted topic: {}", topicPartition);
-        return;
-      }
-
       int partition = topicPartition.partition();
-      long unseenOffset = 0; // offsets[partition];
+      isShardOwned[partition] =  true;
+      this.currentOffset[partition] = currentOffset;
 
       log.info(
-          "Loading messages from partition {}: start-offset={} -> current-offset={}",
+          "Partition assigned: {} - loading messages: next={} -> current={}",
           partition,
-          unseenOffset,
+          nextOffset[partition],
           currentOffset);
 
-      // TODO: reuse! Nicht immer alles neu laden, sondern erst ab offsets[partition]!
-      consumer.seek(topicPartition, unseenOffset);
+      consumer.seek(topicPartition, nextOffset[partition]);
     });
 
     consumer.resume(partitions);
@@ -97,21 +92,16 @@ public class KafkaChatHomeService implements ChatHomeService, Runnable, Consumer
   {
     partitions.forEach(topicPartition ->
     {
-      if (!topicPartition.topic().equals(topic))
-      {
-        log.warn("Ignoring partition from unwanted topic: {}", topicPartition);
-        return;
-      }
-
       int partition = topicPartition.partition();
-      // long unseenOffset = offsets[partition]; TODO: Offset merken...?
+      isShardOwned[partition] = false;
+      log.info("Partition revoked: {} - next={}", partition, nextOffset[partition]);
     });
-    log.info("Revoked partitions: {}", partitions);
   }
 
   @Override
   public void onPartitionsLost(Collection<TopicPartition> partitions)
   {
+    log.warn("Lost partitions: {}, partitions");
     // TODO: Muss auf den Verlust anders reagiert werden?
     onPartitionsRevoked(partitions);
   }
@@ -134,21 +124,47 @@ public class KafkaChatHomeService implements ChatHomeService, Runnable, Consumer
         {
           for (ConsumerRecord<String, MessageTo> record : records)
           {
+            nextOffset[record.partition()] = record.offset() + 1;
             UUID chatRoomId = UUID.fromString(record.key());
             MessageTo messageTo = record.value();
+
+            Message.MessageKey key = Message.MessageKey.of(messageTo.getUser(), messageTo.getId());
+
+            Instant instant = Instant.ofEpochSecond(record.timestamp());
+            LocalDateTime timestamp = LocalDateTime.ofInstant(instant, zoneId);
+
+            Message message = new Message(key, record.offset(), timestamp, messageTo.getText());
+
             ChatRoom chatRoom = chatRoomMaps[record.partition()].get(chatRoomId);
-            Mono<Message> result = chatRoom.addMessage(
-                messageTo.getId(),
-                messageTo.getUser(),
-                messageTo.getText());
-            result.block().
+            KafkaChatRoomService kafkaChatRoomService =
+                (KafkaChatRoomService) chatRoom.getChatRoomService();
+
+            kafkaChatRoomService.persistMessage(message);
+          }
+
+          if (IntStream
+            .range(0, numShards)
+            .filter(shard -> isShardOwned[shard])
+            .mapToObj(shard -> nextOffset[shard] >= currentOffset[shard])
+            .collect(
+                () -> Boolean.TRUE,
+                (acc, v) -> Boolean.valueOf(acc && v),
+                (a, b) -> Boolean.valueOf(a && b)))
+          {
+            log.info("Loading of messages completed! Pausing all owned partitions...");
+            consumer.pause(IntStream
+                .range(0, numShards)
+                .filter(shard -> isShardOwned[shard])
+                .mapToObj(shard -> new TopicPartition(topic, shard))
+                .toList());
+            loadInProgress = false;
           }
         }
         else
         {
           if (!records.isEmpty())
           {
-            throw new IllegalStateException("All owned partions should be paused, when no load is in progress!");
+            throw new IllegalStateException("All owned partitions should be paused, when no load is in progress!");
           }
         }
       }