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