summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorhalil.cakal <halil.cakal@est.tech>2023-06-14 08:10:15 +0100
committerhalil.cakal <halil.cakal@est.tech>2023-06-26 15:19:37 +0100
commit5c81c93cb8c365a549c29f0f0e09d63ae1934062 (patch)
treea21f46dd7871a3a17a4e3ccdc1fb9a0c5e40c70f
parent48110553bc0e7099e5b675bb504cc73937c33d19 (diff)
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 <halil.cakal@est.tech>
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/ResponseTimeoutTask.java4
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventForwarder.java46
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseConsumer.java30
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseOutcome.java20
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/utils/DataNodeHelper.java11
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avc/SubscriptionEventMapperSpec.groovy3
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventForwarderSpec.groovy46
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseConsumerSpec.groovy20
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionEventResponseOutcomeSpec.groovy41
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avcsubscription/SubscriptionOutcomeMapperSpec.groovy1
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/DataNodeHelperSpec.groovy13
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<String> 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<SubscriptionEvent> eventsPublisher;
private final IMap<String, Set<String>> 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<String, Map<String, Map<String, String>>> dmiPropertiesPerCmHandleIdPerServiceName
= DmiServiceNameOrganizer.getDmiPropertiesPerCmHandleIdPerServiceName(yangModelCmHandles);
+ findDmisAndRespond(subscriptionEvent, eventHeaders, cmHandleTargetsAsStrings,
+ dmiPropertiesPerCmHandleIdPerServiceName);
+ }
+
+ private void findDmisAndRespond(final SubscriptionEvent subscriptionEvent, final Headers eventHeaders,
+ final List<String> cmHandleTargetsAsStrings,
+ final Map<String, Map<String, Map<String, String>>>
+ dmiPropertiesPerCmHandleIdPerServiceName) {
+ final List<String> cmHandlesThatExistsInDb = dmiPropertiesPerCmHandleIdPerServiceName.entrySet().stream()
+ .map(Map.Entry::getValue).map(Map::keySet).flatMap(Set::stream).collect(Collectors.toList());
+
+ final List<String> targetCmHandlesDoesNotExistInDb = new ArrayList<>(cmHandleTargetsAsStrings);
+ targetCmHandlesDoesNotExistInDb.removeAll(cmHandlesThatExistsInDb);
+
final Set<String> 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<String> targetCmHandlesDoesNotExistInDb) {
+ final YangModelSubscriptionEvent yangModelSubscriptionEvent =
+ subscriptionEventMapper.toYangModelSubscriptionEvent(subscriptionEvent);
+ yangModelSubscriptionEvent.getPredicates()
+ .setTargetCmHandles(findRejectedCmHandles(targetCmHandlesDoesNotExistInDb,
+ yangModelSubscriptionEvent));
+ subscriptionPersistence.saveSubscriptionEvent(yangModelSubscriptionEvent);
+ }
+
+ private static List<YangModelSubscriptionEvent.TargetCmHandle> findRejectedCmHandles(
+ final List<String> 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<String> 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<DataNode> dataNodeSubscription = subscriptionPersistence.getCmHandlesForSubscriptionEvent(
+ clientId, subscriptionName);
+ final Map<String, SubscriptionStatus> 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<DataNode> dataNodes = subscriptionPersistence.getDataNodesForSubscriptionEvent();
+ private SubscriptionEventOutcome generateResponse(final String subscriptionClientId,
+ final String subscriptionName) {
+ final Collection<DataNode> dataNodes =
+ subscriptionPersistence.getCmHandlesForSubscriptionEvent(subscriptionClientId, subscriptionName);
final List<Map<String, Serializable>> dataNodeLeaves = DataNodeHelper.getDataNodeLeaves(dataNodes);
final List<Collection<Serializable>> cmHandleIdToStatus =
DataNodeHelper.getCmHandleIdToStatus(dataNodeLeaves);
+ final Map<String, SubscriptionStatus> cmHandleIdToStatusMap =
+ DataNodeHelper.getCmHandleIdToStatusMap(cmHandleIdToStatus);
return formSubscriptionOutcomeMessage(cmHandleIdToStatus, subscriptionClientId, subscriptionName,
- isFullOutcomeResponse);
+ isFullOutcomeResponse(cmHandleIdToStatusMap));
}
+ private boolean isFullOutcomeResponse(final Map<String, SubscriptionStatus> cmHandleIdToStatusMap) {
+ return !cmHandleIdToStatusMap.values().contains(SubscriptionStatus.PENDING);
+ }
private SubscriptionEventOutcome formSubscriptionOutcomeMessage(
final List<Collection<Serializable>> 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<String, SubscriptionStatus> getCmHandleIdToStatusMapFromDataNodes(
+ final Collection<DataNode> 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<String, Set<String>> mockForwardedSubscriptionEventCache = Mock(IMap<String, Set<String>>)
@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<String, SubscriptionEvent>('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<String, SubscriptionEvent>('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
+ }
}