summaryrefslogtreecommitdiffstats
path: root/cps-ncmp-service
diff options
context:
space:
mode:
authormpriyank <priyank.maheshwari@est.tech>2023-08-10 14:27:51 +0100
committermpriyank <priyank.maheshwari@est.tech>2023-08-11 12:53:49 +0100
commit2d73a48da8ee7a9e78854cb08ef7ffb72933d613 (patch)
tree288c353e3a157542b02ab460a7ac03e5a80207b4 /cps-ncmp-service
parent2fe802a8bdda1e628c4558f9079439893e7bd52e (diff)
Generic Cloud Event Mapper
- Introduced a generic event cloud mapper to be used all over the codebase - Added as a utility so we dont have to inject it as spring bean - Refactored code and added tests - Incorporated review comments Issue-ID: CPS-1827 Change-Id: I8dab914791e01bbf6492ce96e7ba6eaa8f75a6cb Signed-off-by: mpriyank <priyank.maheshwari@est.tech>
Diffstat (limited to 'cps-ncmp-service')
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionDmiOutEventConsumer.java6
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionNcmpInEventConsumer.java5
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/mapper/CloudEventMapper.java62
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/utils/CmSubscriptionEventCloudMapper.java25
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/utils/SubscriptionEventResponseCloudMapper.java58
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/async/DataOperationEventConsumerSpec.groovy12
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avc/AvcEventConsumerSpec.groovy6
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionDmiOutEventConsumerSpec.groovy4
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionNcmpInEventConsumerSpec.groovy4
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionNcmpInEventForwarderSpec.groovy51
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/mapper/CloudEventMapperSpec.groovy54
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/operations/DmiDataOperationsSpec.groovy5
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/CmSubscriptionEventCloudMapperSpec.groovy31
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/SubscriptionEventResponseCloudMapperSpec.groovy74
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/data/operation/ResourceDataOperationRequestUtilsSpec.groovy8
15 files changed, 156 insertions, 249 deletions
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionDmiOutEventConsumer.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionDmiOutEventConsumer.java
index 3a7e0c6cf..1ac404411 100644
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionDmiOutEventConsumer.java
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionDmiOutEventConsumer.java
@@ -20,6 +20,8 @@
package org.onap.cps.ncmp.api.impl.events.cmsubscription;
+import static org.onap.cps.ncmp.api.impl.events.mapper.CloudEventMapper.toTargetEvent;
+
import com.hazelcast.map.IMap;
import io.cloudevents.CloudEvent;
import java.util.Collection;
@@ -33,7 +35,6 @@ import org.onap.cps.ncmp.api.impl.config.embeddedcache.ForwardedSubscriptionEven
import org.onap.cps.ncmp.api.impl.subscriptions.SubscriptionPersistence;
import org.onap.cps.ncmp.api.impl.subscriptions.SubscriptionStatus;
import org.onap.cps.ncmp.api.impl.utils.DataNodeHelper;
-import org.onap.cps.ncmp.api.impl.utils.SubscriptionEventResponseCloudMapper;
import org.onap.cps.ncmp.api.impl.yangmodels.YangModelSubscriptionEvent;
import org.onap.cps.ncmp.events.cmsubscription1_0_0.dmi_to_ncmp.CmSubscriptionDmiOutEvent;
import org.onap.cps.spi.model.DataNode;
@@ -51,7 +52,6 @@ public class CmSubscriptionDmiOutEventConsumer {
private final CmSubscriptionDmiOutEventToYangModelSubscriptionEventMapper
cmSubscriptionDmiOutEventToYangModelSubscriptionEventMapper;
private final CmSubscriptionNcmpOutEventPublisher cmSubscriptionNcmpOutEventPublisher;
- private final SubscriptionEventResponseCloudMapper subscriptionEventResponseCloudMapper;
@Value("${notification.enabled:true}")
private boolean notificationFeatureEnabled;
@@ -71,7 +71,7 @@ public class CmSubscriptionDmiOutEventConsumer {
final CloudEvent cloudEvent = cmSubscriptionDmiOutConsumerRecord.value();
final String eventType = cmSubscriptionDmiOutConsumerRecord.value().getType();
final CmSubscriptionDmiOutEvent cmSubscriptionDmiOutEvent =
- subscriptionEventResponseCloudMapper.toCmSubscriptionDmiOutEvent(cloudEvent);
+ toTargetEvent(cloudEvent, CmSubscriptionDmiOutEvent.class);
final String clientId = cmSubscriptionDmiOutEvent.getData().getClientId();
log.info("subscription event response of clientId: {} is received.", clientId);
final String subscriptionName = cmSubscriptionDmiOutEvent.getData().getSubscriptionName();
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionNcmpInEventConsumer.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionNcmpInEventConsumer.java
index c64ebacb0..f1c64c788 100644
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionNcmpInEventConsumer.java
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionNcmpInEventConsumer.java
@@ -20,6 +20,7 @@
package org.onap.cps.ncmp.api.impl.events.cmsubscription;
+import static org.onap.cps.ncmp.api.impl.events.mapper.CloudEventMapper.toTargetEvent;
import static org.onap.cps.ncmp.api.impl.operations.DatastoreType.PASSTHROUGH_OPERATIONAL;
import static org.onap.cps.ncmp.api.impl.operations.DatastoreType.PASSTHROUGH_RUNNING;
@@ -28,7 +29,6 @@ import lombok.RequiredArgsConstructor;
import lombok.extern.slf4j.Slf4j;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.onap.cps.ncmp.api.impl.subscriptions.SubscriptionPersistence;
-import org.onap.cps.ncmp.api.impl.utils.CmSubscriptionEventCloudMapper;
import org.onap.cps.ncmp.api.impl.yangmodels.YangModelSubscriptionEvent;
import org.onap.cps.ncmp.events.cmsubscription1_0_0.client_to_ncmp.CmSubscriptionNcmpInEvent;
import org.springframework.beans.factory.annotation.Value;
@@ -44,7 +44,6 @@ public class CmSubscriptionNcmpInEventConsumer {
private final CmSubscriptionNcmpInEventForwarder cmSubscriptionNcmpInEventForwarder;
private final CmSubscriptionNcmpInEventMapper cmSubscriptionNcmpInEventMapper;
private final SubscriptionPersistence subscriptionPersistence;
- private final CmSubscriptionEventCloudMapper cmSubscriptionEventCloudMapper;
@Value("${notification.enabled:true}")
private boolean notificationFeatureEnabled;
@@ -63,7 +62,7 @@ public class CmSubscriptionNcmpInEventConsumer {
final CloudEvent cloudEvent = subscriptionEventConsumerRecord.value();
final String eventType = subscriptionEventConsumerRecord.value().getType();
final CmSubscriptionNcmpInEvent cmSubscriptionNcmpInEvent =
- cmSubscriptionEventCloudMapper.toCmSubscriptionNcmpInEvent(cloudEvent);
+ toTargetEvent(cloudEvent, CmSubscriptionNcmpInEvent.class);
final String eventDatastore = cmSubscriptionNcmpInEvent.getData().getPredicates().getDatastore();
if (!eventDatastore.equals(PASSTHROUGH_RUNNING.getDatastoreName()) || eventDatastore.equals(
PASSTHROUGH_OPERATIONAL.getDatastoreName())) {
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/mapper/CloudEventMapper.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/mapper/CloudEventMapper.java
new file mode 100644
index 000000000..98ba95386
--- /dev/null
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/mapper/CloudEventMapper.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.events.mapper;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import io.cloudevents.CloudEvent;
+import io.cloudevents.core.CloudEventUtils;
+import io.cloudevents.core.data.PojoCloudEventData;
+import io.cloudevents.jackson.PojoCloudEventDataMapper;
+import io.cloudevents.rw.CloudEventRWException;
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public class CloudEventMapper {
+
+ private static final ObjectMapper objectMapper = new ObjectMapper();
+
+ /**
+ * Generic method to map cloud event data to target event class object.
+ *
+ * @param cloudEvent input cloud event
+ * @param targetEventClass target event class
+ * @param <T> target event class type
+ * @return mapped target event
+ */
+ public static <T> T toTargetEvent(final CloudEvent cloudEvent, final Class<T> targetEventClass) {
+ PojoCloudEventData<T> mappedCloudEvent = null;
+
+ try {
+ mappedCloudEvent =
+ CloudEventUtils.mapData(cloudEvent, PojoCloudEventDataMapper.from(objectMapper, targetEventClass));
+
+ } catch (final CloudEventRWException cloudEventRwException) {
+ log.error("Unable to map cloud event to target event class type : {} with cause : {}", targetEventClass,
+ cloudEventRwException.getMessage());
+ }
+
+ return mappedCloudEvent == null ? null : mappedCloudEvent.getValue();
+ }
+
+}
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/utils/CmSubscriptionEventCloudMapper.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/utils/CmSubscriptionEventCloudMapper.java
index 5bc38e1dc..8e565d220 100644
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/utils/CmSubscriptionEventCloudMapper.java
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/utils/CmSubscriptionEventCloudMapper.java
@@ -23,15 +23,11 @@ package org.onap.cps.ncmp.api.impl.utils;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.cloudevents.CloudEvent;
-import io.cloudevents.core.CloudEventUtils;
import io.cloudevents.core.builder.CloudEventBuilder;
-import io.cloudevents.core.data.PojoCloudEventData;
-import io.cloudevents.jackson.PojoCloudEventDataMapper;
import java.net.URI;
import java.util.UUID;
import lombok.RequiredArgsConstructor;
import lombok.extern.slf4j.Slf4j;
-import org.onap.cps.ncmp.events.cmsubscription1_0_0.client_to_ncmp.CmSubscriptionNcmpInEvent;
import org.onap.cps.ncmp.events.cmsubscription1_0_0.ncmp_to_dmi.CmSubscriptionDmiInEvent;
import org.springframework.stereotype.Component;
@@ -45,29 +41,10 @@ public class CmSubscriptionEventCloudMapper {
private static String randomId = UUID.randomUUID().toString();
/**
- * Maps CloudEvent object to CmSubscriptionNcmpInEvent.
- *
- * @param cloudEvent object.
- * @return CmSubscriptionNcmpInEvent deserialized.
- */
- public CmSubscriptionNcmpInEvent toCmSubscriptionNcmpInEvent(final CloudEvent cloudEvent) {
- final PojoCloudEventData<CmSubscriptionNcmpInEvent> deserializedCloudEvent = CloudEventUtils
- .mapData(cloudEvent, PojoCloudEventDataMapper.from(objectMapper, CmSubscriptionNcmpInEvent.class));
- if (deserializedCloudEvent == null) {
- log.debug("No data found in the consumed event");
- return null;
- } else {
- final CmSubscriptionNcmpInEvent cmSubscriptionNcmpInEvent = deserializedCloudEvent.getValue();
- log.debug("Consuming event {}", cmSubscriptionNcmpInEvent);
- return cmSubscriptionNcmpInEvent;
- }
- }
-
- /**
* Maps CmSubscriptionDmiInEvent to a CloudEvent.
*
* @param cmSubscriptionDmiInEvent object.
- * @param eventKey as String.
+ * @param eventKey as String.
* @return CloudEvent built.
*/
public CloudEvent toCloudEvent(final CmSubscriptionDmiInEvent cmSubscriptionDmiInEvent, final String eventKey,
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/utils/SubscriptionEventResponseCloudMapper.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/utils/SubscriptionEventResponseCloudMapper.java
deleted file mode 100644
index 0721d1d56..000000000
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/utils/SubscriptionEventResponseCloudMapper.java
+++ /dev/null
@@ -1,58 +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.utils;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import io.cloudevents.CloudEvent;
-import io.cloudevents.core.CloudEventUtils;
-import io.cloudevents.core.data.PojoCloudEventData;
-import io.cloudevents.jackson.PojoCloudEventDataMapper;
-import lombok.RequiredArgsConstructor;
-import lombok.extern.slf4j.Slf4j;
-import org.onap.cps.ncmp.events.cmsubscription1_0_0.dmi_to_ncmp.CmSubscriptionDmiOutEvent;
-import org.springframework.stereotype.Component;
-
-@Slf4j
-@Component
-@RequiredArgsConstructor
-public class SubscriptionEventResponseCloudMapper {
-
- private final ObjectMapper objectMapper;
-
- /**
- * Maps CloudEvent object to CmSubscriptionDmiOutEvent.
- *
- * @param cloudEvent object
- * @return CmSubscriptionDmiOutEvent deserialized
- */
- public CmSubscriptionDmiOutEvent toCmSubscriptionDmiOutEvent(final CloudEvent cloudEvent) {
- final PojoCloudEventData<CmSubscriptionDmiOutEvent> deserializedCloudEvent = CloudEventUtils
- .mapData(cloudEvent, PojoCloudEventDataMapper.from(objectMapper, CmSubscriptionDmiOutEvent.class));
- if (deserializedCloudEvent == null) {
- log.debug("No data found in the consumed subscription response event");
- return null;
- } else {
- final CmSubscriptionDmiOutEvent cmSubscriptionDmiOutEvent = deserializedCloudEvent.getValue();
- log.debug("Consuming subscription response event {}", cmSubscriptionDmiOutEvent);
- return cmSubscriptionDmiOutEvent;
- }
- }
-}
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/async/DataOperationEventConsumerSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/async/DataOperationEventConsumerSpec.groovy
index 7b54f591a..9b71fbeae 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/async/DataOperationEventConsumerSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/async/DataOperationEventConsumerSpec.groovy
@@ -25,9 +25,7 @@ import io.cloudevents.CloudEvent
import io.cloudevents.kafka.CloudEventDeserializer
import io.cloudevents.kafka.CloudEventSerializer
import io.cloudevents.kafka.impl.KafkaHeaders
-import io.cloudevents.core.CloudEventUtils
import io.cloudevents.core.builder.CloudEventBuilder
-import io.cloudevents.jackson.PojoCloudEventDataMapper
import org.apache.kafka.clients.consumer.ConsumerRecord
import org.apache.kafka.clients.consumer.KafkaConsumer
import org.apache.kafka.common.header.internals.RecordHeaders
@@ -44,7 +42,9 @@ import org.springframework.test.annotation.DirtiesContext
import org.testcontainers.spock.Testcontainers
import java.time.Duration
-@SpringBootTest(classes = [EventsPublisher, DataOperationEventConsumer, RecordFilterStrategies,JsonObjectMapper, ObjectMapper])
+import static org.onap.cps.ncmp.api.impl.events.mapper.CloudEventMapper.toTargetEvent
+
+@SpringBootTest(classes = [EventsPublisher, DataOperationEventConsumer, RecordFilterStrategies, JsonObjectMapper, ObjectMapper])
@Testcontainers
@DirtiesContext
class DataOperationEventConsumerSpec extends MessagingBaseSpec {
@@ -61,9 +61,6 @@ class DataOperationEventConsumerSpec extends MessagingBaseSpec {
@Autowired
RecordFilterStrategy<String, CloudEvent> dataOperationRecordFilterStrategy
- @Autowired
- ObjectMapper objectMapper
-
def cloudEventKafkaConsumer = new KafkaConsumer<>(eventConsumerConfigProperties('test', CloudEventDeserializer))
def static clientTopic = 'client-topic'
def static dataOperationType = 'org.onap.cps.ncmp.events.async1_0_0.DataOperationEvent'
@@ -85,8 +82,7 @@ class DataOperationEventConsumerSpec extends MessagingBaseSpec {
and: 'verify that extension is included into header'
assert KafkaHeaders.getParsedKafkaHeader(consumerRecordOutHeaders, 'ce_destination') == clientTopic
and: 'map consumer record to expected event type'
- def dataOperationResponseEvent = CloudEventUtils.mapData(consumerRecordOut.value(),
- PojoCloudEventDataMapper.from(objectMapper, DataOperationEvent.class)).getValue()
+ def dataOperationResponseEvent = toTargetEvent(consumerRecordOut.value(), DataOperationEvent.class)
and: 'verify published response data properties'
def response = dataOperationResponseEvent.data.responses[0]
response.operationId == 'some-operation-id'
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avc/AvcEventConsumerSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avc/AvcEventConsumerSpec.groovy
index 22852bea4..c9a4bd16a 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avc/AvcEventConsumerSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avc/AvcEventConsumerSpec.groovy
@@ -22,9 +22,7 @@ package org.onap.cps.ncmp.api.impl.events.avc
import com.fasterxml.jackson.databind.ObjectMapper
import io.cloudevents.CloudEvent
-import io.cloudevents.core.CloudEventUtils
import io.cloudevents.core.builder.CloudEventBuilder
-import io.cloudevents.jackson.PojoCloudEventDataMapper
import io.cloudevents.kafka.CloudEventDeserializer
import io.cloudevents.kafka.impl.KafkaHeaders
import org.apache.kafka.clients.consumer.ConsumerRecord
@@ -41,6 +39,8 @@ import org.springframework.test.annotation.DirtiesContext
import org.testcontainers.spock.Testcontainers
import java.time.Duration
+import static org.onap.cps.ncmp.api.impl.events.mapper.CloudEventMapper.toTargetEvent
+
@SpringBootTest(classes = [EventsPublisher, AvcEventConsumer, ObjectMapper, JsonObjectMapper])
@Testcontainers
@DirtiesContext
@@ -82,7 +82,7 @@ class AvcEventConsumerSpec extends MessagingBaseSpec {
and: 'record can be converted to AVC event'
def record = records.iterator().next()
def cloudEvent = record.value() as CloudEvent
- def convertedAvcEvent = CloudEventUtils.mapData(cloudEvent, PojoCloudEventDataMapper.from(new ObjectMapper(), AvcEvent.class)).getValue()
+ def convertedAvcEvent = toTargetEvent(cloudEvent, AvcEvent.class)
and: 'we have correct headers forwarded where correlation id matches'
assert KafkaHeaders.getParsedKafkaHeader(record.headers(), 'ce_correlationid') == 'test-cmhandle1'
and: 'event id differs(as per requirement) between consumed and forwarded'
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionDmiOutEventConsumerSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionDmiOutEventConsumerSpec.groovy
index 29defbecc..a8a21b2ed 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionDmiOutEventConsumerSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionDmiOutEventConsumerSpec.groovy
@@ -26,7 +26,6 @@ import io.cloudevents.CloudEvent
import io.cloudevents.core.builder.CloudEventBuilder
import org.apache.kafka.clients.consumer.ConsumerRecord
import org.onap.cps.ncmp.api.impl.subscriptions.SubscriptionPersistenceImpl
-import org.onap.cps.ncmp.api.impl.utils.SubscriptionEventResponseCloudMapper
import org.onap.cps.ncmp.api.kafka.MessagingBaseSpec
import org.onap.cps.ncmp.events.cmsubscription1_0_0.dmi_to_ncmp.CmSubscriptionDmiOutEvent
import org.onap.cps.ncmp.utils.TestUtils
@@ -48,10 +47,9 @@ class CmSubscriptionDmiOutEventConsumerSpec extends MessagingBaseSpec {
def mockSubscriptionPersistence = Mock(SubscriptionPersistenceImpl)
def mockSubscriptionEventResponseMapper = Mock(CmSubscriptionDmiOutEventToYangModelSubscriptionEventMapper)
def mockSubscriptionEventResponseOutcome = Mock(CmSubscriptionNcmpOutEventPublisher)
- def mockSubscriptionEventResponseCloudMapper = new SubscriptionEventResponseCloudMapper(new ObjectMapper())
def objectUnderTest = new CmSubscriptionDmiOutEventConsumer(mockForwardedSubscriptionEventCache,
- mockSubscriptionPersistence, mockSubscriptionEventResponseMapper, mockSubscriptionEventResponseOutcome, mockSubscriptionEventResponseCloudMapper)
+ mockSubscriptionPersistence, mockSubscriptionEventResponseMapper, mockSubscriptionEventResponseOutcome)
def 'Consume Subscription Event Response where all DMIs have responded'() {
given: 'a consumer record including cloud event having subscription response'
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionNcmpInEventConsumerSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionNcmpInEventConsumerSpec.groovy
index 24e829e28..31e883958 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionNcmpInEventConsumerSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionNcmpInEventConsumerSpec.groovy
@@ -25,7 +25,6 @@ import io.cloudevents.CloudEvent
import io.cloudevents.core.builder.CloudEventBuilder
import org.apache.kafka.clients.consumer.ConsumerRecord
import org.onap.cps.ncmp.api.impl.subscriptions.SubscriptionPersistence
-import org.onap.cps.ncmp.api.impl.utils.CmSubscriptionEventCloudMapper
import org.onap.cps.ncmp.api.impl.yangmodels.YangModelSubscriptionEvent
import org.onap.cps.ncmp.api.kafka.MessagingBaseSpec
import org.onap.cps.ncmp.events.cmsubscription1_0_0.client_to_ncmp.CmSubscriptionNcmpInEvent
@@ -40,8 +39,7 @@ class CmSubscriptionNcmpInEventConsumerSpec extends MessagingBaseSpec {
def mockCmSubscriptionNcmpInEventForwarder = Mock(CmSubscriptionNcmpInEventForwarder)
def mockCmSubscriptionNcmpInEventMapper = Mock(CmSubscriptionNcmpInEventMapper)
def mockSubscriptionPersistence = Mock(SubscriptionPersistence)
- def cmSubscriptionEventCloudMapper = new CmSubscriptionEventCloudMapper(new ObjectMapper())
- def objectUnderTest = new CmSubscriptionNcmpInEventConsumer(mockCmSubscriptionNcmpInEventForwarder, mockCmSubscriptionNcmpInEventMapper, mockSubscriptionPersistence, cmSubscriptionEventCloudMapper)
+ def objectUnderTest = new CmSubscriptionNcmpInEventConsumer(mockCmSubscriptionNcmpInEventForwarder, mockCmSubscriptionNcmpInEventMapper, mockSubscriptionPersistence)
def yangModelSubscriptionEvent = new YangModelSubscriptionEvent()
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionNcmpInEventForwarderSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionNcmpInEventForwarderSpec.groovy
index 29cf8be8b..a13fe53e8 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionNcmpInEventForwarderSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/cmsubscription/CmSubscriptionNcmpInEventForwarderSpec.groovy
@@ -23,9 +23,6 @@ package org.onap.cps.ncmp.api.impl.events.cmsubscription
import com.fasterxml.jackson.databind.ObjectMapper
import com.hazelcast.map.IMap
import io.cloudevents.CloudEvent
-import io.cloudevents.core.CloudEventUtils
-import io.cloudevents.core.data.PojoCloudEventData
-import io.cloudevents.jackson.PojoCloudEventDataMapper
import org.mapstruct.factory.Mappers
import org.onap.cps.ncmp.api.impl.events.EventsPublisher
import org.onap.cps.ncmp.api.impl.subscriptions.SubscriptionPersistence
@@ -39,7 +36,7 @@ import org.onap.cps.ncmp.events.cmsubscription1_0_0.client_to_ncmp.CmSubscriptio
import org.onap.cps.ncmp.events.cmsubscription1_0_0.dmi_to_ncmp.CmSubscriptionDmiOutEvent
import org.onap.cps.ncmp.events.cmsubscription1_0_0.dmi_to_ncmp.Data
import org.onap.cps.ncmp.events.cmsubscription1_0_0.ncmp_to_dmi.CmHandle
-import org.onap.cps.ncmp.events.cmsubscription1_0_0.ncmp_to_dmi.CmSubscriptionDmiInEvent;
+import org.onap.cps.ncmp.events.cmsubscription1_0_0.ncmp_to_dmi.CmSubscriptionDmiInEvent
import org.onap.cps.ncmp.utils.TestUtils
import org.onap.cps.utils.JsonObjectMapper
import org.spockframework.spring.SpringBean
@@ -48,12 +45,13 @@ import org.springframework.boot.test.context.SpringBootTest
import spock.util.concurrent.BlockingVariable
import java.util.concurrent.TimeUnit
+import static org.onap.cps.ncmp.api.impl.events.mapper.CloudEventMapper.toTargetEvent
+
@SpringBootTest(classes = [ObjectMapper, JsonObjectMapper, CmSubscriptionNcmpInEventForwarder])
class CmSubscriptionNcmpInEventForwarderSpec extends MessagingBaseSpec {
@Autowired
CmSubscriptionNcmpInEventForwarder objectUnderTest
-
@SpringBean
InventoryPersistence mockInventoryPersistence = Mock(InventoryPersistence)
@SpringBean
@@ -81,8 +79,8 @@ class CmSubscriptionNcmpInEventForwarderSpec extends MessagingBaseSpec {
def testEventSent = jsonObjectMapper.convertJsonString(jsonData, CmSubscriptionNcmpInEvent.class)
and: 'the InventoryPersistence returns private properties for the supplied CM Handles'
1 * mockInventoryPersistence.getYangModelCmHandles(["CMHandle1", "CMHandle2", "CMHandle3"]) >> [
- createYangModelCmHandleWithDmiProperty(1, 1,"shape","circle"),
- createYangModelCmHandleWithDmiProperty(2, 1,"shape","square")
+ createYangModelCmHandleWithDmiProperty(1, 1, "shape", "circle"),
+ createYangModelCmHandleWithDmiProperty(2, 1, "shape", "square")
]
and: 'the thread creation delay is reduced to 2 seconds for testing'
objectUnderTest.dmiResponseTimeoutInMs = 2000
@@ -97,9 +95,9 @@ class CmSubscriptionNcmpInEventForwarderSpec extends MessagingBaseSpec {
and: 'the event is forwarded twice with the CMHandle private properties and provides a valid listenable future'
1 * mockSubscriptionEventPublisher.publishCloudEvent("ncmp-dmi-cm-avc-subscription-DMIName1", "SCO-9989752-cm-subscription-001-DMIName1",
cloudEvent -> {
- def targets = toSubscriptionEvent(cloudEvent).getData().getPredicates().getTargets()
- def cmHandle2 = createCmHandle('CMHandle2', ['shape':'square'] as Map)
- def cmHandle1 = createCmHandle('CMHandle1', ['shape':'circle'] as Map)
+ def targets = toTargetEvent(cloudEvent, CmSubscriptionDmiInEvent.class).getData().getPredicates().getTargets()
+ def cmHandle2 = createCmHandle('CMHandle2', ['shape': 'square'] as Map)
+ def cmHandle1 = createCmHandle('CMHandle1', ['shape': 'circle'] as Map)
targets == [cmHandle2, cmHandle1]
}
)
@@ -107,7 +105,7 @@ class CmSubscriptionNcmpInEventForwarderSpec extends MessagingBaseSpec {
1 * mockForwardedSubscriptionEventCache.containsKey("SCO-9989752cm-subscription-001") >> true
1 * mockCmSubscriptionNcmpOutEventPublisher.sendResponse(*_)
and: 'the subscription id is removed from the event cache map returning the asynchronous blocking variable'
- 1 * mockForwardedSubscriptionEventCache.remove("SCO-9989752cm-subscription-001") >> {block.set(_)}
+ 1 * mockForwardedSubscriptionEventCache.remove("SCO-9989752cm-subscription-001") >> { block.set(_) }
}
def 'Forward CM create subscription where target CM Handles are #scenario'() {
@@ -137,8 +135,8 @@ class CmSubscriptionNcmpInEventForwarderSpec extends MessagingBaseSpec {
emptySubscriptionEventResponse.getData().setClientId('SCO-9989752');
and: 'the cm handles will be rejected'
def rejectedCmHandles = [new TargetCmHandle('CMHandle1', SubscriptionStatus.REJECTED, 'Cm handle does not exist'),
- new TargetCmHandle('CMHandle2',SubscriptionStatus.REJECTED, 'Cm handle does not exist'),
- new TargetCmHandle('CMHandle3',SubscriptionStatus.REJECTED, 'Cm handle does not exist')]
+ new TargetCmHandle('CMHandle2', SubscriptionStatus.REJECTED, 'Cm handle does not exist'),
+ new TargetCmHandle('CMHandle3', SubscriptionStatus.REJECTED, 'Cm handle does not exist')]
and: 'a yang model subscription event will be saved into the db with rejected cm handles'
def yangModelSubscriptionEvent = cmSubscriptionNcmpInEventMapper.toYangModelSubscriptionEvent(testEventSent)
yangModelSubscriptionEvent.getPredicates().setTargetCmHandles(rejectedCmHandles)
@@ -155,16 +153,16 @@ class CmSubscriptionNcmpInEventForwarderSpec extends MessagingBaseSpec {
and: 'the event is not being forwarded with the CMHandle private properties and does not provides a valid listenable future'
0 * mockSubscriptionEventPublisher.publishCloudEvent("ncmp-dmi-cm-avc-subscription-DMIName1", "SCO-9989752-cm-subscription-001-DMIName1",
cloudEvent -> {
- def targets = toSubscriptionEvent(cloudEvent).getData().getPredicates().getTargets()
- def cmHandle2 = createCmHandle('CMHandle2', ['shape':'square'] as Map)
- def cmHandle1 = createCmHandle('CMHandle1', ['shape':'circle'] as Map)
+ def targets = toTargetEvent(cloudEvent, CmSubscriptionDmiInEvent.class).getData().getPredicates().getTargets()
+ def cmHandle2 = createCmHandle('CMHandle2', ['shape': 'square'] as Map)
+ def cmHandle1 = createCmHandle('CMHandle1', ['shape': 'circle'] as Map)
targets == [cmHandle2, cmHandle1]
}
)
0 * mockSubscriptionEventPublisher.publishCloudEvent("ncmp-dmi-cm-avc-subscription-DMIName2", "SCO-9989752-cm-subscription-001-DMIName2",
cloudEvent -> {
- def targets = toSubscriptionEvent(cloudEvent).getData().getPredicates().getTargets()
- def cmHandle3 = createCmHandle('CMHandle3', ['shape':'triangle'] as Map)
+ def targets = toTargetEvent(cloudEvent, CmSubscriptionDmiInEvent.class).getData().getPredicates().getTargets()
+ def cmHandle3 = createCmHandle('CMHandle3', ['shape': 'triangle'] as Map)
targets == [cmHandle3]
}
)
@@ -172,15 +170,15 @@ class CmSubscriptionNcmpInEventForwarderSpec extends MessagingBaseSpec {
0 * mockForwardedSubscriptionEventCache.containsKey("SCO-9989752cm-subscription-001") >> true
0 * mockForwardedSubscriptionEventCache.get(_)
and: 'the subscription id is removed from the event cache map returning the asynchronous blocking variable'
- 0 * mockForwardedSubscriptionEventCache.remove("SCO-9989752cm-subscription-001") >> {block.set(_)}
+ 0 * mockForwardedSubscriptionEventCache.remove("SCO-9989752cm-subscription-001") >> { block.set(_) }
and: 'the persistence service save target cm handles of the yang model subscription event as rejected '
1 * mockSubscriptionPersistence.saveSubscriptionEvent(yangModelSubscriptionEvent)
and: 'subscription outcome has been sent'
1 * mockCmSubscriptionNcmpOutEventPublisher.sendResponse(emptySubscriptionEventResponse, 'subscriptionCreatedStatus')
}
- static def createYangModelCmHandleWithDmiProperty(id, dmiId,propertyName, propertyValue) {
- return new YangModelCmHandle(id:"CMHandle" + id, dmiDataServiceName: "DMIName" + dmiId, dmiProperties: [new YangModelCmHandle.Property(propertyName,propertyValue)])
+ static def createYangModelCmHandleWithDmiProperty(id, dmiId, propertyName, propertyValue) {
+ return new YangModelCmHandle(id: "CMHandle" + id, dmiDataServiceName: "DMIName" + dmiId, dmiProperties: [new YangModelCmHandle.Property(propertyName, propertyValue)])
}
static def createCmHandle(id, additionalProperties) {
@@ -190,15 +188,4 @@ class CmSubscriptionNcmpInEventForwarderSpec extends MessagingBaseSpec {
return cmHandle
}
- def toSubscriptionEvent(cloudEvent) {
- final PojoCloudEventData<CmSubscriptionDmiInEvent> deserializedCloudEvent = CloudEventUtils
- .mapData(cloudEvent, PojoCloudEventDataMapper.from(objectMapper,
- CmSubscriptionDmiInEvent.class));
- if (deserializedCloudEvent == null) {
- return null;
- } else {
- return deserializedCloudEvent.getValue();
- }
- }
-
}
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/mapper/CloudEventMapperSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/mapper/CloudEventMapperSpec.groovy
new file mode 100644
index 000000000..380aea405
--- /dev/null
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/mapper/CloudEventMapperSpec.groovy
@@ -0,0 +1,54 @@
+/*
+ * ============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.events.mapper
+
+import com.fasterxml.jackson.databind.ObjectMapper
+import io.cloudevents.core.builder.CloudEventBuilder
+import org.onap.cps.ncmp.events.cmsubscription1_0_0.client_to_ncmp.CmSubscriptionNcmpInEvent
+import org.onap.cps.utils.JsonObjectMapper
+import org.springframework.beans.factory.annotation.Autowired
+import org.springframework.boot.test.context.SpringBootTest
+import spock.lang.Specification
+
+@SpringBootTest(classes = [ObjectMapper, JsonObjectMapper])
+class CloudEventMapperSpec extends Specification {
+
+ @Autowired
+ JsonObjectMapper jsonObjectMapper
+
+ def 'Cloud event to Target event type when it is #scenario'() {
+ expect: 'Events mapped correctly'
+ assert mappedCloudEvent == (CloudEventMapper.toTargetEvent(testCloudEvent(), targetClass) != null)
+ where: 'below are the scenarios'
+ scenario | targetClass || mappedCloudEvent
+ 'valid concrete type' | CmSubscriptionNcmpInEvent.class || true
+ 'invalid concrete type' | ArrayList.class || false
+ }
+
+ def testCloudEvent() {
+ return CloudEventBuilder.v1().withData(jsonObjectMapper.asJsonBytes(new CmSubscriptionNcmpInEvent()))
+ .withId("cmhandle1")
+ .withSource(URI.create('test-source'))
+ .withDataSchema(URI.create('test'))
+ .withType('org.onap.cm.events.cm-subscription')
+ .build()
+ }
+}
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/operations/DmiDataOperationsSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/operations/DmiDataOperationsSpec.groovy
index b60f4e65f..76ef23fe5 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/operations/DmiDataOperationsSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/operations/DmiDataOperationsSpec.groovy
@@ -22,8 +22,6 @@
package org.onap.cps.ncmp.api.impl.operations
import com.fasterxml.jackson.databind.ObjectMapper
-import io.cloudevents.core.CloudEventUtils
-import io.cloudevents.jackson.PojoCloudEventDataMapper
import org.onap.cps.ncmp.api.NcmpEventResponseCode
import org.onap.cps.ncmp.api.impl.config.NcmpConfiguration
import org.onap.cps.ncmp.api.impl.events.EventsPublisher
@@ -48,6 +46,7 @@ import static org.onap.cps.ncmp.api.impl.operations.DatastoreType.PASSTHROUGH_RU
import static org.onap.cps.ncmp.api.impl.operations.OperationType.CREATE
import static org.onap.cps.ncmp.api.impl.operations.OperationType.READ
import static org.onap.cps.ncmp.api.impl.operations.OperationType.UPDATE
+import static org.onap.cps.ncmp.api.impl.events.mapper.CloudEventMapper.toTargetEvent
@SpringBootTest
@ContextConfiguration(classes = [EventsPublisher, CpsApplicationContext, NcmpConfiguration.DmiProperties, DmiDataOperations])
@@ -171,6 +170,6 @@ class DmiDataOperationsSpec extends DmiOperationsBaseSpec {
}
def extractDataValue(actualDataOperationCloudEvent) {
- return CloudEventUtils.mapData(actualDataOperationCloudEvent, PojoCloudEventDataMapper.from(new ObjectMapper(), DataOperationEvent.class)).getValue().data.responses[0]
+ return toTargetEvent(actualDataOperationCloudEvent, DataOperationEvent.class).data.responses[0]
}
}
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/CmSubscriptionEventCloudMapperSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/CmSubscriptionEventCloudMapperSpec.groovy
index 4f5d3b19d..fa2828a4f 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/CmSubscriptionEventCloudMapperSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/CmSubscriptionEventCloudMapperSpec.groovy
@@ -23,7 +23,6 @@ package org.onap.cps.ncmp.api.impl.utils
import com.fasterxml.jackson.core.JsonProcessingException
import com.fasterxml.jackson.databind.ObjectMapper
import io.cloudevents.core.builder.CloudEventBuilder
-import org.onap.cps.ncmp.events.cmsubscription1_0_0.client_to_ncmp.CmSubscriptionNcmpInEvent
import org.onap.cps.ncmp.events.cmsubscription1_0_0.ncmp_to_dmi.CmSubscriptionDmiInEvent
import org.onap.cps.ncmp.utils.TestUtils
import org.onap.cps.utils.JsonObjectMapper
@@ -44,36 +43,6 @@ class CmSubscriptionEventCloudMapperSpec extends Specification {
def objectUnderTest = new CmSubscriptionEventCloudMapper(spyObjectMapper)
- def 'Map the data of the cloud event to subscription event'() {
- given: 'a cloud event having a subscription event in the data part'
- def jsonData = TestUtils.getResourceFileContent('cmSubscriptionNcmpInEvent.json')
- def testEventData = jsonObjectMapper.convertJsonString(jsonData, CmSubscriptionNcmpInEvent.class)
- def testCloudEvent = CloudEventBuilder.v1()
- .withData(objectMapper.writeValueAsBytes(testEventData))
- .withId('some-event-id')
- .withType('subscriptionCreated')
- .withSource(URI.create('some-resource'))
- .withExtension('correlationid', 'test-cmhandle1').build()
- when: 'the cloud event map to subscription event'
- def resultSubscriptionEvent = objectUnderTest.toCmSubscriptionNcmpInEvent(testCloudEvent)
- then: 'the subscription event resulted having expected values'
- resultSubscriptionEvent.getData() == testEventData.getData()
- }
-
- def 'Map the null of the data of the cloud event to subscription event'() {
- given: 'a cloud event having a null subscription event in the data part'
- def testCloudEvent = CloudEventBuilder.v1()
- .withData(null)
- .withId('some-event-id')
- .withType('subscriptionCreated')
- .withSource(URI.create('some-resource'))
- .withExtension('correlationid', 'test-cmhandle1').build()
- when: 'the cloud event map to subscription event'
- def resultSubscriptionEvent = objectUnderTest.toCmSubscriptionNcmpInEvent(testCloudEvent)
- then: 'the subscription event resulted having a null value'
- resultSubscriptionEvent == null
- }
-
def 'Map the subscription event to data of the cloud event'() {
given: 'a subscription event'
def jsonData = TestUtils.getResourceFileContent('cmSubscriptionDmiInEvent.json')
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/SubscriptionEventResponseCloudMapperSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/SubscriptionEventResponseCloudMapperSpec.groovy
deleted file mode 100644
index 89b13e26e..000000000
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/SubscriptionEventResponseCloudMapperSpec.groovy
+++ /dev/null
@@ -1,74 +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.utils
-
-import com.fasterxml.jackson.databind.ObjectMapper
-import io.cloudevents.core.builder.CloudEventBuilder
-import org.onap.cps.ncmp.events.cmsubscription1_0_0.dmi_to_ncmp.CmSubscriptionDmiOutEvent
-import org.onap.cps.ncmp.utils.TestUtils
-import org.onap.cps.utils.JsonObjectMapper
-import org.springframework.beans.factory.annotation.Autowired
-import org.springframework.boot.test.context.SpringBootTest
-import spock.lang.Specification
-
-@SpringBootTest(classes = [ObjectMapper, JsonObjectMapper])
-class SubscriptionEventResponseCloudMapperSpec extends Specification {
-
- @Autowired
- JsonObjectMapper jsonObjectMapper
-
- @Autowired
- ObjectMapper objectMapper
-
- def spyObjectMapper = Spy(ObjectMapper)
-
- def objectUnderTest = new SubscriptionEventResponseCloudMapper(spyObjectMapper)
-
- def 'Map the cloud event to subscription event response'() {
- given: 'a cloud event having a subscription event response in the data part'
- def jsonData = TestUtils.getResourceFileContent('cmSubscriptionDmiOutEvent.json')
- def testEventData = jsonObjectMapper.convertJsonString(jsonData, CmSubscriptionDmiOutEvent.class)
- def testCloudEvent = CloudEventBuilder.v1()
- .withData(objectMapper.writeValueAsBytes(testEventData))
- .withId('some-event-id')
- .withType('subscriptionCreatedStatus')
- .withSource(URI.create('some-resource'))
- .withExtension('correlationid', 'test-cmhandle1').build()
- when: 'the cloud event map to subscription event response'
- def resultSubscriptionEvent = objectUnderTest.toCmSubscriptionDmiOutEvent(testCloudEvent)
- then: 'the subscription event resulted having expected values'
- resultSubscriptionEvent.getData() == testEventData.getData()
- }
-
- def 'Map null of the data of the cloud event to subscription event response'() {
- given: 'a cloud event having a null subscription event response in the data part'
- def testCloudEvent = CloudEventBuilder.v1()
- .withData(null)
- .withId('some-event-id')
- .withType('subscriptionCreatedStatus')
- .withSource(URI.create('some-resource'))
- .withExtension('correlationid', 'test-cmhandle1').build()
- when: 'the cloud event map to subscription event response'
- def resultSubscriptionEvent = objectUnderTest.toCmSubscriptionDmiOutEvent(testCloudEvent)
- then: 'the subscription event response resulted having a null value'
- resultSubscriptionEvent == null
- }
-}
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/data/operation/ResourceDataOperationRequestUtilsSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/data/operation/ResourceDataOperationRequestUtilsSpec.groovy
index 401254f54..c86682406 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/data/operation/ResourceDataOperationRequestUtilsSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/data/operation/ResourceDataOperationRequestUtilsSpec.groovy
@@ -22,8 +22,6 @@ package org.onap.cps.ncmp.api.impl.utils.data.operation
import com.fasterxml.jackson.databind.ObjectMapper
import io.cloudevents.CloudEvent
-import io.cloudevents.core.CloudEventUtils
-import io.cloudevents.jackson.PojoCloudEventDataMapper
import io.cloudevents.kafka.CloudEventDeserializer
import io.cloudevents.kafka.impl.KafkaHeaders
import org.apache.kafka.clients.consumer.KafkaConsumer
@@ -42,6 +40,8 @@ import org.springframework.beans.factory.annotation.Autowired
import org.springframework.test.context.ContextConfiguration
import java.time.Duration
+import static org.onap.cps.ncmp.api.impl.events.mapper.CloudEventMapper.toTargetEvent
+
@ContextConfiguration(classes = [EventsPublisher, CpsApplicationContext, ObjectMapper])
class ResourceDataOperationRequestUtilsSpec extends MessagingBaseSpec {
@@ -106,8 +106,8 @@ class ResourceDataOperationRequestUtilsSpec extends MessagingBaseSpec {
assert KafkaHeaders.getParsedKafkaHeader(consumerRecordOutHeaders, 'ce_correlationid') == 'request-id'
assert KafkaHeaders.getParsedKafkaHeader(consumerRecordOutHeaders, 'ce_destination') == clientTopic
and: 'map consumer record to expected event type'
- def dataOperationResponseEvent = CloudEventUtils.mapData(consumerRecordOut.value(),
- PojoCloudEventDataMapper.from(objectMapper, DataOperationEvent.class)).getValue()
+ def dataOperationResponseEvent =
+ toTargetEvent(consumerRecordOut.value(), DataOperationEvent.class)
and: 'data operation response event response size is 3'
dataOperationResponseEvent.data.responses.size() == 3
and: 'verify published response data as json string'