NEU
[demos/kafka/chat] / src / main / java / de / juplo / kafka / chat / backend / persistence / kafka / KafkaChatHomeService.java
index 4fa567c..f105902 100644 (file)
@@ -3,180 +3,228 @@ 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 lombok.RequiredArgsConstructor;
+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 reactor.core.publisher.Flux;
 import reactor.core.publisher.Mono;
 
+import java.time.*;
 import java.util.*;
+import java.util.concurrent.ExecutorService;
 
 
 @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 long[] offsets;
-  private final MessageHandler[] handlers;
-  private final Map<UUID, ChatRoom>[] chatrooms;
+  private final ZoneId zoneId;
+  // private final long[] offsets; Erst mal immer alles neu einlesen
+  private final boolean[] isShardOwned;
+  private final Map<UUID, ChatRoom>[] chatRoomMaps;
+  private final KafkaLikeShardingStrategy shardingStrategy;
+
+  private boolean running;
+  private volatile boolean loadInProgress;
 
 
   public KafkaChatHomeService(
+    ExecutorService executorService,
     Consumer<String, MessageTo> consumer,
+    Producer<String, MessageTo> producer,
     String topic,
+    ZoneId zoneId,
     int numShards)
   {
     log.debug("Creating KafkaChatHomeService");
+    this.executorService = executorService;
     this.consumer = consumer;
+    this.producer = producer;
     this.topic = topic;
-    this.offsets = new long[numShards];
-    this.handlers = new MessageHandler[numShards];
-    for (int i=0; i< numShards; i++)
-    {
-      this.offsets[i] = 0l;
-      this.handlers[i] = new NoOpMessageHandler(i);
-    }
-    this.chatrooms = new Map[numShards];
+    this.zoneId = zoneId;
+    // this.offsets = new long[numShards];
+    // for (int i=0; i< numShards; i++)
+    // {
+    //   this.offsets[i] = 0l;
+    // }
+    this.isShardOwned = new boolean[numShards];
+    this.chatRoomMaps = new Map[numShards];
+    this.shardingStrategy = new KafkaLikeShardingStrategy(numShards);
   }
 
 
   @Override
   public void onPartitionsAssigned(Collection<TopicPartition> partitions)
   {
-    consumer.endOffsets(partitions).forEach((tp, currentOffset) ->
+    loadInProgress = true;
+
+    consumer.endOffsets(partitions).forEach((topicPartition, currentOffset) ->
     {
-      if (!tp.topic().equals(topic))
+      if (!topicPartition.topic().equals(topic))
       {
-        log.warn("Ignoring unwanted TopicPartition", tp);
+        log.warn("Ignoring partition from unwanted topic: {}", topicPartition);
         return;
       }
 
-      int partition = tp.partition();
-      long unseenOffset = offsets[partition];
+      int partition = topicPartition.partition();
+      long unseenOffset = 0; // offsets[partition];
 
-      log.info("Reading partition {} from {} -> {}", partition, unseenOffset, currentOffset);
-      handlers[partition] = new ChatRoomLoadingMessageHandler(partition, currentOffset, unseenOffset);
+      log.info(
+          "Loading messages from partition {}: start-offset={} -> current-offset={}",
+          partition,
+          unseenOffset,
+          currentOffset);
+
+      // TODO: reuse! Nicht immer alles neu laden, sondern erst ab offsets[partition]!
+      consumer.seek(topicPartition, unseenOffset);
     });
+
+    consumer.resume(partitions);
   }
 
   @Override
   public void onPartitionsRevoked(Collection<TopicPartition> partitions)
   {
+    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...?
+    });
     log.info("Revoked partitions: {}", partitions);
   }
 
   @Override
   public void onPartitionsLost(Collection<TopicPartition> partitions)
   {
-    log.info("Revoked partitions: {}", partitions);
+    // TODO: Muss auf den Verlust anders reagiert werden?
+    onPartitionsRevoked(partitions);
   }
 
-  private void foo()
+  @Override
+  public void run()
   {
-    Set<Integer> owned = Arrays
-      .stream(ownedShards)
-      .collect(
-        () -> new HashSet<>(),
-        (set, i) -> set.add(i),
-        (a, b) -> a.addAll(b));
-    for (int shard = 0; shard < numShards; shard++)
+    consumer.subscribe(List.of(topic));
+
+    running = true;
+
+    try
     {
-      chatrooms[shard] = owned.contains(shard)
-        ? new HashMap<>()
-        : null;
-    }
-    chatroomFlux
-      .filter(chatRoom ->
+      while (running)
       {
-        if (owned.contains(chatRoom.getShard()))
+        ConsumerRecords<String, MessageTo> records = consumer.poll(Duration.ofMinutes(5));
+        log.info("Fetched {} messages", records.count());
+
+        if (loadInProgress)
         {
-          return true;
+          for (ConsumerRecord<String, MessageTo> record : records)
+          {
+            UUID chatRoomId = UUID.fromString(record.key());
+            MessageTo messageTo = record.value();
+            ChatRoom chatRoom = chatRoomMaps[record.partition()].get(chatRoomId);
+            KafkaChatRoomService kafkaChatRoomService =
+                (KafkaChatRoomService) chatRoom.getChatRoomService();
+            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());
+            kafkaChatRoomService.persistMessage(message);
+          }
         }
         else
         {
-          log.info("Ignoring not owned chat-room {}", chatRoom);
-          return false;
+          if (!records.isEmpty())
+          {
+            throw new IllegalStateException("All owned partions should be paused, when no load is in progress!");
+          }
         }
-      })
-      .toStream()
-      .forEach(chatroom -> chatrooms[chatroom.getShard()].put(chatroom.getId(), chatroom));
+      }
+    }
   }
 
-  @Override
-  public Mono<ChatRoom> putChatRoom(ChatRoom chatRoom)
+  Mono<Message> sendMessage(
+      UUID chatRoomId,
+      Message.MessageKey key,
+      LocalDateTime timestamp,
+      String text)
   {
-    chatrooms[chatRoom.getShard()].put(chatRoom.getId(), chatRoom);
-    return Mono.just(chatRoom);
+    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(chatrooms[shard].get(id));
-  }
 
   @Override
-  public Flux<ChatRoom> getChatRooms(int shard)
-  {
-    return Flux.fromStream(chatrooms[shard].values().stream());
-  }
-
-
-  interface MessageHandler
-  {
-    MessageHandler handleMessage(Message message);
-  }
-
-
-  @RequiredArgsConstructor
-  class NoOpMessageHandler implements MessageHandler
-  {
-    private final TopicPartition tp;
-
-    @Override
-    public MessageHandler handleMessage(Message message)
-    {
-      log.warn("Not handling message {} for partition {}", message, tp);
-      return this;
-    }
-  }
-
-  class ChatRoomLoadingMessageHandler implements MessageHandler
+  public Mono<ChatRoom> getChatRoom(int shard, UUID id)
   {
-    private final TopicPartition tp;
-    private final long currentOffset;
-    private final long unseenOffset;
-
-    ChatRoomLoadingMessageHandler(TopicPartition tp, long currentOffset, long unseenOffset)
+    if (loadInProgress)
     {
-      this.tp = tp;
-      this.currentOffset = currentOffset;
-      this.unseenOffset = unseenOffset;
-
-      consumer.seek(tp, unseenOffset);
+      throw new ShardNotOwnedException(shard);
     }
-
-    @Override
-    public MessageHandler handleMessage(Message message)
+    else
     {
-      // todo
-      return this;
+      return Mono.justOrEmpty(chatRoomMaps[shard].get(id));
     }
   }
 
-  @RequiredArgsConstructor
-  class DefaultMessageHandler implements MessageHandler
+  @Override
+  public Flux<ChatRoom> getChatRooms(int shard)
   {
-    private final TopicPartition tp;
-
-    @Override
-    public MessageHandler handleMessage(Message message)
+    if (loadInProgress)
+    {
+      throw new ShardNotOwnedException(shard);
+    }
+    else
     {
-      chatrooms[tp.partition()].put()
-      return this;
+      return Flux.fromStream(chatRoomMaps[shard].values().stream());
     }
   }
 }