5133d1a68203713f970a5106a73b67de5d1f17e7
[demos/kafka/chat] / src / main / java / de / juplo / kafka / chat / backend / persistence / kafka / KafkaChatHomeService.java
1 package de.juplo.kafka.chat.backend.persistence.kafka;
2
3 import de.juplo.kafka.chat.backend.domain.ChatHomeService;
4 import de.juplo.kafka.chat.backend.domain.ChatRoom;
5 import de.juplo.kafka.chat.backend.domain.Message;
6 import de.juplo.kafka.chat.backend.domain.ShardNotOwnedException;
7 import de.juplo.kafka.chat.backend.persistence.KafkaLikeShardingStrategy;
8 import lombok.extern.slf4j.Slf4j;
9 import org.apache.kafka.clients.consumer.Consumer;
10 import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
11 import org.apache.kafka.clients.consumer.ConsumerRecord;
12 import org.apache.kafka.clients.consumer.ConsumerRecords;
13 import org.apache.kafka.clients.producer.Producer;
14 import org.apache.kafka.clients.producer.ProducerRecord;
15 import org.apache.kafka.common.TopicPartition;
16 import reactor.core.publisher.Flux;
17 import reactor.core.publisher.Mono;
18
19 import java.time.*;
20 import java.util.*;
21 import java.util.concurrent.ExecutorService;
22
23
24 @Slf4j
25 public class KafkaChatHomeService implements ChatHomeService, Runnable, ConsumerRebalanceListener
26 {
27   private final ExecutorService executorService;
28   private final Consumer<String, MessageTo> consumer;
29   private final Producer<String, MessageTo> producer;
30   private final String topic;
31   private final ZoneId zoneId;
32   // private final long[] offsets; Erst mal immer alles neu einlesen
33   private final boolean[] isShardOwned;
34   private final Map<UUID, ChatRoom>[] chatRoomMaps;
35   private final KafkaLikeShardingStrategy shardingStrategy;
36
37   private boolean running;
38   private volatile boolean loadInProgress;
39
40
41   public KafkaChatHomeService(
42     ExecutorService executorService,
43     Consumer<String, MessageTo> consumer,
44     Producer<String, MessageTo> producer,
45     String topic,
46     ZoneId zoneId,
47     int numShards)
48   {
49     log.debug("Creating KafkaChatHomeService");
50     this.executorService = executorService;
51     this.consumer = consumer;
52     this.producer = producer;
53     this.topic = topic;
54     this.zoneId = zoneId;
55     // this.offsets = new long[numShards];
56     // for (int i=0; i< numShards; i++)
57     // {
58     //   this.offsets[i] = 0l;
59     // }
60     this.isShardOwned = new boolean[numShards];
61     this.chatRoomMaps = new Map[numShards];
62     this.shardingStrategy = new KafkaLikeShardingStrategy(numShards);
63   }
64
65
66   @Override
67   public void onPartitionsAssigned(Collection<TopicPartition> partitions)
68   {
69     loadInProgress = true;
70
71     consumer.endOffsets(partitions).forEach((topicPartition, currentOffset) ->
72     {
73       if (!topicPartition.topic().equals(topic))
74       {
75         log.warn("Ignoring partition from unwanted topic: {}", topicPartition);
76         return;
77       }
78
79       int partition = topicPartition.partition();
80       long unseenOffset = 0; // offsets[partition];
81
82       log.info(
83           "Loading messages from partition {}: start-offset={} -> current-offset={}",
84           partition,
85           unseenOffset,
86           currentOffset);
87
88       // TODO: reuse! Nicht immer alles neu laden, sondern erst ab offsets[partition]!
89       consumer.seek(topicPartition, unseenOffset);
90     });
91
92     consumer.resume(partitions);
93   }
94
95   @Override
96   public void onPartitionsRevoked(Collection<TopicPartition> partitions)
97   {
98     partitions.forEach(topicPartition ->
99     {
100       if (!topicPartition.topic().equals(topic))
101       {
102         log.warn("Ignoring partition from unwanted topic: {}", topicPartition);
103         return;
104       }
105
106       int partition = topicPartition.partition();
107       // long unseenOffset = offsets[partition]; TODO: Offset merken...?
108     });
109     log.info("Revoked partitions: {}", partitions);
110   }
111
112   @Override
113   public void onPartitionsLost(Collection<TopicPartition> partitions)
114   {
115     // TODO: Muss auf den Verlust anders reagiert werden?
116     onPartitionsRevoked(partitions);
117   }
118
119   @Override
120   public void run()
121   {
122     consumer.subscribe(List.of(topic));
123
124     running = true;
125
126     try
127     {
128       while (running)
129       {
130         ConsumerRecords<String, MessageTo> records = consumer.poll(Duration.ofMinutes(5));
131         log.info("Fetched {} messages", records.count());
132
133         if (loadInProgress)
134         {
135           for (ConsumerRecord<String, MessageTo> record : records)
136           {
137             UUID chatRoomId = UUID.fromString(record.key());
138             MessageTo messageTo = record.value();
139             ChatRoom chatRoom = chatRoomMaps[record.partition()].get(chatRoomId);
140             Mono<Message> result = chatRoom.addMessage(
141                 messageTo.getId(),
142                 messageTo.getUser(),
143                 messageTo.getText());
144             result.block().
145           }
146         }
147         else
148         {
149           if (!records.isEmpty())
150           {
151             throw new IllegalStateException("All owned partions should be paused, when no load is in progress!");
152           }
153         }
154       }
155     }
156   }
157
158   Mono<Message> sendMessage(
159       UUID chatRoomId,
160       Message.MessageKey key,
161       LocalDateTime timestamp,
162       String text)
163   {
164     int shard = this.shardingStrategy.selectShard(chatRoomId);
165     TopicPartition tp = new TopicPartition(topic, shard);
166     ZonedDateTime zdt = ZonedDateTime.of(timestamp, zoneId);
167     return Mono.create(sink ->
168     {
169       ProducerRecord<String, MessageTo> record =
170           new ProducerRecord<>(
171               tp.topic(),
172               tp.partition(),
173               zdt.toEpochSecond(),
174               chatRoomId.toString(),
175               MessageTo.of(key.getUsername(), key.getMessageId(), text));
176
177       producer.send(record, ((metadata, exception) ->
178       {
179         if (metadata != null)
180         {
181           // On successful send
182           Message message = new Message(key, metadata.offset(), timestamp, text);
183           log.info("Successfully send message {}", message);
184           sink.success(message);
185         }
186         else
187         {
188           // On send-failure
189           log.error(
190               "Could not send message for chat-room={}, key={}, timestamp={}, text={}: {}",
191               chatRoomId,
192               key,
193               timestamp,
194               text,
195               exception);
196           sink.error(exception);
197         }
198       }));
199     });
200   }
201
202
203   @Override
204   public Mono<ChatRoom> getChatRoom(int shard, UUID id)
205   {
206     if (loadInProgress)
207     {
208       throw new ShardNotOwnedException(shard);
209     }
210     else
211     {
212       return Mono.justOrEmpty(chatRoomMaps[shard].get(id));
213     }
214   }
215
216   @Override
217   public Flux<ChatRoom> getChatRooms(int shard)
218   {
219     if (loadInProgress)
220     {
221       throw new ShardNotOwnedException(shard);
222     }
223     else
224     {
225       return Flux.fromStream(chatRoomMaps[shard].values().stream());
226     }
227   }
228 }