NONRTRIC - Implement DMaaP mediator producer service in Java
[nonrtric.git] / dmaap-adaptor-java / src / test / java / org / oran / dmaapadapter / IntegrationWithKafka.java
index 31ef970..c38af8a 100644 (file)
@@ -22,9 +22,11 @@ package org.oran.dmaapadapter;
 
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.awaitility.Awaitility.await;
+import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import com.google.gson.JsonParser;
 
+import java.time.Duration;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -47,6 +49,8 @@ import org.oran.dmaapadapter.repository.InfoType;
 import org.oran.dmaapadapter.repository.InfoTypes;
 import org.oran.dmaapadapter.repository.Job;
 import org.oran.dmaapadapter.repository.Jobs;
+import org.oran.dmaapadapter.tasks.KafkaJobDataConsumer;
+import org.oran.dmaapadapter.tasks.KafkaTopicConsumers;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
@@ -71,10 +75,12 @@ import reactor.kafka.sender.SenderRecord;
 @TestPropertySource(properties = { //
         "server.ssl.key-store=./config/keystore.jks", //
         "app.webclient.trust-store=./config/truststore.jks", //
-        "app.configuration-filepath=./src/test/resources/test_application_configuration_kafka.json"//
+        "app.configuration-filepath=./src/test/resources/test_application_configuration.json"//
 })
 class IntegrationWithKafka {
 
+    final String TYPE_ID = "KafkaInformationType";
+
     @Autowired
     private ApplicationConfig applicationConfig;
 
@@ -88,9 +94,12 @@ class IntegrationWithKafka {
     private ConsumerController consumerController;
 
     @Autowired
-    private EcsSimulatorController ecsSimulatorController;
+    private IcsSimulatorController icsSimulatorController;
+
+    @Autowired
+    private KafkaTopicConsumers kafkaTopicConsumers;
 
-    private com.google.gson.Gson gson = new com.google.gson.GsonBuilder().create();
+    private static com.google.gson.Gson gson = new com.google.gson.GsonBuilder().create();
 
     private static final Logger logger = LoggerFactory.getLogger(IntegrationWithKafka.class);
 
@@ -99,7 +108,7 @@ class IntegrationWithKafka {
 
     static class TestApplicationConfig extends ApplicationConfig {
         @Override
-        public String getEcsBaseUrl() {
+        public String getIcsBaseUrl() {
             return thisProcessUrl();
         }
 
@@ -142,7 +151,7 @@ class IntegrationWithKafka {
     @AfterEach
     void reset() {
         this.consumerController.testResults.reset();
-        this.ecsSimulatorController.testResults.reset();
+        this.icsSimulatorController.testResults.reset();
         this.jobs.clear();
     }
 
@@ -174,14 +183,15 @@ class IntegrationWithKafka {
         return "https://localhost:" + this.applicationConfig.getLocalServerHttpPort();
     }
 
-    private Object jobParametersAsJsonObject(String filter, int maxTimeMiliseconds, int maxSize) {
-        Job.Parameters param = new Job.Parameters(filter,
-                new Job.Parameters.BufferTimeout(maxSize, maxTimeMiliseconds));
+    private static Object jobParametersAsJsonObject(String filter, long maxTimeMiliseconds, int maxSize,
+            int maxConcurrency) {
+        Job.Parameters param =
+                new Job.Parameters(filter, new Job.BufferTimeout(maxSize, maxTimeMiliseconds), maxConcurrency);
         String str = gson.toJson(param);
         return jsonObject(str);
     }
 
-    private Object jsonObject(String json) {
+    private static Object jsonObject(String json) {
         try {
             return JsonParser.parseString(json).getAsJsonObject();
         } catch (Exception e) {
@@ -189,13 +199,12 @@ class IntegrationWithKafka {
         }
     }
 
-    private ConsumerJobInfo consumerJobInfo(String filter, int maxTimeMiliseconds, int maxSize) {
+    ConsumerJobInfo consumerJobInfo(String filter, Duration maxTime, int maxSize, int maxConcurrency) {
         try {
-            InfoType type = this.types.getAll().iterator().next();
-            String typeId = type.getId();
             String targetUri = baseUrl() + ConsumerController.CONSUMER_TARGET_URL;
-            return new ConsumerJobInfo(typeId, jobParametersAsJsonObject(filter, maxTimeMiliseconds, maxSize), "owner",
-                    targetUri, "");
+            return new ConsumerJobInfo(TYPE_ID,
+                    jobParametersAsJsonObject(filter, maxTime.toMillis(), maxSize, maxConcurrency), "owner", targetUri,
+                    "");
         } catch (Exception e) {
             return null;
         }
@@ -213,45 +222,97 @@ class IntegrationWithKafka {
         return SenderOptions.create(props);
     }
 
-    private SenderRecord<Integer, String, Integer> senderRecord(String data, int i) {
-        final InfoType infoType = this.types.getAll().iterator().next();
-        return SenderRecord.create(new ProducerRecord<>(infoType.getKafkaInputTopic(), i, data + i), i);
+    private SenderRecord<Integer, String, Integer> senderRecord(String data) {
+        final InfoType infoType = this.types.get(TYPE_ID);
+        int key = 1;
+        int correlationMetadata = 2;
+        return SenderRecord.create(new ProducerRecord<>(infoType.getKafkaInputTopic(), key, data), correlationMetadata);
+    }
+
+    private void sendDataToStream(Flux<SenderRecord<Integer, String, Integer>> dataToSend) {
+        final KafkaSender<Integer, String> sender = KafkaSender.create(senderOptions());
+
+        sender.send(dataToSend) //
+                .doOnError(e -> logger.error("Send failed", e)) //
+                .blockLast();
+
+    }
+
+    private void verifiedReceivedByConsumer(String... strings) {
+        ConsumerController.TestResults consumer = this.consumerController.testResults;
+        await().untilAsserted(() -> assertThat(consumer.receivedBodies.size()).isEqualTo(strings.length));
+        for (String s : strings) {
+            assertTrue(consumer.hasReceived(s));
+        }
     }
 
     @Test
-    void kafkaIntegrationTest() throws InterruptedException {
+    void kafkaIntegrationTest() throws Exception {
         final String JOB_ID1 = "ID1";
         final String JOB_ID2 = "ID2";
 
         // Register producer, Register types
-        await().untilAsserted(() -> assertThat(ecsSimulatorController.testResults.registrationInfo).isNotNull());
-        assertThat(ecsSimulatorController.testResults.registrationInfo.supportedTypeIds).hasSize(1);
+        await().untilAsserted(() -> assertThat(icsSimulatorController.testResults.registrationInfo).isNotNull());
+        assertThat(icsSimulatorController.testResults.registrationInfo.supportedTypeIds).hasSize(this.types.size());
+
+        // Create two jobs. One buffering and one with a filter
+        this.icsSimulatorController.addJob(consumerJobInfo(null, Duration.ofMillis(400), 1000, 20), JOB_ID1,
+                restClient());
+        this.icsSimulatorController.addJob(consumerJobInfo("^Message_1$", Duration.ZERO, 0, 1), JOB_ID2, restClient());
 
-        // Create a job
-        this.ecsSimulatorController.addJob(consumerJobInfo(".*", 10, 1000), JOB_ID1, restClient());
-        this.ecsSimulatorController.addJob(consumerJobInfo(".*Message_1.*", 0, 0), JOB_ID2, restClient());
         await().untilAsserted(() -> assertThat(this.jobs.size()).isEqualTo(2));
 
-        final KafkaSender<Integer, String> sender = KafkaSender.create(senderOptions());
+        var dataToSend = Flux.range(1, 3).map(i -> senderRecord("Message_" + i)); // Message_1, Message_2 etc.
+        sendDataToStream(dataToSend);
 
-        var dataToSend = Flux.range(1, 3).map(i -> senderRecord("Message_", i)); // Message_1, Message_2 etc.
+        verifiedReceivedByConsumer("Message_1", "[\"Message_1\", \"Message_2\", \"Message_3\"]");
 
-        sender.send(dataToSend) //
-                .doOnError(e -> logger.error("Send failed", e)) //
-                .doOnNext(senderResult -> logger.debug("Sent {}", senderResult)) //
-                .doOnError(t -> logger.error("Error {}", t)) //
-                .blockLast();
+        // Delete the jobs
+        this.icsSimulatorController.deleteJob(JOB_ID1, restClient());
+        this.icsSimulatorController.deleteJob(JOB_ID2, restClient());
 
-        ConsumerController.TestResults consumer = this.consumerController.testResults;
-        await().untilAsserted(() -> assertThat(consumer.receivedBodies.size()).isEqualTo(2));
-        assertThat(consumer.receivedBodies.get(0)).isEqualTo("Message_1");
-        assertThat(consumer.receivedBodies.get(1)).isEqualTo("[Message_1, Message_2, Message_3]");
+        await().untilAsserted(() -> assertThat(this.jobs.size()).isZero());
+        await().untilAsserted(() -> assertThat(this.kafkaTopicConsumers.getConsumers().keySet()).isEmpty());
+    }
+
+    @Test
+    void kafkaIOverflow() throws InterruptedException {
+        final String JOB_ID1 = "ID1";
+        final String JOB_ID2 = "ID2";
+
+        // Register producer, Register types
+        await().untilAsserted(() -> assertThat(icsSimulatorController.testResults.registrationInfo).isNotNull());
+        assertThat(icsSimulatorController.testResults.registrationInfo.supportedTypeIds).hasSize(this.types.size());
+
+        // Create two jobs.
+        this.icsSimulatorController.addJob(consumerJobInfo(null, Duration.ofMillis(400), 1000, 1), JOB_ID1,
+                restClient());
+        this.icsSimulatorController.addJob(consumerJobInfo(null, Duration.ZERO, 0, 1), JOB_ID2, restClient());
+
+        await().untilAsserted(() -> assertThat(this.jobs.size()).isEqualTo(2));
+
+        var dataToSend = Flux.range(1, 1000000).map(i -> senderRecord("Message_" + i)); // Message_1, Message_2 etc.
+        sendDataToStream(dataToSend); // this should overflow
+
+        KafkaJobDataConsumer consumer = kafkaTopicConsumers.getConsumers().get(TYPE_ID).iterator().next();
+        await().untilAsserted(() -> assertThat(consumer.isRunning()).isFalse());
+        this.consumerController.testResults.reset();
+
+        this.icsSimulatorController.deleteJob(JOB_ID2, restClient()); // Delete one job
+        kafkaTopicConsumers.restartNonRunningTopics();
+        Thread.sleep(1000); // Restarting the input seems to take some asynch time
+
+        dataToSend = Flux.just(senderRecord("Howdy\""));
+        sendDataToStream(dataToSend);
+
+        verifiedReceivedByConsumer("[\"Howdy\\\"\"]");
 
-        // Delete the job
-        this.ecsSimulatorController.deleteJob(JOB_ID1, restClient());
-        this.ecsSimulatorController.deleteJob(JOB_ID2, restClient());
+        // Delete the jobs
+        this.icsSimulatorController.deleteJob(JOB_ID1, restClient());
+        this.icsSimulatorController.deleteJob(JOB_ID2, restClient());
 
         await().untilAsserted(() -> assertThat(this.jobs.size()).isZero());
+        await().untilAsserted(() -> assertThat(this.kafkaTopicConsumers.getConsumers().keySet()).isEmpty());
     }
 
 }