abe51f4a61aa5c737ab7d8ba793115e1cd29745e
[demos/kafka/chat] / src / main / java / de / juplo / kafka / chat / backend / implementation / kafka / DataChannel.java
1 package de.juplo.kafka.chat.backend.implementation.kafka;
2
3 import de.juplo.kafka.chat.backend.domain.ChatRoomData;
4 import de.juplo.kafka.chat.backend.domain.ChatRoomInfo;
5 import de.juplo.kafka.chat.backend.domain.Message;
6 import de.juplo.kafka.chat.backend.domain.ShardingPublisherStrategy;
7 import de.juplo.kafka.chat.backend.domain.exceptions.ShardNotOwnedException;
8 import de.juplo.kafka.chat.backend.implementation.kafka.messages.AbstractMessageTo;
9 import de.juplo.kafka.chat.backend.implementation.kafka.messages.data.EventChatMessageReceivedTo;
10 import lombok.Getter;
11 import lombok.ToString;
12 import lombok.extern.slf4j.Slf4j;
13 import org.apache.kafka.clients.consumer.*;
14 import org.apache.kafka.clients.producer.Producer;
15 import org.apache.kafka.clients.producer.ProducerRecord;
16 import org.apache.kafka.common.TopicPartition;
17 import org.apache.kafka.common.errors.WakeupException;
18 import reactor.core.publisher.Mono;
19
20 import java.time.*;
21 import java.util.Collection;
22 import java.util.HashMap;
23 import java.util.Map;
24 import java.util.UUID;
25 import java.util.stream.IntStream;
26
27
28 @ToString(of = { "topic", "instanceId" })
29 @Slf4j
30 public class DataChannel implements Channel, ConsumerRebalanceListener
31 {
32   private final String instanceId;
33   private final String topic;
34   private final Producer<String, AbstractMessageTo> producer;
35   private final Consumer<String, AbstractMessageTo> consumer;
36   private final ZoneId zoneId;
37   private final int numShards;
38   private final Duration pollingInterval;
39   private final int historyLimit;
40   private final Clock clock;
41   private final boolean[] isShardOwned;
42   private final long[] currentOffset;
43   private final long[] nextOffset;
44   private final Map<UUID, ChatRoomData>[] chatRoomData;
45   private final ChannelMediator channelMediator;
46   private final ShardingPublisherStrategy shardingPublisherStrategy;
47
48   private boolean running;
49   @Getter
50   private volatile ChannelState channelState = ChannelState.STARTING;
51
52
53   public DataChannel(
54     String instanceId,
55     String topic,
56     Producer<String, AbstractMessageTo> producer,
57     Consumer<String, AbstractMessageTo> dataChannelConsumer,
58     ZoneId zoneId,
59     int numShards,
60     Duration pollingInterval,
61     int historyLimit,
62     Clock clock,
63     ChannelMediator channelMediator,
64     ShardingPublisherStrategy shardingPublisherStrategy)
65   {
66     log.debug(
67         "{}: Creating DataChannel for topic {} with {} partitions",
68         instanceId,
69         topic,
70         numShards);
71     this.instanceId = instanceId;
72     this.topic = topic;
73     this.consumer = dataChannelConsumer;
74     this.producer = producer;
75     this.zoneId = zoneId;
76     this.numShards = numShards;
77     this.pollingInterval = pollingInterval;
78     this.historyLimit = historyLimit;
79     this.clock = clock;
80     this.isShardOwned = new boolean[numShards];
81     this.currentOffset = new long[numShards];
82     this.nextOffset = new long[numShards];
83     this.chatRoomData = new Map[numShards];
84     IntStream
85         .range(0, numShards)
86         .forEach(shard -> this.chatRoomData[shard] = new HashMap<>());
87     this.channelMediator = channelMediator;
88     this.shardingPublisherStrategy = shardingPublisherStrategy;
89   }
90
91
92
93   Mono<Message> sendChatMessage(
94       UUID chatRoomId,
95       Message.MessageKey key,
96       LocalDateTime timestamp,
97       String text)
98   {
99     ZonedDateTime zdt = ZonedDateTime.of(timestamp, zoneId);
100     return Mono.create(sink ->
101     {
102       ProducerRecord<String, AbstractMessageTo> record =
103           new ProducerRecord<>(
104               topic,
105               null,
106               zdt.toEpochSecond(),
107               chatRoomId.toString(),
108               EventChatMessageReceivedTo.of(key.getUsername(), key.getMessageId(), text));
109
110       producer.send(record, ((metadata, exception) ->
111       {
112         if (exception == null)
113         {
114           // On successful send
115           Message message = new Message(key, metadata.offset(), timestamp, text);
116           log.info("Successfully send message {}", message);
117           sink.success(message);
118         }
119         else
120         {
121           // On send-failure
122           log.error(
123               "Could not send message for chat-room={}, key={}, timestamp={}, text={}: {}",
124               chatRoomId,
125               key,
126               timestamp,
127               text,
128               exception);
129           sink.error(exception);
130         }
131       }));
132     });
133   }
134
135   @Override
136   public void onPartitionsAssigned(Collection<TopicPartition> partitions)
137   {
138     log.info("Newly assigned partitions! Pausing normal operations...");
139     channelState = ChannelState.LOAD_IN_PROGRESS;
140
141     consumer.endOffsets(partitions).forEach((topicPartition, currentOffset) ->
142     {
143       int partition = topicPartition.partition();
144       isShardOwned[partition] =  true;
145       this.currentOffset[partition] = currentOffset;
146
147       log.info(
148           "Partition assigned: {} - loading messages: next={} -> current={}",
149           partition,
150           nextOffset[partition],
151           currentOffset);
152
153       consumer.seek(topicPartition, nextOffset[partition]);
154       channelMediator.shardAssigned(partition);
155       shardingPublisherStrategy
156           .publishOwnership(partition)
157           .doOnSuccess(instanceId -> log.info(
158               "Successfully published instance {} as owner of shard {}",
159               instanceId,
160               partition))
161           .doOnError(throwable -> log.error(
162               "Could not publish instance {} as owner of shard {}: {}",
163               instanceId,
164               partition,
165               throwable.toString()))
166           .onErrorComplete()
167           .block();
168     });
169
170     consumer.resume(partitions);
171   }
172
173   @Override
174   public void onPartitionsRevoked(Collection<TopicPartition> partitions)
175   {
176     partitions.forEach(topicPartition ->
177     {
178       int partition = topicPartition.partition();
179       isShardOwned[partition] = false;
180       nextOffset[partition] = consumer.position(topicPartition);
181
182       log.info("Partition revoked: {} - next={}", partition, nextOffset[partition]);
183
184       chatRoomData[partition]
185           .values()
186           .forEach(chatRoomData -> chatRoomData.deactivate());
187
188       channelMediator.shardRevoked(partition);
189     });
190   }
191
192   @Override
193   public void onPartitionsLost(Collection<TopicPartition> partitions)
194   {
195     log.warn("Lost partitions: {}, partitions");
196     // TODO: Muss auf den Verlust anders reagiert werden?
197     onPartitionsRevoked(partitions);
198   }
199
200   @Override
201   public void run()
202   {
203     running = true;
204
205     while (running)
206     {
207       try
208       {
209         ConsumerRecords<String, AbstractMessageTo> records = consumer.poll(pollingInterval);
210         log.info("Fetched {} messages", records.count());
211
212         switch (channelState)
213         {
214           case LOAD_IN_PROGRESS ->
215           {
216             loadChatRoomData(records);
217
218             if (isLoadingCompleted())
219             {
220               log.info("Loading of messages completed! Pausing all owned partitions...");
221               pauseAllOwnedPartions();
222               activateAllOwnedChatRooms();
223               log.info("Resuming normal operations...");
224               channelState = ChannelState.READY;
225             }
226           }
227           case SHUTTING_DOWN -> log.info("Shutdown in progress: ignoring {} fetched messages.", records.count());
228           default ->
229           {
230             if (!records.isEmpty())
231             {
232               throw new IllegalStateException("All owned partitions should be paused, when in state " + channelState);
233             }
234           }
235         }
236       }
237       catch (WakeupException e)
238       {
239         log.info("Received WakeupException, exiting!");
240         channelState = ChannelState.SHUTTING_DOWN;
241         running = false;
242       }
243     }
244
245     log.info("Exiting normally");
246   }
247
248   private void loadChatRoomData(ConsumerRecords<String, AbstractMessageTo> records)
249   {
250     for (ConsumerRecord<String, AbstractMessageTo> record : records)
251     {
252       UUID chatRoomId = UUID.fromString(record.key());
253
254       switch (record.value().getType())
255       {
256         case EVENT_CHATMESSAGE_RECEIVED:
257           Instant instant = Instant.ofEpochSecond(record.timestamp());
258           LocalDateTime timestamp = LocalDateTime.ofInstant(instant, zoneId);
259           loadChatMessage(
260               chatRoomId,
261               timestamp,
262               record.offset(),
263               (EventChatMessageReceivedTo) record.value(),
264               record.partition());
265           break;
266
267         default:
268           log.debug(
269               "Ignoring message for chat-room {} with offset {}: {}",
270               chatRoomId,
271               record.offset(),
272               record.value());
273       }
274
275       nextOffset[record.partition()] = record.offset() + 1;
276     }
277   }
278
279   private void loadChatMessage(
280       UUID chatRoomId,
281       LocalDateTime timestamp,
282       long offset,
283       EventChatMessageReceivedTo chatMessageTo,
284       int partition)
285   {
286     Message.MessageKey key = Message.MessageKey.of(chatMessageTo.getUser(), chatMessageTo.getId());
287     Message message = new Message(key, offset, timestamp, chatMessageTo.getText());
288
289     ChatRoomData chatRoomData = computeChatRoomData(chatRoomId, partition);
290     KafkaChatMessageService kafkaChatRoomService =
291         (KafkaChatMessageService) chatRoomData.getChatRoomService();
292
293     log.debug(
294         "Loaded message from partition={} at offset={}: {}",
295         partition,
296         offset,
297         message);
298     kafkaChatRoomService.persistMessage(message);
299   }
300
301   private boolean isLoadingCompleted()
302   {
303     return IntStream
304         .range(0, numShards)
305         .filter(shard -> isShardOwned[shard])
306         .allMatch(shard ->
307         {
308           TopicPartition partition = new TopicPartition(topic, shard);
309           long position = consumer.position(partition);
310           return position >= currentOffset[shard];
311         });
312   }
313
314   private void pauseAllOwnedPartions()
315   {
316     consumer.pause(IntStream
317         .range(0, numShards)
318         .filter(shard -> isShardOwned[shard])
319         .mapToObj(shard -> new TopicPartition(topic, shard))
320         .toList());
321   }
322
323   private void activateAllOwnedChatRooms()
324   {
325     IntStream
326         .range(0, numShards)
327         .filter(shard -> isShardOwned[shard])
328         .forEach(shard -> chatRoomData[shard]
329             .values()
330             .forEach(chatRoomData -> chatRoomData.activate()));
331   }
332
333
334   int[] getOwnedShards()
335   {
336     return IntStream
337         .range(0, numShards)
338         .filter(shard -> isShardOwned[shard])
339         .toArray();
340   }
341
342   void createChatRoomData(ChatRoomInfo chatRoomInfo)
343   {
344     int shard = chatRoomInfo.getShard();
345
346     ChatRoomData chatRoomData = computeChatRoomData(
347         chatRoomInfo.getId(),
348         chatRoomInfo.getShard());
349
350     // TODO: Possible race-condition in case of an ongoing rebalance!
351     if (isShardOwned[shard])
352     {
353       chatRoomData.activate();
354     }
355   }
356
357   Mono<ChatRoomData> getChatRoomData(int shard, UUID id)
358   {
359     ChannelState capturedState = channelState;
360     if (capturedState != ChannelState.READY)
361     {
362       return Mono.error(new ChannelNotReadyException(capturedState));
363     }
364
365     if (!isShardOwned[shard])
366     {
367       return Mono.error(new ShardNotOwnedException(instanceId, shard));
368     }
369
370     return Mono.justOrEmpty(chatRoomData[shard].get(id));
371   }
372
373   private ChatRoomData computeChatRoomData(UUID chatRoomId, int shard)
374   {
375     ChatRoomData chatRoomData = this.chatRoomData[shard].get(chatRoomId);
376
377     if (chatRoomData != null)
378     {
379       log.info(
380           "Ignoring request to create already existing ChatRoomData for {}",
381           chatRoomId);
382     }
383     else
384     {
385       log.info("Creating ChatRoomData {} with history-limit {}", chatRoomId, historyLimit);
386       KafkaChatMessageService service = new KafkaChatMessageService(this, chatRoomId);
387       chatRoomData = new ChatRoomData(clock, service, historyLimit);
388       this.chatRoomData[shard].put(chatRoomId, chatRoomData);
389     }
390
391     return chatRoomData;
392   }
393
394   ConsumerGroupMetadata getConsumerGroupMetadata()
395   {
396     return consumer.groupMetadata();
397   }
398 }