NEU
authorKai Moritz <kai@juplo.de>
Sat, 15 Apr 2023 13:02:31 +0000 (15:02 +0200)
committerKai Moritz <kai@juplo.de>
Sat, 15 Apr 2023 13:02:31 +0000 (15:02 +0200)
src/main/java/de/juplo/kafka/chat/backend/persistence/kafka/KafkaChatHomeService.java

index b4dccbd..3ca5b7f 100644 (file)
@@ -13,6 +13,8 @@ import org.apache.kafka.clients.consumer.ConsumerRecords;
 import org.apache.kafka.clients.producer.Producer;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.RecordDeserializationException;
+import org.apache.kafka.common.errors.WakeupException;
 import reactor.core.publisher.Flux;
 import reactor.core.publisher.Mono;
 
@@ -67,6 +69,7 @@ public class KafkaChatHomeService implements ChatHomeService, Runnable, Consumer
   @Override
   public void onPartitionsAssigned(Collection<TopicPartition> partitions)
   {
+    log.info("Newly assigned partitions! Pausing normal operations...");
     loadInProgress = true;
 
     consumer.endOffsets(partitions).forEach((topicPartition, currentOffset) ->
@@ -113,50 +116,22 @@ public class KafkaChatHomeService implements ChatHomeService, Runnable, Consumer
 
     running = true;
 
-    try
+    while (running)
     {
-      while (running)
+      try
       {
         ConsumerRecords<String, MessageTo> records = consumer.poll(Duration.ofMinutes(5));
         log.info("Fetched {} messages", records.count());
 
         if (loadInProgress)
         {
-          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);
-            KafkaChatRoomService kafkaChatRoomService =
-                (KafkaChatRoomService) chatRoom.getChatRoomService();
-
-            kafkaChatRoomService.persistMessage(message);
-          }
+          loadMessages(records);
 
-          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)))
+          if (isLoadingCompleted())
           {
             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());
+            pauseAllOwnedPartions();
+            log.info("Resuming normal operations...");
             loadInProgress = false;
           }
         }
@@ -168,9 +143,59 @@ public class KafkaChatHomeService implements ChatHomeService, Runnable, Consumer
           }
         }
       }
+      catch (WakeupException e)
+      {
+      }
+      catch (RecordDeserializationException e)
+      {
+      }
     }
   }
 
+  void loadMessages(ConsumerRecords<String, MessageTo> records)
+  {
+    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);
+      KafkaChatRoomService kafkaChatRoomService =
+          (KafkaChatRoomService) chatRoom.getChatRoomService();
+
+      kafkaChatRoomService.persistMessage(message);
+    }
+  }
+
+  boolean isLoadingCompleted()
+  {
+    return 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));
+  }
+
+  void pauseAllOwnedPartions()
+  {
+    consumer.pause(IntStream
+        .range(0, numShards)
+        .filter(shard -> isShardOwned[shard])
+        .mapToObj(shard -> new TopicPartition(topic, shard))
+        .toList());
+  }
+
   Mono<Message> sendMessage(
       UUID chatRoomId,
       Message.MessageKey key,