NEU
[demos/kafka/chat] / src / main / java / de / juplo / kafka / chat / backend / persistence / kafka / KafkaChatHomeService.java
index a95df54..3ca5b7f 100644 (file)
@@ -2,30 +2,45 @@ package de.juplo.kafka.chat.backend.persistence.kafka;
 
 import de.juplo.kafka.chat.backend.domain.ChatHomeService;
 import de.juplo.kafka.chat.backend.domain.ChatRoom;
+import de.juplo.kafka.chat.backend.domain.Message;
+import de.juplo.kafka.chat.backend.domain.ShardNotOwnedException;
+import de.juplo.kafka.chat.backend.persistence.KafkaLikeShardingStrategy;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.kafka.clients.consumer.Consumer;
 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.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;
 
-import java.time.ZoneId;
+import java.time.*;
 import java.util.*;
 import java.util.concurrent.ExecutorService;
+import java.util.stream.IntStream;
 
 
 @Slf4j
-public class KafkaChatHomeService implements ChatHomeService, ConsumerRebalanceListener
+public class KafkaChatHomeService implements ChatHomeService, Runnable, ConsumerRebalanceListener
 {
   private final ExecutorService executorService;
   private final Consumer<String, MessageTo> 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 ChatHomeLoader[] chatHomeLoaders;
+  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;
+
+  private boolean running;
+  private volatile boolean loadInProgress;
 
 
   public KafkaChatHomeService(
@@ -42,43 +57,37 @@ public class KafkaChatHomeService implements ChatHomeService, ConsumerRebalanceL
     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.chatHomeLoaders = new ChatHomeLoader[numShards];
+    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);
   }
 
 
   @Override
   public void onPartitionsAssigned(Collection<TopicPartition> partitions)
   {
+    log.info("Newly assigned partitions! Pausing normal operations...");
+    loadInProgress = true;
+
     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);
-      chatHomeLoaders[partition] = new ChatHomeLoader(
-          producer,
-          currentOffset,
-          zoneId);
+      consumer.seek(topicPartition, nextOffset[partition]);
     });
+
+    consumer.resume(partitions);
   }
 
   @Override
@@ -86,34 +95,175 @@ public class KafkaChatHomeService implements ChatHomeService, ConsumerRebalanceL
   {
     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);
   }
 
+  @Override
+  public void run()
+  {
+    consumer.subscribe(List.of(topic));
+
+    running = true;
+
+    while (running)
+    {
+      try
+      {
+        ConsumerRecords<String, MessageTo> records = consumer.poll(Duration.ofMinutes(5));
+        log.info("Fetched {} messages", records.count());
+
+        if (loadInProgress)
+        {
+          loadMessages(records);
+
+          if (isLoadingCompleted())
+          {
+            log.info("Loading of messages completed! Pausing all owned partitions...");
+            pauseAllOwnedPartions();
+            log.info("Resuming normal operations...");
+            loadInProgress = false;
+          }
+        }
+        else
+        {
+          if (!records.isEmpty())
+          {
+            throw new IllegalStateException("All owned partitions should be paused, when no load is in progress!");
+          }
+        }
+      }
+      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,
+      LocalDateTime timestamp,
+      String text)
+  {
+    int shard = this.shardingStrategy.selectShard(chatRoomId);
+    TopicPartition tp = new TopicPartition(topic, shard);
+    ZonedDateTime zdt = ZonedDateTime.of(timestamp, zoneId);
+    return Mono.create(sink ->
+    {
+      ProducerRecord<String, MessageTo> record =
+          new ProducerRecord<>(
+              tp.topic(),
+              tp.partition(),
+              zdt.toEpochSecond(),
+              chatRoomId.toString(),
+              MessageTo.of(key.getUsername(), key.getMessageId(), text));
+
+      producer.send(record, ((metadata, exception) ->
+      {
+        if (metadata != null)
+        {
+          // On successful send
+          Message message = new Message(key, metadata.offset(), timestamp, text);
+          log.info("Successfully send message {}", message);
+          sink.success(message);
+        }
+        else
+        {
+          // On send-failure
+          log.error(
+              "Could not send message for chat-room={}, key={}, timestamp={}, text={}: {}",
+              chatRoomId,
+              key,
+              timestamp,
+              text,
+              exception);
+          sink.error(exception);
+        }
+      }));
+    });
+  }
+
+
   @Override
   public Mono<ChatRoom> getChatRoom(int shard, UUID id)
   {
-    return Mono.justOrEmpty(chatRoomMaps[shard].get(id));
+    if (loadInProgress)
+    {
+      throw new ShardNotOwnedException(shard);
+    }
+    else
+    {
+      return Mono.justOrEmpty(chatRoomMaps[shard].get(id));
+    }
   }
 
   @Override
   public Flux<ChatRoom> getChatRooms(int shard)
   {
-    return Flux.fromStream(chatRoomMaps[shard].values().stream());
+    if (loadInProgress)
+    {
+      throw new ShardNotOwnedException(shard);
+    }
+    else
+    {
+      return Flux.fromStream(chatRoomMaps[shard].values().stream());
+    }
   }
 }