NG
[demos/kafka/chat] / src / main / java / de / juplo / kafka / chat / backend / persistence / kafka / ChatRoomChannel.java
index 0c553e6..8bbc82e 100644 (file)
@@ -1,59 +1,37 @@
 package de.juplo.kafka.chat.backend.persistence.kafka;
 
 import de.juplo.kafka.chat.backend.domain.*;
-import de.juplo.kafka.chat.backend.persistence.KafkaLikeShardingStrategy;
+import lombok.RequiredArgsConstructor;
 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.*;
-import java.util.Collection;
 import java.util.List;
-import java.util.Map;
 import java.util.UUID;
-import java.util.stream.IntStream;
 
 
+@RequiredArgsConstructor
 @Slf4j
 public class ChatRoomChannel implements Runnable
 {
   private final String topic;
-  private final Consumer<Integer, ChatRoomTo> consumer;
-  private final Producer<Integer, ChatRoomTo> producer;
+  private final Producer<Integer, CreateChatRoomRequestTo> producer;
+  private final Consumer<Integer, CreateChatRoomRequestTo> consumer;
   private final ShardingStrategy shardingStrategy;
   private final ChatMessageChannel chatMessageChannel;
+  private final Clock clock;
+  private final int bufferSize;
 
   private boolean running;
 
 
-  public ChatRoomChannel(
-    String topic,
-    Consumer<Integer, ChatRoomTo> consumer,
-    Producer<Integer, ChatRoomTo> producer,
-    int numShards,
-    ChatMessageChannel chatMessageChannel)
-  {
-    log.debug(
-        "Creating ChatRoomChannel for topic {} with sharding for {} partitions",
-        topic,
-        numShards);
-    this.topic = topic;
-    this.consumer = consumer;
-    this.producer = producer;
-    this.shardingStrategy = new KafkaLikeShardingStrategy(numShards);
-    this.chatMessageChannel = chatMessageChannel;
-  }
-
-
   @Override
   public void run()
   {
@@ -65,54 +43,33 @@ public class ChatRoomChannel implements Runnable
     {
       try
       {
-        ConsumerRecords<String, MessageTo> records = consumer.poll(Duration.ofMinutes(5));
+        ConsumerRecords<Integer, CreateChatRoomRequestTo> records = consumer.poll(Duration.ofMinutes(5));
         log.info("Fetched {} messages", records.count());
 
+        for (ConsumerRecord<Integer, CreateChatRoomRequestTo> record : records)
+        {
+          createChatRoom(record.value().toChatRoomInfo());
+        }
       }
       catch (WakeupException e)
       {
-      }
-      catch (RecordDeserializationException e)
-      {
+        log.info("Received WakeupException, exiting!");
+        running = false;
       }
     }
+
+    log.info("Exiting normally");
   }
 
 
-  Mono<ChatRoomInfo> sendCreateChatRoomRequest(
-      UUID chatRoomId,
-      String name)
+  void createChatRoom(ChatRoomInfo chatRoomInfo)
   {
-    int shard = this.shardingStrategy.selectShard(chatRoomId);
-    ChatRoomTo chatRoomTo = ChatRoomTo.of(chatRoomId, name, shard);
-    return Mono.create(sink ->
-    {
-      ProducerRecord<Integer, ChatRoomTo> record =
-          new ProducerRecord<>(
-              topic,
-              shard,
-              chatRoomTo);
-
-      producer.send(record, ((metadata, exception) ->
-      {
-        if (metadata != null)
-        {
-          log.info("Successfully send chreate-request {}", chatRoomTo);
-          sink.success(chatRoomTo.toChatRoomInfo());
-        }
-        else
-        {
-          // On send-failure
-          log.error(
-              "Could not create-request for chat-room={}, key={}, timestamp={}, text={}: {}",
-              chatRoomId,
-              key,
-              timestamp,
-              text,
-              exception);
-          sink.error(exception);
-        }
-      }));
-    });
+    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(chatMessageChannel, id);
+    ChatRoom chatRoom = new ChatRoom(id, name, shard, clock, service, bufferSize);
+    chatMessageChannel.putChatRoom(chatRoom);
   }
 }