X-Git-Url: https://juplo.de/gitweb/?a=blobdiff_plain;f=src%2Fmain%2Fjava%2Fde%2Fjuplo%2Fkafka%2Fchat%2Fbackend%2Fpersistence%2Fkafka%2FKafkaChatHomeService.java;h=e23f08db4677975bac0f5fcbf286a3278ec5d099;hb=9aaca321143c4fd08859dc14cdd7e602cc1714f6;hp=4460432254379ca6ca0a420db8e4d795f03aa4d7;hpb=394aa7848abcfc7b4510cd2688c00fce01b3c225;p=demos%2Fkafka%2Fchat diff --git a/src/main/java/de/juplo/kafka/chat/backend/persistence/kafka/KafkaChatHomeService.java b/src/main/java/de/juplo/kafka/chat/backend/persistence/kafka/KafkaChatHomeService.java index 44604322..e23f08db 100644 --- a/src/main/java/de/juplo/kafka/chat/backend/persistence/kafka/KafkaChatHomeService.java +++ b/src/main/java/de/juplo/kafka/chat/backend/persistence/kafka/KafkaChatHomeService.java @@ -2,103 +2,97 @@ package de.juplo.kafka.chat.backend.persistence.kafka; import de.juplo.kafka.chat.backend.domain.ChatHomeService; import de.juplo.kafka.chat.backend.domain.ChatRoom; +import de.juplo.kafka.chat.backend.domain.Message; import de.juplo.kafka.chat.backend.domain.ShardNotOwnedException; +import de.juplo.kafka.chat.backend.persistence.KafkaLikeShardingStrategy; 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.Duration; -import java.time.ZoneId; +import java.time.*; import java.util.*; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReadWriteLock; -import java.util.concurrent.locks.ReentrantLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.IntStream; @Slf4j public class KafkaChatHomeService implements ChatHomeService, Runnable, ConsumerRebalanceListener { - private final ExecutorService executorService; - private final Consumer consumer; - private final Producer producer; - private final String topic; + private final String chatRoomsTopic; + private final Consumer chatRoomsConsumer; + private final Producer chatRoomsProducer; + private final String chatMessagesTopic; + private final Consumer chatMessagesConsumer; + private final Producer chatMessagesProducer; private final ZoneId zoneId; - // private final long[] offsets; Erst mal immer alles neu einlesen + private final int numShards; private final boolean[] isShardOwned; - private final Map[] chatRoomMaps; - private final ReadWriteLock lock = new ReentrantReadWriteLock(); + private final long[] currentOffset; + private final long[] nextOffset; + private final Map[] chatrooms; + private final KafkaLikeShardingStrategy shardingStrategy; private boolean running; + private volatile boolean loadInProgress; public KafkaChatHomeService( - ExecutorService executorService, - Consumer consumer, - Producer producer, - String topic, + String chatRoomsTopic, + Consumer chatRoomsConsumer, + Producer chatRoomsProducer, + String chatMessagesTopic, + Consumer chatMessagesConsumer, + Producer chatMessagesProducer, ZoneId zoneId, int numShards) { log.debug("Creating KafkaChatHomeService"); - this.executorService = executorService; - this.consumer = consumer; - this.producer = producer; - this.topic = topic; + this.chatRoomsTopic = chatRoomsTopic; + this.chatRoomsConsumer = chatRoomsConsumer; + this.chatRoomsProducer = chatRoomsProducer; + this.chatMessagesTopic = chatMessagesTopic; + this.chatMessagesConsumer = chatMessagesConsumer; + this.chatMessagesProducer = chatMessagesProducer; this.zoneId = zoneId; - // this.offsets = new long[numShards]; - // for (int i=0; i< numShards; i++) - // { - // this.offsets[i] = 0l; - // } + this.numShards = numShards; this.isShardOwned = new boolean[numShards]; - this.chatRoomMaps = new Map[numShards]; + this.currentOffset = new long[numShards]; + this.nextOffset = new long[numShards]; + this.chatrooms = new Map[numShards]; + this.shardingStrategy = new KafkaLikeShardingStrategy(numShards); } @Override public void onPartitionsAssigned(Collection partitions) { - try - { - lock.writeLock().lock(); - - consumer.endOffsets(partitions).forEach((topicPartition, currentOffset) -> - { - if (!topicPartition.topic().equals(topic)) - { - log.warn("Ignoring partition from unwanted topic: {}", topicPartition); - return; - } + log.info("Newly assigned partitions! Pausing normal operations..."); + loadInProgress = true; - int partition = topicPartition.partition(); - long unseenOffset = 0; // offsets[partition]; + chatMessagesConsumer.endOffsets(partitions).forEach((topicPartition, currentOffset) -> + { + int partition = topicPartition.partition(); + isShardOwned[partition] = true; + this.currentOffset[partition] = currentOffset; - log.info( - "Loading messages from partition {}: start-offset={} -> current-offset={}", - partition, - unseenOffset, - currentOffset); + log.info( + "Partition assigned: {} - loading messages: next={} -> current={}", + partition, + nextOffset[partition], + currentOffset); - // TODO: reuse! Nicht immer alles neu laden, sondern erst ab offsets[partition]! - consumer.seek(topicPartition, unseenOffset); - }); + chatMessagesConsumer.seek(topicPartition, nextOffset[partition]); + }); - consumer.resume(partitions); - } - finally - { - lock.writeLock().unlock(); - } + chatMessagesConsumer.resume(partitions); } @Override @@ -106,21 +100,16 @@ public class KafkaChatHomeService implements ChatHomeService, Runnable, Consumer { partitions.forEach(topicPartition -> { - if (!topicPartition.topic().equals(topic)) - { - log.warn("Ignoring partition from unwanted topic: {}", topicPartition); - return; - } - int partition = topicPartition.partition(); - // long unseenOffset = offsets[partition]; TODO: Offset merken...? + isShardOwned[partition] = false; + log.info("Partition revoked: {} - next={}", partition, nextOffset[partition]); }); - log.info("Revoked partitions: {}", partitions); } @Override public void onPartitionsLost(Collection partitions) { + log.warn("Lost partitions: {}, partitions"); // TODO: Muss auf den Verlust anders reagiert werden? onPartitionsRevoked(partitions); } @@ -128,62 +117,166 @@ public class KafkaChatHomeService implements ChatHomeService, Runnable, Consumer @Override public void run() { - consumer.subscribe(List.of(topic)); + chatMessagesConsumer.subscribe(List.of(chatMessagesTopic)); running = true; - try + while (running) { - while (running) + try { - ConsumerRecords records = consumer.poll(Duration.ofMinutes(5)); + ConsumerRecords records = chatMessagesConsumer.poll(Duration.ofMinutes(5)); log.info("Fetched {} messages", records.count()); - for (ConsumerRecord record : records) + if (loadInProgress) { + loadMessages(records); + if (isLoadingCompleted()) + { + log.info("Loading of messages completed! Pausing all owned partitions..."); + pauseAllOwnedPartions(); + log.info("Resuming normal operations..."); + loadInProgress = false; + } + } + else + { + if (!records.isEmpty()) + { + throw new IllegalStateException("All owned partitions should be paused, when no load is in progress!"); + } } } + catch (WakeupException e) + { + } + catch (RecordDeserializationException e) + { + } } } + void loadMessages(ConsumerRecords records) + { + for (ConsumerRecord record : records) + { + nextOffset[record.partition()] = record.offset() + 1; + UUID chatRoomId = UUID.fromString(record.key()); + MessageTo messageTo = record.value(); + + Message.MessageKey key = Message.MessageKey.of(messageTo.getUser(), messageTo.getId()); + + Instant instant = Instant.ofEpochSecond(record.timestamp()); + LocalDateTime timestamp = LocalDateTime.ofInstant(instant, zoneId); + + Message message = new Message(key, record.offset(), timestamp, messageTo.getText()); + + ChatRoom chatRoom = chatrooms[record.partition()].get(chatRoomId); + KafkaChatRoomService kafkaChatRoomService = + (KafkaChatRoomService) chatRoom.getChatRoomService(); + + kafkaChatRoomService.persistMessage(message); + } + } + + boolean isLoadingCompleted() + { + return IntStream + .range(0, numShards) + .filter(shard -> isShardOwned[shard]) + .mapToObj(shard -> nextOffset[shard] >= currentOffset[shard]) + .collect( + () -> Boolean.TRUE, + (acc, v) -> Boolean.valueOf(acc && v), + (a, b) -> Boolean.valueOf(a && b)); + } + + void pauseAllOwnedPartions() + { + chatMessagesConsumer.pause(IntStream + .range(0, numShards) + .filter(shard -> isShardOwned[shard]) + .mapToObj(shard -> new TopicPartition(chatMessagesTopic, shard)) + .toList()); + } + + Mono sendMessage( + UUID chatRoomId, + Message.MessageKey key, + LocalDateTime timestamp, + String text) + { + int shard = this.shardingStrategy.selectShard(chatRoomId); + TopicPartition tp = new TopicPartition(chatMessagesTopic, shard); + ZonedDateTime zdt = ZonedDateTime.of(timestamp, zoneId); + return Mono.create(sink -> + { + ProducerRecord record = + new ProducerRecord<>( + tp.topic(), + tp.partition(), + zdt.toEpochSecond(), + chatRoomId.toString(), + MessageTo.of(key.getUsername(), key.getMessageId(), text)); + + chatMessagesProducer.send(record, ((metadata, exception) -> + { + if (metadata != null) + { + // On successful send + Message message = new Message(key, metadata.offset(), timestamp, text); + log.info("Successfully send message {}", message); + sink.success(message); + } + else + { + // On send-failure + log.error( + "Could not send message for chat-room={}, key={}, timestamp={}, text={}: {}", + chatRoomId, + key, + timestamp, + text, + exception); + sink.error(exception); + } + })); + }); + } + + + public void putChatRoom(ChatRoom chatRoom) + { + + ProducerRecord record = new ProducerRecord<>(chatRoom.getShard(), ); + // TODO: Nachricht senden! + chatrooms[chatRoom.getShard()].put(chatRoom.getId(), chatRoom); + } + @Override public Mono getChatRoom(int shard, UUID id) { - if (lock.readLock().tryLock()) + if (loadInProgress) { - try - { - return Mono.justOrEmpty(chatRoomMaps[shard].get(id)); - } - finally - { - lock.readLock().unlock(); - } + throw new ShardNotOwnedException(shard); } else { - throw new ShardNotOwnedException(shard); + return Mono.justOrEmpty(chatrooms[shard].get(id)); } } @Override public Flux getChatRooms(int shard) { - if (lock.readLock().tryLock()) + if (loadInProgress) { - try - { - return Flux.fromStream(chatRoomMaps[shard].values().stream()); - } - finally - { - lock.readLock().unlock(); - } + throw new ShardNotOwnedException(shard); } else { - throw new ShardNotOwnedException(shard); + return Flux.fromStream(chatrooms[shard].values().stream()); } } }