test: Fixed `AbstractConfigurationIT#testPutMessageInNewChatRoom()`
[demos/kafka/chat] / src / test / java / de / juplo / kafka / chat / backend / KafkaConfigurationIT.java
index 961275e..1fa767f 100644 (file)
@@ -1,92 +1,90 @@
 package de.juplo.kafka.chat.backend;
 
-import de.juplo.kafka.chat.backend.persistence.kafka.ChatRoomChannel;
-import de.juplo.kafka.chat.backend.persistence.kafka.KafkaServicesApplicationRunner;
+import de.juplo.kafka.chat.backend.implementation.kafka.ChannelTaskExecutor;
+import de.juplo.kafka.chat.backend.implementation.kafka.DataChannel;
+import de.juplo.kafka.chat.backend.implementation.kafka.KafkaTestUtils;
+import de.juplo.kafka.chat.backend.implementation.kafka.WorkAssignor;
+import de.juplo.kafka.chat.backend.implementation.kafka.messages.AbstractMessageTo;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.kafka.clients.consumer.Consumer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.TopicPartition;
-import org.junit.jupiter.api.AfterAll;
 import org.junit.jupiter.api.BeforeAll;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.boot.test.context.SpringBootTest;
-import org.springframework.boot.test.mock.mockito.MockBean;
+import org.springframework.boot.test.context.TestConfiguration;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Import;
 import org.springframework.kafka.core.KafkaTemplate;
-import org.springframework.kafka.support.SendResult;
 import org.springframework.kafka.test.context.EmbeddedKafka;
 import org.springframework.scheduling.concurrent.ThreadPoolTaskExecutor;
 
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-
-import static de.juplo.kafka.chat.backend.KafkaConfigurationIT.TOPIC;
+import static de.juplo.kafka.chat.backend.KafkaConfigurationIT.DATA_TOPIC;
+import static de.juplo.kafka.chat.backend.KafkaConfigurationIT.INFO_TOPIC;
 
 
 @SpringBootTest(
     webEnvironment = SpringBootTest.WebEnvironment.RANDOM_PORT,
     properties = {
+        "spring.main.allow-bean-definition-overriding=true",
         "chat.backend.services=kafka",
         "chat.backend.kafka.client-id-PREFIX=TEST",
         "chat.backend.kafka.bootstrap-servers=${spring.embedded.kafka.brokers}",
         "spring.kafka.bootstrap-servers=${spring.embedded.kafka.brokers}",
-        "chat.backend.kafka.chatroom-channel-topic=" + TOPIC,
+        "chat.backend.kafka.info-channel-topic=" + INFO_TOPIC,
+        "chat.backend.kafka.data-channel-topic=" + DATA_TOPIC,
         "chat.backend.kafka.num-partitions=10",
         })
-@EmbeddedKafka(topics = { TOPIC }, partitions = 10)
+@EmbeddedKafka(
+    topics = { INFO_TOPIC, DATA_TOPIC },
+    partitions = 10)
 @Slf4j
 class KafkaConfigurationIT extends AbstractConfigurationWithShardingIT
 {
-  final static String TOPIC = "TEST_CHATROOM_CHANNEL";
-
-  static CompletableFuture<Void> CONSUMER_JOB;
-
-  @MockBean
-  KafkaServicesApplicationRunner kafkaServicesApplicationRunner;
+  final static String INFO_TOPIC = "KAFKA_CONFIGURATION_IT_INFO_CHANNEL";
+  final static String DATA_TOPIC = "KAFKA_CONFIGURATION_IT_DATA_CHANNEL";
 
   @BeforeAll
-  public static void sendAndLoadStoredData(
+  static void sendAndLoadStoredData(
       @Autowired KafkaTemplate<String, String> messageTemplate,
-      @Autowired Consumer chatRoomChannelConsumer,
-      @Autowired ThreadPoolTaskExecutor taskExecutor,
-      @Autowired ChatRoomChannel chatRoomChannel)
+      @Autowired ChannelTaskExecutor dataChannelTaskExecutor)
   {
-    send(messageTemplate, "5c73531c-6fc4-426c-adcb-afc5c140a0f7","{ \"id\": \"5c73531c-6fc4-426c-adcb-afc5c140a0f7\", \"shard\": 2, \"name\": \"FOO\" }", "command_create_chatroom");
-    send(messageTemplate,"5c73531c-6fc4-426c-adcb-afc5c140a0f7","{ \"id\" : 1, \"user\" : \"peter\", \"text\" : \"Hallo, ich heiße Peter!\" }", "event_chatmessage_received");
-    send(messageTemplate,"5c73531c-6fc4-426c-adcb-afc5c140a0f7","{ \"id\" : 1, \"user\" : \"ute\", \"text\" : \"Ich bin Ute...\" }", "event_chatmessage_received");
-    send(messageTemplate,"5c73531c-6fc4-426c-adcb-afc5c140a0f7","{ \"id\" : 2, \"user\" : \"peter\", \"text\" : \"Willst du mit mir gehen?\" }", "event_chatmessage_received");
-    send(messageTemplate,"5c73531c-6fc4-426c-adcb-afc5c140a0f7","{ \"id\" : 1, \"user\" : \"klaus\", \"text\" : \"Ja? Nein? Vielleicht??\" }", "event_chatmessage_received");
+    KafkaTestUtils.sendAndLoadStoredData(
+        messageTemplate,
+        INFO_TOPIC,
+        DATA_TOPIC);
 
-    List<TopicPartition> assignedPartitions = List.of(new TopicPartition(TOPIC, 2));
-    chatRoomChannelConsumer.assign(assignedPartitions);
-    chatRoomChannel.onPartitionsAssigned(assignedPartitions);
-    CONSUMER_JOB = taskExecutor
-        .submitCompletable(chatRoomChannel)
-        .exceptionally(e ->
-        {
-          log.error("The consumer for the ChatRoomChannel exited abnormally!", e);
-          return null;
-        });
+    // The initialization of the data-channel must happen,
+    // after the messages were sent into the topic of the
+    // test-cluster.
+    // Otherwise, the initial loading of the data might be
+    // completed, before these messages arrive, so that
+    // they are ignored and the state is never restored.
+    dataChannelTaskExecutor.executeChannelTask();
   }
 
-  static void send(KafkaTemplate<String, String> kafkaTemplate, String key, String value, String typeId)
-  {
-    ProducerRecord<String, String> record = new ProducerRecord<>(TOPIC, key, value);
-    record.headers().add("__TypeId__", typeId.getBytes());
-    SendResult<String, String> result = kafkaTemplate.send(record).join();
-    log.info(
-        "Sent {}={} to {}",
-        key,
-        value,
-        new TopicPartition(result.getRecordMetadata().topic(), result.getRecordMetadata().partition()));
-  }
 
-  @AfterAll
-  static void joinConsumerJob(@Autowired Consumer chatRoomChannelConsumer)
+  @TestConfiguration
+  @Import(KafkaTestUtils.KafkaTestConfiguration.class)
+  static class KafkaConfigurationITConfiguration
   {
-    log.info("Signaling the consumer of the CahtRoomChannel to quit its work");
-    chatRoomChannelConsumer.wakeup();
-    log.info("Waiting for the consumer of the ChatRoomChannel to finish its work");
-    CONSUMER_JOB.join();
-    log.info("Joined the consumer of the ChatRoomChannel");
+    /**
+     * The definition of this bean has to be overruled, so
+     * that the configuration of the `initMethod`, which
+     * has to be called explicitly, _after_ the messages
+     * were sent to and received by the test-culster, can
+     * be dropped.
+     */
+    @Bean(destroyMethod = "join")
+    ChannelTaskExecutor dataChannelTaskExecutor(
+        ThreadPoolTaskExecutor taskExecutor,
+        DataChannel dataChannel,
+        Consumer<String, AbstractMessageTo> dataChannelConsumer,
+        WorkAssignor dataChannelWorkAssignor)
+    {
+      return new ChannelTaskExecutor(
+          taskExecutor,
+          dataChannel,
+          dataChannelConsumer,
+          dataChannelWorkAssignor);
+    }
   }
 }