NEU
[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 org.apache.kafka.common.errors.RecordDeserializationException;
17 import org.apache.kafka.common.errors.WakeupException;
18 import reactor.core.publisher.Flux;
19 import reactor.core.publisher.Mono;
20
21 import java.time.*;
22 import java.util.*;
23 import java.util.stream.IntStream;
24
25
26 @Slf4j
27 public class KafkaChatHomeService implements ChatHomeService, Runnable, ConsumerRebalanceListener
28 {
29   private final Consumer<String, MessageTo> consumer;
30   private final Producer<String, MessageTo> producer;
31   private final String topic;
32   private final ZoneId zoneId;
33   private final int numShards;
34   private final boolean[] isShardOwned;
35   private final long[] currentOffset;
36   private final long[] nextOffset;
37   private final Map<UUID, ChatRoom>[] chatrooms;
38   private final KafkaLikeShardingStrategy shardingStrategy;
39
40   private boolean running;
41   private volatile boolean loadInProgress;
42
43
44   public KafkaChatHomeService(
45     Consumer<String, MessageTo> consumer,
46     Producer<String, MessageTo> producer,
47     String topic,
48     ZoneId zoneId,
49     int numShards)
50   {
51     log.debug("Creating KafkaChatHomeService");
52     this.consumer = consumer;
53     this.producer = producer;
54     this.topic = topic;
55     this.zoneId = zoneId;
56     this.numShards = numShards;
57     this.isShardOwned = new boolean[numShards];
58     this.currentOffset = new long[numShards];
59     this.nextOffset = new long[numShards];
60     this.chatrooms = new Map[numShards];
61     this.shardingStrategy = new KafkaLikeShardingStrategy(numShards);
62   }
63
64
65   @Override
66   public void onPartitionsAssigned(Collection<TopicPartition> partitions)
67   {
68     log.info("Newly assigned partitions! Pausing normal operations...");
69     loadInProgress = true;
70
71     consumer.endOffsets(partitions).forEach((topicPartition, currentOffset) ->
72     {
73       int partition = topicPartition.partition();
74       isShardOwned[partition] =  true;
75       this.currentOffset[partition] = currentOffset;
76
77       log.info(
78           "Partition assigned: {} - loading messages: next={} -> current={}",
79           partition,
80           nextOffset[partition],
81           currentOffset);
82
83       consumer.seek(topicPartition, nextOffset[partition]);
84     });
85
86     consumer.resume(partitions);
87   }
88
89   @Override
90   public void onPartitionsRevoked(Collection<TopicPartition> partitions)
91   {
92     partitions.forEach(topicPartition ->
93     {
94       int partition = topicPartition.partition();
95       isShardOwned[partition] = false;
96       log.info("Partition revoked: {} - next={}", partition, nextOffset[partition]);
97     });
98   }
99
100   @Override
101   public void onPartitionsLost(Collection<TopicPartition> partitions)
102   {
103     log.warn("Lost partitions: {}, partitions");
104     // TODO: Muss auf den Verlust anders reagiert werden?
105     onPartitionsRevoked(partitions);
106   }
107
108   @Override
109   public void run()
110   {
111     consumer.subscribe(List.of(topic));
112
113     running = true;
114
115     while (running)
116     {
117       try
118       {
119         ConsumerRecords<String, MessageTo> records = consumer.poll(Duration.ofMinutes(5));
120         log.info("Fetched {} messages", records.count());
121
122         if (loadInProgress)
123         {
124           loadMessages(records);
125
126           if (isLoadingCompleted())
127           {
128             log.info("Loading of messages completed! Pausing all owned partitions...");
129             pauseAllOwnedPartions();
130             log.info("Resuming normal operations...");
131             loadInProgress = false;
132           }
133         }
134         else
135         {
136           if (!records.isEmpty())
137           {
138             throw new IllegalStateException("All owned partitions should be paused, when no load is in progress!");
139           }
140         }
141       }
142       catch (WakeupException e)
143       {
144       }
145       catch (RecordDeserializationException e)
146       {
147       }
148     }
149   }
150
151   void loadMessages(ConsumerRecords<String, MessageTo> records)
152   {
153     for (ConsumerRecord<String, MessageTo> record : records)
154     {
155       nextOffset[record.partition()] = record.offset() + 1;
156       UUID chatRoomId = UUID.fromString(record.key());
157       MessageTo messageTo = record.value();
158
159       Message.MessageKey key = Message.MessageKey.of(messageTo.getUser(), messageTo.getId());
160
161       Instant instant = Instant.ofEpochSecond(record.timestamp());
162       LocalDateTime timestamp = LocalDateTime.ofInstant(instant, zoneId);
163
164       Message message = new Message(key, record.offset(), timestamp, messageTo.getText());
165
166       ChatRoom chatRoom = chatrooms[record.partition()].get(chatRoomId);
167       KafkaChatRoomService kafkaChatRoomService =
168           (KafkaChatRoomService) chatRoom.getChatRoomService();
169
170       kafkaChatRoomService.persistMessage(message);
171     }
172   }
173
174   boolean isLoadingCompleted()
175   {
176     return IntStream
177         .range(0, numShards)
178         .filter(shard -> isShardOwned[shard])
179         .mapToObj(shard -> nextOffset[shard] >= currentOffset[shard])
180         .collect(
181             () -> Boolean.TRUE,
182             (acc, v) -> Boolean.valueOf(acc && v),
183             (a, b) -> Boolean.valueOf(a && b));
184   }
185
186   void pauseAllOwnedPartions()
187   {
188     consumer.pause(IntStream
189         .range(0, numShards)
190         .filter(shard -> isShardOwned[shard])
191         .mapToObj(shard -> new TopicPartition(topic, shard))
192         .toList());
193   }
194
195   Mono<Message> sendMessage(
196       UUID chatRoomId,
197       Message.MessageKey key,
198       LocalDateTime timestamp,
199       String text)
200   {
201     int shard = this.shardingStrategy.selectShard(chatRoomId);
202     TopicPartition tp = new TopicPartition(topic, shard);
203     ZonedDateTime zdt = ZonedDateTime.of(timestamp, zoneId);
204     return Mono.create(sink ->
205     {
206       ProducerRecord<String, MessageTo> record =
207           new ProducerRecord<>(
208               tp.topic(),
209               tp.partition(),
210               zdt.toEpochSecond(),
211               chatRoomId.toString(),
212               MessageTo.of(key.getUsername(), key.getMessageId(), text));
213
214       producer.send(record, ((metadata, exception) ->
215       {
216         if (metadata != null)
217         {
218           // On successful send
219           Message message = new Message(key, metadata.offset(), timestamp, text);
220           log.info("Successfully send message {}", message);
221           sink.success(message);
222         }
223         else
224         {
225           // On send-failure
226           log.error(
227               "Could not send message for chat-room={}, key={}, timestamp={}, text={}: {}",
228               chatRoomId,
229               key,
230               timestamp,
231               text,
232               exception);
233           sink.error(exception);
234         }
235       }));
236     });
237   }
238
239
240   public void putChatRoom(ChatRoom chatRoom)
241   {
242     // TODO: Nachricht senden!
243     chatrooms[chatRoom.getShard()].put(chatRoom.getId(), chatRoom);
244   }
245
246   @Override
247   public Mono<ChatRoom> getChatRoom(int shard, UUID id)
248   {
249     if (loadInProgress)
250     {
251       throw new ShardNotOwnedException(shard);
252     }
253     else
254     {
255       return Mono.justOrEmpty(chatrooms[shard].get(id));
256     }
257   }
258
259   @Override
260   public Flux<ChatRoom> getChatRooms(int shard)
261   {
262     if (loadInProgress)
263     {
264       throw new ShardNotOwnedException(shard);
265     }
266     else
267     {
268       return Flux.fromStream(chatrooms[shard].values().stream());
269     }
270   }
271 }