NG
[demos/kafka/chat] / src / main / java / de / juplo / kafka / chat / backend / persistence / kafka / ChatMessageChannel.java
index 7620461..94f6fa6 100644 (file)
@@ -1,8 +1,8 @@
 package de.juplo.kafka.chat.backend.persistence.kafka;
 
 import de.juplo.kafka.chat.backend.domain.ChatRoom;
+import de.juplo.kafka.chat.backend.domain.ChatRoomInfo;
 import de.juplo.kafka.chat.backend.domain.Message;
-import de.juplo.kafka.chat.backend.persistence.KafkaLikeShardingStrategy;
 import lombok.Getter;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.kafka.clients.consumer.Consumer;
@@ -12,16 +12,12 @@ 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.*;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import java.util.*;
 import java.util.stream.IntStream;
 
 
@@ -29,15 +25,16 @@ import java.util.stream.IntStream;
 public class ChatMessageChannel implements Runnable, ConsumerRebalanceListener
 {
   private final String topic;
-  private final Consumer<String, MessageTo> consumer;
-  private final Producer<String, MessageTo> producer;
+  private final Producer<String, AbstractTo> producer;
+  private final Consumer<String, AbstractTo> consumer;
   private final ZoneId zoneId;
   private final int numShards;
+  private final int bufferSize;
+  private final Clock clock;
   private final boolean[] isShardOwned;
   private final long[] currentOffset;
   private final long[] nextOffset;
   private final Map<UUID, ChatRoom>[] chatrooms;
-  private final KafkaLikeShardingStrategy shardingStrategy;
 
   private boolean running;
   @Getter
@@ -46,10 +43,12 @@ public class ChatMessageChannel implements Runnable, ConsumerRebalanceListener
 
   public ChatMessageChannel(
     String topic,
-    Consumer<String, MessageTo> consumer,
-    Producer<String, MessageTo> producer,
+    Producer<String, AbstractTo> producer,
+    Consumer<String, AbstractTo> consumer,
     ZoneId zoneId,
-    int numShards)
+    int numShards,
+    int bufferSize,
+    Clock clock)
   {
     log.debug(
         "Creating ChatMessageChannel for topic {} with {} partitions",
@@ -60,13 +59,96 @@ public class ChatMessageChannel implements Runnable, ConsumerRebalanceListener
     this.producer = producer;
     this.zoneId = zoneId;
     this.numShards = numShards;
+    this.bufferSize = bufferSize;
+    this.clock = clock;
     this.isShardOwned = new boolean[numShards];
     this.currentOffset = new long[numShards];
     this.nextOffset = new long[numShards];
     this.chatrooms = new Map[numShards];
-    this.shardingStrategy = new KafkaLikeShardingStrategy(numShards);
+    IntStream
+        .range(0, numShards)
+        .forEach(shard -> this.chatrooms[shard] = new HashMap<>());
+  }
+
+
+
+  Mono<ChatRoomInfo> sendCreateChatRoomRequest(
+      UUID chatRoomId,
+      String name)
+  {
+    CreateChatRoomRequestTo createChatRoomRequestTo = CreateChatRoomRequestTo.of(name);
+    return Mono.create(sink ->
+    {
+      ProducerRecord<String, CreateChatRoomRequestTo> record =
+          new ProducerRecord<>(
+              topic,
+              chatRoomId.toString(),
+              createChatRoomRequestTo);
+
+      producer.send(record, ((metadata, exception) ->
+      {
+        if (metadata != null)
+        {
+          log.info("Successfully send chreate-request for chat room: {}", createChatRoomRequestTo);
+          ChatRoomInfo chatRoomInfo = ChatRoomInfo.of(chatRoomId, name, record.partition());
+          createChatRoom(chatRoomInfo);
+          sink.success(chatRoomInfo);
+        }
+        else
+        {
+          // On send-failure
+          log.error(
+              "Could not send create-request for chat room (id={}, name={}): {}",
+              chatRoomId,
+              name,
+              exception);
+          sink.error(exception);
+        }
+      }));
+    });
   }
 
+  Mono<Message> sendChatMessage(
+      UUID chatRoomId,
+      Message.MessageKey key,
+      LocalDateTime timestamp,
+      String text)
+  {
+    ZonedDateTime zdt = ZonedDateTime.of(timestamp, zoneId);
+    return Mono.create(sink ->
+    {
+      ProducerRecord<String, AbstractTo> record =
+          new ProducerRecord<>(
+              topic,
+              null,
+              zdt.toEpochSecond(),
+              chatRoomId.toString(),
+              ChatMessageTo.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 void onPartitionsAssigned(Collection<TopicPartition> partitions)
@@ -114,7 +196,7 @@ public class ChatMessageChannel implements Runnable, ConsumerRebalanceListener
   @Override
   public void run()
   {
-    consumer.subscribe(List.of(topic));
+    consumer.subscribe(List.of(topic), this);
 
     running = true;
 
@@ -122,12 +204,12 @@ public class ChatMessageChannel implements Runnable, ConsumerRebalanceListener
     {
       try
       {
-        ConsumerRecords<String, MessageTo> records = consumer.poll(Duration.ofMinutes(5));
+        ConsumerRecords<String, AbstractTo> records = consumer.poll(Duration.ofMinutes(5));
         log.info("Fetched {} messages", records.count());
 
         if (loadInProgress)
         {
-          loadMessages(records);
+          loadChatRoom(records);
 
           if (isLoadingCompleted())
           {
@@ -147,50 +229,98 @@ public class ChatMessageChannel implements Runnable, ConsumerRebalanceListener
       }
       catch (WakeupException e)
       {
-      }
-      catch (RecordDeserializationException e)
-      {
+        log.info("Received WakeupException, exiting!");
+        running = false;
       }
     }
+
+    log.info("Exiting normally");
   }
 
-  void loadMessages(ConsumerRecords<String, MessageTo> records)
+  void loadChatRoom(ConsumerRecords<String, AbstractTo> records)
   {
-    for (ConsumerRecord<String, MessageTo> record : records)
+    for (ConsumerRecord<String, AbstractTo> 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 = chatrooms[record.partition()].get(chatRoomId);
-      if (chatRoom == null)
+      switch (record.value().getType())
       {
-        // Alles pausieren und erst von putChatRoom wieder resumen lassen!
+        case CREATE_CHATROOM_REQUEST:
+          createChatRoom(
+              chatRoomId,
+              (CreateChatRoomRequestTo) record.value(),
+              record.partition());
+          break;
+
+        case MESSAGE_SENT:
+          Instant instant = Instant.ofEpochSecond(record.timestamp());
+          LocalDateTime timestamp = LocalDateTime.ofInstant(instant, zoneId);
+          loadChatMessage(
+              chatRoomId,
+              timestamp,
+              record.offset(),
+              (ChatMessageTo) record.value(),
+              record.partition());
+          break;
+
+        default:
+          log.debug(
+              "Ignoring message for chat-room {} with offset {}: {}",
+              chatRoomId,
+              record.offset(),
+              record.value());
       }
-      KafkaChatRoomService kafkaChatRoomService =
-          (KafkaChatRoomService) chatRoom.getChatRoomService();
 
-      kafkaChatRoomService.persistMessage(message);
+      nextOffset[record.partition()] = record.offset() + 1;
     }
   }
 
+  void createChatRoom(
+      UUID chatRoomId,
+      CreateChatRoomRequestTo createChatRoomRequestTo,
+      int partition)
+  {
+    putChatRoom(ChatRoomInfo.of(
+        chatRoomId,
+        createChatRoomRequestTo.getName(),
+        partition));
+  }
+
+
+  void createChatRoom(ChatRoomInfo chatRoomInfo)
+  {
+    UUID id = chatRoomInfo.getId();
+    String name = chatRoomInfo.getName();
+    int shard = chatRoomInfo.getShard();
+    log.info("Creating ChatRoom {} with buffer-size {}", id, bufferSize);
+    KafkaChatRoomService service = new KafkaChatRoomService(this, id);
+    ChatRoom chatRoom = new ChatRoom(id, name, shard, clock, service, bufferSize);
+    putChatRoom(chatRoom);
+  }
+
+  void loadChatMessage(
+      UUID chatRoomId,
+      LocalDateTime timestamp,
+      long offset,
+      ChatMessageTo chatMessageTo,
+      int partition)
+  {
+    Message.MessageKey key = Message.MessageKey.of(chatMessageTo.getUser(), chatMessageTo.getId());
+    Message message = new Message(key, offset, timestamp, chatMessageTo.getText());
+
+    ChatRoom chatRoom = chatrooms[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, // TODO: Boolean is immutable
-            (acc, v) -> Boolean.valueOf(acc && v), // TODO: Boolean is immutable
-            (a, b) -> Boolean.valueOf(a && b)); // TODO: Boolean is immutable
+        .allMatch(shard -> nextOffset[shard] >= currentOffset[shard]);
   }
 
   void pauseAllOwnedPartions()
@@ -202,63 +332,28 @@ public class ChatMessageChannel implements Runnable, ConsumerRebalanceListener
         .toList());
   }
 
-  Mono<Message> sendMessage(
-      UUID chatRoomId,
-      Message.MessageKey key,
-      LocalDateTime timestamp,
-      String text)
+
+  private void putChatRoom(ChatRoom chatRoom)
   {
-    int shard = this.shardingStrategy.selectShard(chatRoomId);
-    TopicPartition tp = new TopicPartition(topic, shard);
-    ZonedDateTime zdt = ZonedDateTime.of(timestamp, zoneId);
-    return Mono.create(sink ->
+    Integer partition = chatRoom.getShard();
+    UUID chatRoomId = chatRoom.getId();
+    if (chatrooms[partition].containsKey(chatRoomId))
     {
-      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);
-        }
-      }));
-    });
-  }
-
+      log.warn("Ignoring existing chat-room: " + chatRoom);
+    }
+    else
+    {
+      log.info(
+          "Adding new chat-room to partition {}: {}",
+          partition,
+          chatRoom);
 
-  void putChatRoom(ChatRoom chatRoom)
-  {
-    chatrooms[chatRoom.getShard()].put(chatRoom.getId(), chatRoom);
+      chatrooms[partition].put(chatRoomId, chatRoom);
+    }
   }
 
   Mono<ChatRoom> getChatRoom(int shard, UUID id)
   {
     return Mono.justOrEmpty(chatrooms[shard].get(id));
   }
-
-  Flux<ChatRoom> getChatRooms(int shard)
-  {
-    return Flux.fromStream(chatrooms[shard].values().stream());
-  }
 }