From 5c81c93cb8c365a549c29f0f0e09d63ae1934062 Mon Sep 17 00:00:00 2001 From: "halil.cakal" Date: Wed, 14 Jun 2023 08:10:15 +0100 Subject: [PATCH] Streamline outcome response for subscription creation - Pivot to outcome response generated from db for the corresponding subscription - Fine tuning to wait till the cutoff time for all cm handles that subject to subscription - Update cm handles as rejected if dmi plugin or cm handles does not exist in db - Add more test for missed branches in mappers and response outcome Issue-ID: CPS-1741 Change-Id: I82ab6958e673059977abc7708974d5a2cb13e339 Signed-off-by: halil.cakal --- .../avcsubscription/ResponseTimeoutTask.java | 4 +- .../SubscriptionEventForwarder.java | 46 +++++++++++++++++++++- .../SubscriptionEventResponseConsumer.java | 30 +++++++++----- .../SubscriptionEventResponseOutcome.java | 20 ++++++---- .../cps/ncmp/api/impl/utils/DataNodeHelper.java | 11 ++++++ .../events/avc/SubscriptionEventMapperSpec.groovy | 3 +- .../SubscriptionEventForwarderSpec.groovy | 46 ++++++++++++++-------- .../SubscriptionEventResponseConsumerSpec.groovy | 20 ++++++---- .../SubscriptionEventResponseOutcomeSpec.groovy | 41 +++++++++++++++---- .../SubscriptionOutcomeMapperSpec.groovy | 1 + .../ncmp/api/impl/utils/DataNodeHelperSpec.groovy | 13 ++++++ 11 files changed, 182 insertions(+), 53 deletions(-) diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/ResponseTimeoutTask.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/ResponseTimeoutTask.java index a81f8fd73..c178700ee 100644 --- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/ResponseTimeoutTask.java +++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/ResponseTimeoutTask.java @@ -49,9 +49,7 @@ public class ResponseTimeoutTask implements Runnable { private void generateAndSendResponse() { final String subscriptionEventId = subscriptionClientId + subscriptionName; if (forwardedSubscriptionEventCache.containsKey(subscriptionEventId)) { - final Set dmiNames = forwardedSubscriptionEventCache.get(subscriptionEventId); - subscriptionEventResponseOutcome.sendResponse(subscriptionClientId, subscriptionName, - dmiNames.isEmpty()); + subscriptionEventResponseOutcome.sendResponse(subscriptionClientId, subscriptionName); forwardedSubscriptionEventCache.remove(subscriptionEventId); } } diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventForwarder.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventForwarder.java index 9e363f3cd..1d87a057a 100644 --- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventForwarder.java +++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventForwarder.java @@ -21,6 +21,7 @@ package org.onap.cps.ncmp.api.impl.events.avcsubscription; import com.hazelcast.map.IMap; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashSet; @@ -37,8 +38,11 @@ import lombok.extern.slf4j.Slf4j; import org.apache.kafka.common.header.Headers; import org.onap.cps.ncmp.api.impl.config.embeddedcache.ForwardedSubscriptionEventCacheConfig; import org.onap.cps.ncmp.api.impl.events.EventsPublisher; +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.DmiServiceNameOrganizer; import org.onap.cps.ncmp.api.impl.yangmodels.YangModelCmHandle; +import org.onap.cps.ncmp.api.impl.yangmodels.YangModelSubscriptionEvent; import org.onap.cps.ncmp.api.inventory.InventoryPersistence; import org.onap.cps.ncmp.event.model.SubscriptionEvent; import org.onap.cps.spi.exceptions.OperationNotYetSupportedException; @@ -55,6 +59,8 @@ public class SubscriptionEventForwarder { private final EventsPublisher eventsPublisher; private final IMap> forwardedSubscriptionEventCache; private final SubscriptionEventResponseOutcome subscriptionEventResponseOutcome; + private final SubscriptionEventMapper subscriptionEventMapper; + private final SubscriptionPersistence subscriptionPersistence; private final ScheduledExecutorService executorService = Executors.newSingleThreadScheduledExecutor(); @Value("${app.ncmp.avc.subscription-forward-topic-prefix}") private String dmiAvcSubscriptionTopicPrefix; @@ -83,11 +89,29 @@ public class SubscriptionEventForwarder { final Map>> dmiPropertiesPerCmHandleIdPerServiceName = DmiServiceNameOrganizer.getDmiPropertiesPerCmHandleIdPerServiceName(yangModelCmHandles); + findDmisAndRespond(subscriptionEvent, eventHeaders, cmHandleTargetsAsStrings, + dmiPropertiesPerCmHandleIdPerServiceName); + } + + private void findDmisAndRespond(final SubscriptionEvent subscriptionEvent, final Headers eventHeaders, + final List cmHandleTargetsAsStrings, + final Map>> + dmiPropertiesPerCmHandleIdPerServiceName) { + final List cmHandlesThatExistsInDb = dmiPropertiesPerCmHandleIdPerServiceName.entrySet().stream() + .map(Map.Entry::getValue).map(Map::keySet).flatMap(Set::stream).collect(Collectors.toList()); + + final List targetCmHandlesDoesNotExistInDb = new ArrayList<>(cmHandleTargetsAsStrings); + targetCmHandlesDoesNotExistInDb.removeAll(cmHandlesThatExistsInDb); + final Set dmisToRespond = new HashSet<>(dmiPropertiesPerCmHandleIdPerServiceName.keySet()); + + if (dmisToRespond.isEmpty() || !targetCmHandlesDoesNotExistInDb.isEmpty()) { + updatesCmHandlesToRejectedAndPersistSubscriptionEvent(subscriptionEvent, targetCmHandlesDoesNotExistInDb); + } if (dmisToRespond.isEmpty()) { final String clientID = subscriptionEvent.getEvent().getSubscription().getClientID(); final String subscriptionName = subscriptionEvent.getEvent().getSubscription().getName(); - subscriptionEventResponseOutcome.sendResponse(clientID, subscriptionName, true); + subscriptionEventResponseOutcome.sendResponse(clientID, subscriptionName); } else { startResponseTimeout(subscriptionEvent, dmisToRespond); forwardEventToDmis(dmiPropertiesPerCmHandleIdPerServiceName, subscriptionEvent, eventHeaders); @@ -130,4 +154,24 @@ public class SubscriptionEventForwarder { + "-" + dmiName; } + + private void updatesCmHandlesToRejectedAndPersistSubscriptionEvent( + final SubscriptionEvent subscriptionEvent, + final List targetCmHandlesDoesNotExistInDb) { + final YangModelSubscriptionEvent yangModelSubscriptionEvent = + subscriptionEventMapper.toYangModelSubscriptionEvent(subscriptionEvent); + yangModelSubscriptionEvent.getPredicates() + .setTargetCmHandles(findRejectedCmHandles(targetCmHandlesDoesNotExistInDb, + yangModelSubscriptionEvent)); + subscriptionPersistence.saveSubscriptionEvent(yangModelSubscriptionEvent); + } + + private static List findRejectedCmHandles( + final List targetCmHandlesDoesNotExistInDb, + final YangModelSubscriptionEvent yangModelSubscriptionEvent) { + return yangModelSubscriptionEvent.getPredicates().getTargetCmHandles().stream() + .filter(targetCmHandle -> targetCmHandlesDoesNotExistInDb.contains(targetCmHandle.getCmHandleId())) + .map(target -> new YangModelSubscriptionEvent.TargetCmHandle(target.getCmHandleId(), + SubscriptionStatus.REJECTED)).collect(Collectors.toList()); + } } diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseConsumer.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseConsumer.java index a1860a613..20df706c0 100644 --- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseConsumer.java +++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseConsumer.java @@ -21,6 +21,8 @@ package org.onap.cps.ncmp.api.impl.events.avcsubscription; import com.hazelcast.map.IMap; +import java.util.Collection; +import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; import lombok.RequiredArgsConstructor; @@ -28,8 +30,11 @@ import lombok.extern.slf4j.Slf4j; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.onap.cps.ncmp.api.impl.config.embeddedcache.ForwardedSubscriptionEventCacheConfig; 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.yangmodels.YangModelSubscriptionEvent; import org.onap.cps.ncmp.api.models.SubscriptionEventResponse; +import org.onap.cps.spi.model.DataNode; import org.springframework.beans.factory.annotation.Value; import org.springframework.kafka.annotation.KafkaListener; import org.springframework.stereotype.Component; @@ -64,28 +69,35 @@ public class SubscriptionEventResponseConsumer { log.info("subscription event response of clientId: {} is received.", clientId); final String subscriptionName = subscriptionEventResponse.getSubscriptionName(); final String subscriptionEventId = clientId + subscriptionName; - boolean isFullOutcomeResponse = false; + boolean createOutcomeResponse = false; if (forwardedSubscriptionEventCache.containsKey(subscriptionEventId)) { final Set dmiNames = forwardedSubscriptionEventCache.get(subscriptionEventId); dmiNames.remove(subscriptionEventResponse.getDmiName()); forwardedSubscriptionEventCache.put(subscriptionEventId, dmiNames, ForwardedSubscriptionEventCacheConfig.SUBSCRIPTION_FORWARD_STARTED_TTL_SECS, TimeUnit.SECONDS); - isFullOutcomeResponse = forwardedSubscriptionEventCache.get(subscriptionEventId).isEmpty(); - - if (isFullOutcomeResponse) { - forwardedSubscriptionEventCache.remove(subscriptionEventId); - } + createOutcomeResponse = forwardedSubscriptionEventCache.get(subscriptionEventId).isEmpty(); } if (subscriptionModelLoaderEnabled) { updateSubscriptionEvent(subscriptionEventResponse); } - if (isFullOutcomeResponse && notificationFeatureEnabled) { - subscriptionEventResponseOutcome.sendResponse(clientId, subscriptionName, - isFullOutcomeResponse); + if (createOutcomeResponse + && notificationFeatureEnabled + && hasNoPendingCmHandles(clientId, subscriptionName)) { + subscriptionEventResponseOutcome.sendResponse(clientId, subscriptionName); + forwardedSubscriptionEventCache.remove(subscriptionEventId); } } + private boolean hasNoPendingCmHandles(final String clientId, final String subscriptionName) { + final Collection dataNodeSubscription = subscriptionPersistence.getCmHandlesForSubscriptionEvent( + clientId, subscriptionName); + final Map cmHandleIdToStatusMap = + DataNodeHelper.getCmHandleIdToStatusMapFromDataNodes( + dataNodeSubscription); + return !cmHandleIdToStatusMap.values().contains(SubscriptionStatus.PENDING); + } + private void updateSubscriptionEvent(final SubscriptionEventResponse subscriptionEventResponse) { final YangModelSubscriptionEvent yangModelSubscriptionEvent = subscriptionEventResponseMapper diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseOutcome.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseOutcome.java index 1bfc4ab28..8fdff1794 100644 --- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseOutcome.java +++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseOutcome.java @@ -57,28 +57,32 @@ public class SubscriptionEventResponseOutcome { * * @param subscriptionClientId client id of the subscription. * @param subscriptionName name of the subscription. - * @param isFullOutcomeResponse the flag to decide on complete or partial response to be generated. */ - public void sendResponse(final String subscriptionClientId, final String subscriptionName, - final boolean isFullOutcomeResponse) { + public void sendResponse(final String subscriptionClientId, final String subscriptionName) { final SubscriptionEventOutcome subscriptionEventOutcome = generateResponse( - subscriptionClientId, subscriptionName, isFullOutcomeResponse); + subscriptionClientId, subscriptionName); final Headers headers = new RecordHeaders(); final String subscriptionEventId = subscriptionClientId + subscriptionName; outcomeEventsPublisher.publishEvent(subscriptionOutcomeEventTopic, subscriptionEventId, headers, subscriptionEventOutcome); } - private SubscriptionEventOutcome generateResponse(final String subscriptionClientId, final String subscriptionName, - final boolean isFullOutcomeResponse) { - final Collection dataNodes = subscriptionPersistence.getDataNodesForSubscriptionEvent(); + private SubscriptionEventOutcome generateResponse(final String subscriptionClientId, + final String subscriptionName) { + final Collection dataNodes = + subscriptionPersistence.getCmHandlesForSubscriptionEvent(subscriptionClientId, subscriptionName); final List> dataNodeLeaves = DataNodeHelper.getDataNodeLeaves(dataNodes); final List> cmHandleIdToStatus = DataNodeHelper.getCmHandleIdToStatus(dataNodeLeaves); + final Map cmHandleIdToStatusMap = + DataNodeHelper.getCmHandleIdToStatusMap(cmHandleIdToStatus); return formSubscriptionOutcomeMessage(cmHandleIdToStatus, subscriptionClientId, subscriptionName, - isFullOutcomeResponse); + isFullOutcomeResponse(cmHandleIdToStatusMap)); } + private boolean isFullOutcomeResponse(final Map cmHandleIdToStatusMap) { + return !cmHandleIdToStatusMap.values().contains(SubscriptionStatus.PENDING); + } private SubscriptionEventOutcome formSubscriptionOutcomeMessage( final List> cmHandleIdToStatus, final String subscriptionClientId, diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/utils/DataNodeHelper.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/utils/DataNodeHelper.java index 8d44592ae..f42a378fc 100644 --- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/utils/DataNodeHelper.java +++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/utils/DataNodeHelper.java @@ -93,4 +93,15 @@ public class DataNodeHelper { } return resultMap; } + + /** + * Extracts the mapping of cm handle id to status from data node collection. + * + * @param dataNodes as a collection + * @return cm handle id to status mapping + */ + public static Map getCmHandleIdToStatusMapFromDataNodes( + final Collection dataNodes) { + return getCmHandleIdToStatusMap(getCmHandleIdToStatus(getDataNodeLeaves(dataNodes))); + } } diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avc/SubscriptionEventMapperSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avc/SubscriptionEventMapperSpec.groovy index f2ff1f7b2..6d02ac719 100644 --- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avc/SubscriptionEventMapperSpec.groovy +++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avc/SubscriptionEventMapperSpec.groovy @@ -60,7 +60,7 @@ class SubscriptionEventMapperSpec extends Specification { assert result.topic == null } - def 'Map null subscription event to yang model subscription event where #scenario'() { + def 'Map empty subscription event to yang model subscription event'() { given: 'a new Subscription Event with no data' def testEventToMap = new SubscriptionEvent() when: 'the event is mapped to a yang model subscription' @@ -76,5 +76,4 @@ class SubscriptionEventMapperSpec extends Specification { and: 'the topic is null' assert result.topic == null } - } \ No newline at end of file diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventForwarderSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventForwarderSpec.groovy index a9eaaee91..41597edec 100644 --- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventForwarderSpec.groovy +++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventForwarderSpec.groovy @@ -23,8 +23,12 @@ package org.onap.cps.ncmp.api.impl.events.avcsubscription import com.fasterxml.jackson.databind.ObjectMapper import com.hazelcast.map.IMap import org.apache.kafka.clients.consumer.ConsumerRecord +import org.mapstruct.factory.Mappers import org.onap.cps.ncmp.api.impl.events.EventsPublisher +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.yangmodels.YangModelCmHandle +import org.onap.cps.ncmp.api.impl.yangmodels.YangModelSubscriptionEvent.TargetCmHandle import org.onap.cps.ncmp.api.inventory.InventoryPersistence import org.onap.cps.ncmp.api.kafka.MessagingBaseSpec import org.onap.cps.ncmp.event.model.SubscriptionEvent @@ -52,6 +56,10 @@ class SubscriptionEventForwarderSpec extends MessagingBaseSpec { IMap> mockForwardedSubscriptionEventCache = Mock(IMap>) @SpringBean SubscriptionEventResponseOutcome mockSubscriptionEventResponseOutcome = Mock(SubscriptionEventResponseOutcome) + @SpringBean + SubscriptionPersistence mockSubscriptionPersistence = Mock(SubscriptionPersistence) + @SpringBean + SubscriptionEventMapper subscriptionEventMapper = Mappers.getMapper(SubscriptionEventMapper) @Autowired JsonObjectMapper jsonObjectMapper @@ -60,11 +68,17 @@ class SubscriptionEventForwarderSpec extends MessagingBaseSpec { def jsonData = TestUtils.getResourceFileContent('avcSubscriptionCreationEvent.json') def testEventSent = jsonObjectMapper.convertJsonString(jsonData, SubscriptionEvent.class) def consumerRecord = new ConsumerRecord('topic-name', 0, 0, 'event-key', testEventSent) + and: 'the some of the cm handles will be accepted and some of rejected' + def cmHandlesToBeSavedInDb = [new TargetCmHandle('CMHandle1', SubscriptionStatus.ACCEPTED), + new TargetCmHandle('CMHandle2',SubscriptionStatus.ACCEPTED), + new TargetCmHandle('CMHandle3',SubscriptionStatus.REJECTED)] + and: 'a yang model subscription event will be saved into the db' + def yangModelSubscriptionEventWithAcceptedAndRejectedCmHandles = subscriptionEventMapper.toYangModelSubscriptionEvent(testEventSent) + yangModelSubscriptionEventWithAcceptedAndRejectedCmHandles.getPredicates().setTargetCmHandles(cmHandlesToBeSavedInDb) 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(3, 2,"shape","triangle") + createYangModelCmHandleWithDmiProperty(2, 1,"shape","square") ] and: 'the thread creation delay is reduced to 2 seconds for testing' objectUnderTest.dmiResponseTimeoutInMs = 2000 @@ -75,7 +89,7 @@ class SubscriptionEventForwarderSpec extends MessagingBaseSpec { then: 'An asynchronous call is made to the blocking variable' block.get() then: 'the event is added to the forwarded subscription event cache' - 1 * mockForwardedSubscriptionEventCache.put("SCO-9989752cm-subscription-001", ["DMIName1", "DMIName2"] as Set, 600, TimeUnit.SECONDS) + 1 * mockForwardedSubscriptionEventCache.put("SCO-9989752cm-subscription-001", ["DMIName1"] as Set, 600, TimeUnit.SECONDS) and: 'the event is forwarded twice with the CMHandle private properties and provides a valid listenable future' 1 * mockSubscriptionEventPublisher.publishEvent("ncmp-dmi-cm-avc-subscription-DMIName1", "SCO-9989752-cm-subscription-001-DMIName1", consumerRecord.headers(), subscriptionEvent -> { @@ -84,22 +98,13 @@ class SubscriptionEventForwarderSpec extends MessagingBaseSpec { targets["CMHandle2"] == ["shape":"square"] } ) - 1 * mockSubscriptionEventPublisher.publishEvent("ncmp-dmi-cm-avc-subscription-DMIName2", "SCO-9989752-cm-subscription-001-DMIName2", - consumerRecord.headers(), subscriptionEvent -> { - Map targets = subscriptionEvent.getEvent().getPredicates().getTargets().get(0) - targets["CMHandle3"] == ["shape":"triangle"] - } - ) + and: 'the persistence service save the yang model subscription event' + 1 * mockSubscriptionPersistence.saveSubscriptionEvent(yangModelSubscriptionEventWithAcceptedAndRejectedCmHandles) and: 'a separate thread has been created where the map is polled' 1 * mockForwardedSubscriptionEventCache.containsKey("SCO-9989752cm-subscription-001") >> true - 1 * mockForwardedSubscriptionEventCache.get(_) >> DMINamesInMap 1 * mockSubscriptionEventResponseOutcome.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(_)} - where: - scenario | DMINamesInMap - 'there are dmis which have not responded' | ["DMIName1", "DMIName2"] as Set - 'all dmis have responded' | [] as Set } def 'Forward CM create subscription where target CM Handles are #scenario'() { @@ -125,6 +130,13 @@ class SubscriptionEventForwarderSpec extends MessagingBaseSpec { def jsonData = TestUtils.getResourceFileContent('avcSubscriptionCreationEvent.json') def testEventSent = jsonObjectMapper.convertJsonString(jsonData, SubscriptionEvent.class) def consumerRecord = new ConsumerRecord('topic-name', 0, 0, 'event-key', testEventSent) + and: 'the cm handles will be rejected' + def rejectedCmHandles = [new TargetCmHandle('CMHandle1', SubscriptionStatus.REJECTED), + new TargetCmHandle('CMHandle2',SubscriptionStatus.REJECTED), + new TargetCmHandle('CMHandle3',SubscriptionStatus.REJECTED)] + and: 'a yang model subscription event will be saved into the db with rejected cm handles' + def yangModelSubscriptionEventWithRejectedCmHandles = subscriptionEventMapper.toYangModelSubscriptionEvent(testEventSent) + yangModelSubscriptionEventWithRejectedCmHandles.getPredicates().setTargetCmHandles(rejectedCmHandles) and: 'the InventoryPersistence returns no private properties for the supplied CM Handles' 1 * mockInventoryPersistence.getYangModelCmHandles(["CMHandle1", "CMHandle2", "CMHandle3"]) >> [] and: 'the thread creation delay is reduced to 2 seconds for testing' @@ -135,7 +147,7 @@ class SubscriptionEventForwarderSpec extends MessagingBaseSpec { objectUnderTest.forwardCreateSubscriptionEvent(testEventSent, consumerRecord.headers()) then: 'the event is not added to the forwarded subscription event cache' 0 * mockForwardedSubscriptionEventCache.put("SCO-9989752cm-subscription-001", ["DMIName1", "DMIName2"] as Set) - and: 'the event is forwarded twice with the CMHandle private properties and provides a valid listenable future' + and: 'the event is not being forwarded with the CMHandle private properties and does not provides a valid listenable future' 0 * mockSubscriptionEventPublisher.publishEvent("ncmp-dmi-cm-avc-subscription-DMIName1", "SCO-9989752-cm-subscription-001-DMIName1", consumerRecord.headers(),subscriptionEvent -> { Map targets = subscriptionEvent.getEvent().getPredicates().getTargets().get(0) @@ -154,8 +166,10 @@ class SubscriptionEventForwarderSpec extends MessagingBaseSpec { 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(_)} + and: 'the persistence service save target cm handles of the yang model subscription event as rejected ' + 1 * mockSubscriptionPersistence.saveSubscriptionEvent(yangModelSubscriptionEventWithRejectedCmHandles) and: 'subscription outcome has been sent' - 1 * mockSubscriptionEventResponseOutcome.sendResponse('SCO-9989752', 'cm-subscription-001', true) + 1 * mockSubscriptionEventResponseOutcome.sendResponse('SCO-9989752', 'cm-subscription-001') } static def createYangModelCmHandleWithDmiProperty(id, dmiId,propertyName, propertyValue) { diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseConsumerSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseConsumerSpec.groovy index 26bb7e78e..5355dd8b9 100644 --- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseConsumerSpec.groovy +++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseConsumerSpec.groovy @@ -26,6 +26,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord import org.onap.cps.ncmp.api.impl.subscriptions.SubscriptionPersistenceImpl import org.onap.cps.ncmp.api.kafka.MessagingBaseSpec import org.onap.cps.ncmp.api.models.SubscriptionEventResponse +import org.onap.cps.spi.model.DataNodeBuilder import org.onap.cps.utils.JsonObjectMapper import org.springframework.boot.test.context.SpringBootTest @@ -50,6 +51,13 @@ class SubscriptionEventResponseConsumerSpec extends MessagingBaseSpec { objectUnderTest.notificationFeatureEnabled = isNotificationFeatureEnabled and: 'subscription model loader is enabled' objectUnderTest.subscriptionModelLoaderEnabled = true + and: 'a data node exist in db' + def leaves1 = [status:'ACCEPTED', cmHandleId:'cmhandle1'] as Map + def dataNode = new DataNodeBuilder().withDataspace('NCMP-Admin') + .withAnchor('AVC-Subscriptions').withXpath('/subscription-registry/subscription') + .withLeaves(leaves1).build() + and: 'subscription persistence service returns data node' + mockSubscriptionPersistence.getCmHandlesForSubscriptionEvent(*_) >> [dataNode] when: 'the valid event is consumed' objectUnderTest.consumeSubscriptionEventResponse(consumerRecord) then: 'the forwarded subscription event cache returns only the received dmiName existing for the subscription create event' @@ -58,15 +66,13 @@ class SubscriptionEventResponseConsumerSpec extends MessagingBaseSpec { and: 'the forwarded subscription event cache returns an empty Map when the dmiName has been removed' 1 * mockForwardedSubscriptionEventCache.get('some-client-idsome-subscription-name') >> ([] as Set) and: 'the subscription event is removed from the map' - 1 * mockForwardedSubscriptionEventCache.remove('some-client-idsome-subscription-name') + numberOfExpectedCallToRemove * mockForwardedSubscriptionEventCache.remove('some-client-idsome-subscription-name') and: 'a response outcome has been created' - numberOfExpectedCallToSendResponse * mockSubscriptionEventResponseOutcome.sendResponse('some-client-id', 'some-subscription-name', isFullOutcomeResponse) + numberOfExpectedCallToSendResponse * mockSubscriptionEventResponseOutcome.sendResponse('some-client-id', 'some-subscription-name') where: 'the following values are used' - scenario | isNotificationFeatureEnabled | isFullOutcomeResponse || numberOfExpectedCallToSendResponse - 'Response sent' | true | true || 1 - 'Response not sent' | true | false || 0 - 'Response not sent' | false | true || 0 - 'Response not sent' | false | false || 0 + scenario | isNotificationFeatureEnabled || numberOfExpectedCallToRemove || numberOfExpectedCallToSendResponse + 'Response sent' | true || 1 || 1 + 'Response not sent' | false || 0 || 0 } def 'Consume Subscription Event Response where another DMI has not yet responded'() { diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseOutcomeSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseOutcomeSpec.groovy index 3570a9e36..bb0e7b73a 100644 --- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseOutcomeSpec.groovy +++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseOutcomeSpec.groovy @@ -21,9 +21,11 @@ package org.onap.cps.ncmp.api.impl.events.avcsubscription import com.fasterxml.jackson.databind.ObjectMapper +import org.apache.kafka.common.header.internals.RecordHeaders import org.mapstruct.factory.Mappers import org.onap.cps.ncmp.api.impl.events.EventsPublisher 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.DataNodeBaseSpec import org.onap.cps.ncmp.events.avc.subscription.v1.SubscriptionEventOutcome import org.onap.cps.ncmp.utils.TestUtils @@ -48,22 +50,47 @@ class SubscriptionEventResponseOutcomeSpec extends DataNodeBaseSpec { @Autowired JsonObjectMapper jsonObjectMapper + def 'Send response to the client apps successfully'() { + given: 'a subscription client id and subscription name' + def clientId = 'some-client-id' + def subscriptionName = 'some-subscription-name' + and: 'the persistence service return a data node' + mockSubscriptionPersistence.getCmHandlesForSubscriptionEvent(*_) >> [dataNode4] + and: 'the response is being generated from the db' + def eventOutcome = objectUnderTest.generateResponse(clientId, subscriptionName) + when: 'the response is being sent' + objectUnderTest.sendResponse(clientId, subscriptionName) + then: 'the publisher publish the response with expected parameters' + 1 * mockSubscriptionEventOutcomePublisher.publishEvent('cm-avc-subscription-response', clientId + subscriptionName, new RecordHeaders(), eventOutcome) + } + + def 'Check cm handle id to status map to see if it is a full outcome response'() { + when: 'is full outcome response evaluated' + def response = objectUnderTest.isFullOutcomeResponse(cmHandleIdToStatusMap) + then: 'the result will be as expected' + response == expectedResult + where: 'the following values are used' + scenario | cmHandleIdToStatusMap || expectedResult + 'The map contains PENDING status' | ['CMHandle1': SubscriptionStatus.PENDING] as Map || false + 'The map contains ACCEPTED status' | ['CMHandle1': SubscriptionStatus.ACCEPTED] as Map || true + 'The map contains REJECTED status' | ['CMHandle1': SubscriptionStatus.REJECTED] as Map || true + 'The map contains PENDING and ACCEPTED statuses' | ['CMHandle1': SubscriptionStatus.PENDING,'CMHandle2': SubscriptionStatus.ACCEPTED] as Map || false + 'The map contains REJECTED and ACCEPTED statuses' | ['CMHandle1': SubscriptionStatus.REJECTED,'CMHandle2': SubscriptionStatus.ACCEPTED] as Map || true + 'The map contains PENDING and REJECTED statuses' | ['CMHandle1': SubscriptionStatus.PENDING,'CMHandle2': SubscriptionStatus.REJECTED] as Map || false + } + def 'Generate response via fetching data nodes from database.'() { given: 'a db call to get data nodes for subscription event' - 1 * mockSubscriptionPersistence.getDataNodesForSubscriptionEvent() >> [dataNode4] + 1 * mockSubscriptionPersistence.getCmHandlesForSubscriptionEvent(*_) >> [dataNode4] when: 'a response is generated' - def result = objectUnderTest.generateResponse('some-client-id', 'some-subscription-name', isFullOutcomeResponse) + def result = objectUnderTest.generateResponse('some-client-id', 'some-subscription-name') then: 'the result will have the same values as same as in dataNode4' - result.eventType == expectedEventType + result.eventType == SubscriptionEventOutcome.EventType.PARTIAL_OUTCOME result.getEvent().getSubscription().getClientID() == 'some-client-id' result.getEvent().getSubscription().getName() == 'some-subscription-name' result.getEvent().getPredicates().getPendingTargets() == ['CMHandle3'] result.getEvent().getPredicates().getRejectedTargets() == ['CMHandle1'] result.getEvent().getPredicates().getAcceptedTargets() == ['CMHandle2'] - where: 'the following values are used' - scenario | isFullOutcomeResponse || expectedEventType - 'is full outcome' | true || SubscriptionEventOutcome.EventType.COMPLETE_OUTCOME - 'is partial outcome' | false || SubscriptionEventOutcome.EventType.PARTIAL_OUTCOME } def 'Form subscription outcome message with a list of cm handle id to status mapping'() { diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionOutcomeMapperSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionOutcomeMapperSpec.groovy index b05e983c0..7f1a62829 100644 --- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionOutcomeMapperSpec.groovy +++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionOutcomeMapperSpec.groovy @@ -57,4 +57,5 @@ class SubscriptionOutcomeMapperSpec extends Specification { 'is full outcome' || SubscriptionEventOutcome.EventType.COMPLETE_OUTCOME 'is partial outcome' || SubscriptionEventOutcome.EventType.PARTIAL_OUTCOME } + } \ No newline at end of file diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/DataNodeHelperSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/DataNodeHelperSpec.groovy index ee726a908..819f1fa08 100644 --- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/DataNodeHelperSpec.groovy +++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/DataNodeHelperSpec.groovy @@ -70,4 +70,17 @@ class DataNodeHelperSpec extends DataNodeBaseSpec { result.keySet() == ['CMHandle3', 'CMHandle2', 'CMHandle1'] as Set result.values() as List == [SubscriptionStatus.PENDING, SubscriptionStatus.ACCEPTED, SubscriptionStatus.REJECTED] } + + + def 'Get cm handle id to status map as expected from a nested data node.'() { + given: 'a nested data node' + def dataNode = new DataNodeBuilder().withDataspace('NCMP-Admin') + .withAnchor('AVC-Subscriptions').withXpath('/subscription-registry/subscription') + .withLeaves([clientID:'SCO-9989752', isTagged:false, subscriptionName:'cm-subscription-001']) + .withChildDataNodes([dataNode4]).build() + when:'cm handle id to status is being extracted' + def result = DataNodeHelper.getCmHandleIdToStatusMapFromDataNodes([dataNode]); + then: 'the keys are retrieved as expected' + result.keySet() == ['CMHandle3','CMHandle2','CMHandle1'] as Set + } } -- 2.16.6