From ffe0e1ee121e2ca89f82824f193ed0a0e599a2c0 Mon Sep 17 00:00:00 2001
From: "raviteja.karumuri" <raviteja.karumuri@est.tech>
Date: Thu, 29 Jun 2023 13:57:40 +0100
Subject: Updating the Kafka listener compliance to could events and legacy

# Added filter property to the legay(DMIAsync) Consumer
# Added containerFactory property to the CloudEvent(DataOperation) consumer to use 'cloudEventDeserializtion'

Issue-ID: CPS-1746
Signed-off-by: raviteja.karumuri <raviteja.karumuri@est.tech>
Change-Id: I73315b816eb6446b7e163fd8779bc35fd6cbef63
---
 .../AsyncRestRequestResponseEventConsumer.java     | 62 +++++++++++++++++
 .../api/impl/async/DataOperationEventConsumer.java | 63 ++++++++++++++++++
 .../async/DataOperationRecordFilterStrategy.java   | 55 ----------------
 .../async/NcmpAsyncDataOperationEventConsumer.java | 63 ------------------
 .../NcmpAsyncRequestResponseEventConsumer.java     | 60 -----------------
 .../api/impl/async/RecordFilterStrategies.java     | 77 ++++++++++++++++++++++
 6 files changed, 202 insertions(+), 178 deletions(-)
 create mode 100644 cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/AsyncRestRequestResponseEventConsumer.java
 create mode 100644 cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/DataOperationEventConsumer.java
 delete mode 100644 cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/DataOperationRecordFilterStrategy.java
 delete mode 100644 cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/NcmpAsyncDataOperationEventConsumer.java
 delete mode 100644 cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/NcmpAsyncRequestResponseEventConsumer.java
 create mode 100644 cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/RecordFilterStrategies.java

(limited to 'cps-ncmp-service/src/main/java/org')

diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/AsyncRestRequestResponseEventConsumer.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/AsyncRestRequestResponseEventConsumer.java
new file mode 100644
index 0000000000..0044182ddd
--- /dev/null
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/AsyncRestRequestResponseEventConsumer.java
@@ -0,0 +1,62 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (c) 2023 Nordix Foundation.
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *         http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ * SPDX-License-Identifier: Apache-2.0
+ * ============LICENSE_END=========================================================
+ */
+
+package org.onap.cps.ncmp.api.impl.async;
+
+import lombok.RequiredArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+import org.onap.cps.ncmp.api.impl.events.EventsPublisher;
+import org.onap.cps.ncmp.event.model.DmiAsyncRequestResponseEvent;
+import org.onap.cps.ncmp.event.model.NcmpAsyncRequestResponseEvent;
+import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty;
+import org.springframework.kafka.annotation.KafkaListener;
+import org.springframework.stereotype.Component;
+
+/**
+ * Listener for cps-ncmp async request response events.
+ */
+@Component
+@Slf4j
+@RequiredArgsConstructor
+@ConditionalOnProperty(name = "notification.enabled", havingValue = "true", matchIfMissing = true)
+public class AsyncRestRequestResponseEventConsumer {
+
+    private final EventsPublisher<NcmpAsyncRequestResponseEvent> eventsPublisher;
+    private final NcmpAsyncRequestResponseEventMapper ncmpAsyncRequestResponseEventMapper;
+
+    /**
+     * Consume the specified event.
+     *
+     * @param dmiAsyncRequestResponseEvent the event to be consumed and produced.
+     */
+    @KafkaListener(
+            topics = "${app.ncmp.async-m2m.topic}",
+            filter = "includeNonCloudEventsOnly",
+            groupId = "ncmp-async-rest-request-event-group",
+            properties = {"spring.json.value.default.type=org.onap.cps.ncmp.event.model.DmiAsyncRequestResponseEvent"})
+    public void consumeAndForward(final DmiAsyncRequestResponseEvent dmiAsyncRequestResponseEvent) {
+        log.debug("Consuming event {} ...", dmiAsyncRequestResponseEvent);
+        final NcmpAsyncRequestResponseEvent ncmpAsyncRequestResponseEvent =
+                ncmpAsyncRequestResponseEventMapper.toNcmpAsyncEvent(dmiAsyncRequestResponseEvent);
+        eventsPublisher.publishEvent(ncmpAsyncRequestResponseEvent.getEventTarget(),
+                                     ncmpAsyncRequestResponseEvent.getEventId(),
+                                     ncmpAsyncRequestResponseEvent);
+    }
+}
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/DataOperationEventConsumer.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/DataOperationEventConsumer.java
new file mode 100644
index 0000000000..9649b0109d
--- /dev/null
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/DataOperationEventConsumer.java
@@ -0,0 +1,63 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2023 Nordix Foundation
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ * SPDX-License-Identifier: Apache-2.0
+ * ============LICENSE_END=========================================================
+ */
+
+package org.onap.cps.ncmp.api.impl.async;
+
+import io.cloudevents.CloudEvent;
+import io.cloudevents.kafka.impl.KafkaHeaders;
+import lombok.RequiredArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.onap.cps.ncmp.api.impl.events.EventsPublisher;
+import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty;
+import org.springframework.kafka.annotation.KafkaListener;
+import org.springframework.stereotype.Component;
+
+/**
+ * Listener for cps-ncmp async data operation events.
+ */
+@Component
+@Slf4j
+@RequiredArgsConstructor
+@ConditionalOnProperty(name = "notification.enabled", havingValue = "true", matchIfMissing = true)
+public class DataOperationEventConsumer {
+
+    private final EventsPublisher<CloudEvent> eventsPublisher;
+
+    /**
+     * Consume the DataOperation cloud event published by producer to topic 'async-m2m.topic'
+     * and publish the same to client specified topic.
+     *
+     * @param dataOperationEventConsumerRecord consuming event as a ConsumerRecord.
+     */
+    @KafkaListener(
+            topics = "${app.ncmp.async-m2m.topic}",
+            filter = "includeDataOperationEventsOnly",
+            groupId = "ncmp-data-operation-event-group",
+            containerFactory = "cloudEventConcurrentKafkaListenerContainerFactory")
+    public void consumeAndPublish(final ConsumerRecord<String, CloudEvent> dataOperationEventConsumerRecord) {
+        log.info("Consuming event payload {} ...", dataOperationEventConsumerRecord.value());
+        final String eventTarget = KafkaHeaders.getParsedKafkaHeader(
+                dataOperationEventConsumerRecord.headers(), "ce_destination");
+        final String eventId = KafkaHeaders.getParsedKafkaHeader(
+                dataOperationEventConsumerRecord.headers(), "ce_id");
+        eventsPublisher.publishCloudEvent(eventTarget, eventId, dataOperationEventConsumerRecord.value());
+    }
+}
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/DataOperationRecordFilterStrategy.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/DataOperationRecordFilterStrategy.java
deleted file mode 100644
index 76cc0c4b7b..0000000000
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/DataOperationRecordFilterStrategy.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * ============LICENSE_START=======================================================
- * Copyright (C) 2023 Nordix Foundation
- * ================================================================================
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *       http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- * SPDX-License-Identifier: Apache-2.0
- * ============LICENSE_END=========================================================
- */
-
-package org.onap.cps.ncmp.api.impl.async;
-
-import io.cloudevents.CloudEvent;
-import io.cloudevents.kafka.impl.KafkaHeaders;
-import lombok.extern.slf4j.Slf4j;
-import org.springframework.context.annotation.Bean;
-import org.springframework.context.annotation.Configuration;
-import org.springframework.kafka.listener.adapter.RecordFilterStrategy;
-
-/**
- * Data operation record filter strategy, which helps to filter the consumer records.
- *
- */
-@Configuration
-@Slf4j
-public class DataOperationRecordFilterStrategy {
-
-    /**
-     *  Filtering the consumer records based on the eventType header, It
-     *  returns boolean, true means filter the consumer record and false
-     *  means not filter the consumer record.
-     * @return boolean value.
-     */
-    @Bean
-    public RecordFilterStrategy<String, CloudEvent> includeDataOperationEventsOnly() {
-        return consumedRecord -> {
-            final String eventTypeHeaderValue = KafkaHeaders.getParsedKafkaHeader(consumedRecord.headers(), "ce_type");
-            if (eventTypeHeaderValue == null) {
-                log.trace("No ce_type header found, possibly a legacy event (ignored)");
-                return true;
-            }
-            return !(eventTypeHeaderValue.contains("DataOperationEvent"));
-        };
-    }
-}
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/NcmpAsyncDataOperationEventConsumer.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/NcmpAsyncDataOperationEventConsumer.java
deleted file mode 100644
index 4a0ec5c493..0000000000
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/NcmpAsyncDataOperationEventConsumer.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * ============LICENSE_START=======================================================
- * Copyright (C) 2023 Nordix Foundation
- * ================================================================================
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *       http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- * SPDX-License-Identifier: Apache-2.0
- * ============LICENSE_END=========================================================
- */
-
-package org.onap.cps.ncmp.api.impl.async;
-
-import io.cloudevents.CloudEvent;
-import io.cloudevents.kafka.impl.KafkaHeaders;
-import lombok.RequiredArgsConstructor;
-import lombok.extern.slf4j.Slf4j;
-import org.apache.kafka.clients.consumer.ConsumerRecord;
-import org.onap.cps.ncmp.api.impl.events.EventsPublisher;
-import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty;
-import org.springframework.kafka.annotation.KafkaListener;
-import org.springframework.stereotype.Component;
-
-/**
- * Listener for cps-ncmp async data operation events.
- */
-@Component
-@Slf4j
-@RequiredArgsConstructor
-@ConditionalOnProperty(name = "notification.enabled", havingValue = "true", matchIfMissing = true)
-public class NcmpAsyncDataOperationEventConsumer {
-
-    private final EventsPublisher<CloudEvent> eventsPublisher;
-
-    /**
-     * Consume the DataOperationResponseEvent published by producer to topic 'async-m2m.topic'
-     * and publish the same to the client specified topic.
-     *
-     * @param dataOperationEventConsumerRecord consuming event as a ConsumerRecord.
-     */
-    @KafkaListener(
-            topics = "${app.ncmp.async-m2m.topic}",
-            filter = "includeDataOperationEventsOnly",
-            groupId = "ncmp-data-operation-event-group",
-            properties = {"spring.json.value.default.type=org.onap.cps.ncmp.events.async1_0_0.DataOperationEvent"})
-    public void consumeAndPublish(final ConsumerRecord<String, CloudEvent> dataOperationEventConsumerRecord) {
-        log.info("Consuming event payload {} ...", dataOperationEventConsumerRecord.value());
-        final String eventTarget = KafkaHeaders.getParsedKafkaHeader(
-                dataOperationEventConsumerRecord.headers(), "ce_destination");
-        final String eventId = KafkaHeaders.getParsedKafkaHeader(
-                dataOperationEventConsumerRecord.headers(), "ce_id");
-        eventsPublisher.publishCloudEvent(eventTarget, eventId, dataOperationEventConsumerRecord.value());
-    }
-}
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/NcmpAsyncRequestResponseEventConsumer.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/NcmpAsyncRequestResponseEventConsumer.java
deleted file mode 100644
index 0ac0fb92de..0000000000
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/NcmpAsyncRequestResponseEventConsumer.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * ============LICENSE_START=======================================================
- * Copyright (c) 2023 Nordix Foundation.
- * ================================================================================
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *         http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- * SPDX-License-Identifier: Apache-2.0
- * ============LICENSE_END=========================================================
- */
-
-package org.onap.cps.ncmp.api.impl.async;
-
-import lombok.RequiredArgsConstructor;
-import lombok.extern.slf4j.Slf4j;
-import org.onap.cps.ncmp.api.impl.events.EventsPublisher;
-import org.onap.cps.ncmp.event.model.DmiAsyncRequestResponseEvent;
-import org.onap.cps.ncmp.event.model.NcmpAsyncRequestResponseEvent;
-import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty;
-import org.springframework.kafka.annotation.KafkaListener;
-import org.springframework.stereotype.Component;
-
-/**
- * Listener for cps-ncmp async request response events.
- */
-@Component
-@Slf4j
-@RequiredArgsConstructor
-@ConditionalOnProperty(name = "notification.enabled", havingValue = "true", matchIfMissing = true)
-public class NcmpAsyncRequestResponseEventConsumer {
-
-    private final EventsPublisher<NcmpAsyncRequestResponseEvent> eventsPublisher;
-    private final NcmpAsyncRequestResponseEventMapper ncmpAsyncRequestResponseEventMapper;
-
-    /**
-     * Consume the specified event.
-     *
-     * @param dmiAsyncRequestResponseEvent the event to be consumed and produced.
-     */
-    @KafkaListener(
-            topics = "${app.ncmp.async-m2m.topic}",
-            properties = {"spring.json.value.default.type=org.onap.cps.ncmp.event.model.DmiAsyncRequestResponseEvent"})
-    public void consumeAndForward(final DmiAsyncRequestResponseEvent dmiAsyncRequestResponseEvent) {
-        log.debug("Consuming event {} ...", dmiAsyncRequestResponseEvent);
-
-        final NcmpAsyncRequestResponseEvent ncmpAsyncRequestResponseEvent =
-                ncmpAsyncRequestResponseEventMapper.toNcmpAsyncEvent(dmiAsyncRequestResponseEvent);
-        eventsPublisher.publishEvent(ncmpAsyncRequestResponseEvent.getEventTarget(),
-                ncmpAsyncRequestResponseEvent.getEventId(), ncmpAsyncRequestResponseEvent);
-    }
-}
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/RecordFilterStrategies.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/RecordFilterStrategies.java
new file mode 100644
index 0000000000..0404790408
--- /dev/null
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/RecordFilterStrategies.java
@@ -0,0 +1,77 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2023 Nordix Foundation
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ * SPDX-License-Identifier: Apache-2.0
+ * ============LICENSE_END=========================================================
+ */
+
+package org.onap.cps.ncmp.api.impl.async;
+
+import io.cloudevents.CloudEvent;
+import io.cloudevents.kafka.impl.KafkaHeaders;
+import java.io.Serializable;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.kafka.common.header.Headers;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.kafka.listener.adapter.RecordFilterStrategy;
+
+/**
+ * Record filter strategies, which helps to filter the consumer records based on some conditions.
+ *
+ */
+@Configuration
+@Slf4j
+public class RecordFilterStrategies {
+
+    private static final boolean EXCLUDE_EVENT = true;
+
+    /**
+     *  Include only DataOperation events based on the cloud event type header, It
+     *  returns boolean, true means exclude the consumer record and false
+     *  means include the consumer record.
+     * @return boolean value.
+     */
+    @Bean
+    public RecordFilterStrategy<String, CloudEvent> includeDataOperationEventsOnly() {
+        return consumerRecord ->
+                isNotCloudEventOfType(consumerRecord.headers(), "DataOperationEvent");
+    }
+
+    /**
+     *  Includes the consumer records based on the cloud event type header, It  returns boolean,
+     *  true means exclude the consumer record and false means include the consumer record.
+     *  It includes only the legacy events i.e. non-cloud events
+     * @return boolean value.
+     */
+    @Bean
+    public RecordFilterStrategy<String, Serializable> includeNonCloudEventsOnly() {
+        return consumerRecord -> isCloudEvent(consumerRecord.headers());
+    }
+
+    private boolean isCloudEvent(final Headers headers) {
+        return headers.lastHeader("ce_type") != null;
+    }
+
+    private boolean isNotCloudEventOfType(final Headers headers, final String requiredEventType) {
+        final String eventTypeHeaderValue = KafkaHeaders.getParsedKafkaHeader(headers, "ce_type");
+        if (eventTypeHeaderValue == null) {
+            log.trace("No ce_type header found, possibly a legacy event (ignored)");
+            return EXCLUDE_EVENT;
+        }
+        return !(eventTypeHeaderValue.contains(requiredEventType));
+    }
+}
-- 
cgit