summaryrefslogtreecommitdiffstats
path: root/cps-ncmp-service
diff options
context:
space:
mode:
authorToine Siebelink <toine.siebelink@est.tech>2023-11-29 08:34:03 +0000
committerGerrit Code Review <gerrit@onap.org>2023-11-29 08:34:03 +0000
commit626547adfd590e5a7cfe98bfaf68fcefb00be9af (patch)
tree69dcc0b3de4ee0815eb978f6bdb57464e068c9a0 /cps-ncmp-service
parent0fafbf504b52c78d1d09b71234957f7df0696fa1 (diff)
parent2e71615fa27cefd1aee3fb85d04907f3d95d5d14 (diff)
Merge "Publish trust level notification event"
Diffstat (limited to 'cps-ncmp-service')
-rwxr-xr-xcps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImpl.java54
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/client/DmiRestClient.java10
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avc/ncmptoclient/AvcEventPublisher.java82
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandler.java7
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandlerImpl.java10
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/trustlevel/DeviceHeartbeatConsumer.java19
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/trustlevel/TrustLevelManager.java90
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/trustlevel/dmiavailability/DmiPluginWatchDog.java36
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/models/CmHandleRegistrationResponse.java2
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplRegistrationSpec.groovy25
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplSpec.groovy14
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/client/DmiRestClientSpec.groovy6
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avc/ncmptoclient/AvcEventPublisherSpec.groovy63
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandlerImplSpec.groovy9
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/DeviceHeartbeatConsumerSpec.groovy29
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/TrustLevelManagerSpec.groovy73
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/TrustLevelSpec.groovy (renamed from cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/TrustLevelTest.groovy)2
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/dmiavailability/DmiPluginWatchDogSpec.groovy32
18 files changed, 445 insertions, 118 deletions
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImpl.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImpl.java
index f0336900f..1afe5c711 100755
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImpl.java
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImpl.java
@@ -64,6 +64,7 @@ import org.onap.cps.ncmp.api.impl.inventory.sync.ModuleOperationsUtils;
import org.onap.cps.ncmp.api.impl.operations.DmiDataOperations;
import org.onap.cps.ncmp.api.impl.operations.OperationType;
import org.onap.cps.ncmp.api.impl.trustlevel.TrustLevel;
+import org.onap.cps.ncmp.api.impl.trustlevel.TrustLevelManager;
import org.onap.cps.ncmp.api.impl.utils.CmHandleQueryConditions;
import org.onap.cps.ncmp.api.impl.utils.InventoryQueryConditions;
import org.onap.cps.ncmp.api.impl.utils.YangDataConverter;
@@ -101,8 +102,8 @@ public class NetworkCmProxyDataServiceImpl implements NetworkCmProxyDataService
private final LcmEventsCmHandleStateHandler lcmEventsCmHandleStateHandler;
private final CpsDataService cpsDataService;
private final IMap<String, Object> moduleSyncStartedOnCmHandles;
- private final Map<String, TrustLevel> trustLevelPerCmHandle;
private final Map<String, TrustLevel> trustLevelPerDmiPlugin;
+ private final TrustLevelManager trustLevelManager;
@Override
public DmiPluginRegistrationResponse updateDmiRegistrationAndSyncModule(
@@ -120,7 +121,6 @@ public class NetworkCmProxyDataServiceImpl implements NetworkCmProxyDataService
if (!dmiPluginRegistration.getCreatedCmHandles().isEmpty()) {
dmiPluginRegistrationResponse.setCreatedCmHandles(
parseAndProcessCreatedCmHandlesInRegistration(dmiPluginRegistration));
- populateTrustLevelPerCmHandleCache(dmiPluginRegistration);
}
if (!dmiPluginRegistration.getUpdatedCmHandles().isEmpty()) {
dmiPluginRegistrationResponse.setUpdatedCmHandles(
@@ -318,17 +318,21 @@ public class NetworkCmProxyDataServiceImpl implements NetworkCmProxyDataService
*/
public List<CmHandleRegistrationResponse> parseAndProcessCreatedCmHandlesInRegistration(
final DmiPluginRegistration dmiPluginRegistration) {
- final Map<YangModelCmHandle, CmHandleState> cmHandleStatePerCmHandle = new HashMap<>();
- dmiPluginRegistration.getCreatedCmHandles().forEach(cmHandle -> {
- final YangModelCmHandle yangModelCmHandle = YangModelCmHandle.toYangModelCmHandle(
- dmiPluginRegistration.getDmiPlugin(),
- dmiPluginRegistration.getDmiDataPlugin(),
- dmiPluginRegistration.getDmiModelPlugin(),
- cmHandle,
- cmHandle.getModuleSetTag());
- cmHandleStatePerCmHandle.put(yangModelCmHandle, CmHandleState.ADVISED);
- });
- return registerNewCmHandles(cmHandleStatePerCmHandle);
+ final List<NcmpServiceCmHandle> cmHandlesToBeCreated = dmiPluginRegistration.getCreatedCmHandles();
+ final Map<String, TrustLevel> initialTrustLevelPerCmHandleId = new HashMap<>(cmHandlesToBeCreated.size());
+ final List<YangModelCmHandle> yangModelCmHandles = new ArrayList<>(cmHandlesToBeCreated.size());
+ cmHandlesToBeCreated
+ .forEach(cmHandle -> {
+ final YangModelCmHandle yangModelCmHandle = YangModelCmHandle.toYangModelCmHandle(
+ dmiPluginRegistration.getDmiPlugin(),
+ dmiPluginRegistration.getDmiDataPlugin(),
+ dmiPluginRegistration.getDmiModelPlugin(),
+ cmHandle,
+ cmHandle.getModuleSetTag());
+ yangModelCmHandles.add(yangModelCmHandle);
+ initialTrustLevelPerCmHandleId.put(cmHandle.getCmHandleId(), cmHandle.getRegistrationTrustLevel());
+ });
+ return registerNewCmHandles(yangModelCmHandles, initialTrustLevelPerCmHandleId);
}
protected List<CmHandleRegistrationResponse> parseAndProcessDeletedCmHandlesInRegistration(
@@ -470,11 +474,13 @@ public class NetworkCmProxyDataServiceImpl implements NetworkCmProxyDataService
}
}
- private List<CmHandleRegistrationResponse> registerNewCmHandles(final Map<YangModelCmHandle, CmHandleState>
- cmHandleStatePerCmHandle) {
- final List<String> cmHandleIds = getCmHandleIds(cmHandleStatePerCmHandle);
+ private List<CmHandleRegistrationResponse> registerNewCmHandles(final List<YangModelCmHandle> yangModelCmHandles,
+ final Map<String, TrustLevel>
+ initialTrustLevelPerCmHandleId) {
+ final Set<String> cmHandleIds = initialTrustLevelPerCmHandleId.keySet();
try {
- lcmEventsCmHandleStateHandler.updateCmHandleStateBatch(cmHandleStatePerCmHandle);
+ lcmEventsCmHandleStateHandler.initiateStateAdvised(yangModelCmHandles);
+ trustLevelManager.handleInitialRegistrationOfTrustLevels(initialTrustLevelPerCmHandleId);
return CmHandleRegistrationResponse.createSuccessResponses(cmHandleIds);
} catch (final AlreadyDefinedException alreadyDefinedException) {
return CmHandleRegistrationResponse.createFailureResponses(
@@ -512,20 +518,6 @@ public class NetworkCmProxyDataServiceImpl implements NetworkCmProxyDataService
return cmHandleStatePerCmHandle.keySet().stream().map(YangModelCmHandle::getId).toList();
}
- private void populateTrustLevelPerCmHandleCache(final DmiPluginRegistration dmiPluginRegistration) {
- for (final NcmpServiceCmHandle cmHandle: dmiPluginRegistration.getCreatedCmHandles()) {
- if (cmHandle.getRegistrationTrustLevel() == null) {
- if (trustLevelPerCmHandle.containsKey(cmHandle.getCmHandleId())) {
- log.warn("CmHandle : {}, Already exists, Initial trustLevel ignored.", cmHandle.getCmHandleId());
- } else {
- trustLevelPerCmHandle.put(cmHandle.getCmHandleId(), TrustLevel.COMPLETE);
- }
- } else {
- trustLevelPerCmHandle.put(cmHandle.getCmHandleId(), cmHandle.getRegistrationTrustLevel());
- }
- }
- }
-
private void setTrustLevelPerDmiPlugin(final DmiPluginRegistration dmiPluginRegistration) {
if (DmiPluginRegistration.isNullEmptyOrBlank(dmiPluginRegistration.getDmiDataPlugin())) {
trustLevelPerDmiPlugin.put(dmiPluginRegistration.getDmiPlugin(), TrustLevel.COMPLETE);
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/client/DmiRestClient.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/client/DmiRestClient.java
index b6eb09218..5b93eb485 100644
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/client/DmiRestClient.java
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/client/DmiRestClient.java
@@ -41,7 +41,7 @@ import org.springframework.web.client.RestTemplate;
public class DmiRestClient {
private static final String HEALTH_CHECK_URL_EXTENSION = "/actuator/health";
- private static final String EMPTY_STRING = "";
+ private static final String NOT_SPECIFIED = "";
private final RestTemplate restTemplate;
private final DmiProperties dmiProperties;
@@ -70,19 +70,19 @@ public class DmiRestClient {
* Get DMI plugin health status.
*
* @param dmiPluginBaseUrl the base URL of the dmi-plugin
- * @return plugin health status ("UP" is all OK, EMPTY_STRING in case of any exception)
+ * @return plugin health status ("UP" is all OK, "" (not-specified) in case of any exception)
*/
public String getDmiHealthStatus(final String dmiPluginBaseUrl) {
final HttpEntity<Object> httpHeaders = new HttpEntity<>(configureHttpHeaders(new HttpHeaders()));
try {
final JsonNode responseHealthStatus =
restTemplate.getForObject(dmiPluginBaseUrl + HEALTH_CHECK_URL_EXTENSION,
- JsonNode.class, httpHeaders);
- return responseHealthStatus == null ? EMPTY_STRING :
+ JsonNode.class, httpHeaders);
+ return responseHealthStatus == null ? NOT_SPECIFIED :
responseHealthStatus.get("status").asText();
} catch (final Exception e) {
log.warn("Failed to retrieve health status from {}. Error Message: {}", dmiPluginBaseUrl, e.getMessage());
- return EMPTY_STRING;
+ return NOT_SPECIFIED;
}
}
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avc/ncmptoclient/AvcEventPublisher.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avc/ncmptoclient/AvcEventPublisher.java
new file mode 100644
index 000000000..cba1f76ff
--- /dev/null
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/avc/ncmptoclient/AvcEventPublisher.java
@@ -0,0 +1,82 @@
+/*
+ * ============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.avc.ncmptoclient;
+
+import io.cloudevents.CloudEvent;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import lombok.RequiredArgsConstructor;
+import org.onap.cps.ncmp.api.impl.events.EventsPublisher;
+import org.onap.cps.ncmp.api.impl.events.NcmpCloudEventBuilder;
+import org.onap.cps.ncmp.events.avc.ncmp_to_client.Avc;
+import org.onap.cps.ncmp.events.avc.ncmp_to_client.AvcEvent;
+import org.onap.cps.ncmp.events.avc.ncmp_to_client.Data;
+import org.springframework.beans.factory.annotation.Value;
+import org.springframework.stereotype.Service;
+
+@Service
+@RequiredArgsConstructor
+public class AvcEventPublisher {
+
+ private final EventsPublisher<CloudEvent> eventsPublisher;
+
+ @Value("${app.ncmp.avc.cm-events-topic}")
+ private String avcTopic;
+
+ /**
+ * Publish attribute value change event.
+ *
+ * @param eventKey id of the cmHandle being registered
+ */
+ public void publishAvcEvent(final String eventKey, final String attributeName,
+ final String oldAttributeValue, final String newAttributeValue) {
+ final AvcEvent avcEvent = buildAvcEvent(attributeName, oldAttributeValue, newAttributeValue);
+
+ final Map<String, String> extensions = createAvcEventExtensions(eventKey);
+ final CloudEvent avcCloudEvent =
+ NcmpCloudEventBuilder.builder().type(AvcEvent.class.getTypeName())
+ .event(avcEvent).extensions(extensions).setCloudEvent().build();
+
+ eventsPublisher.publishCloudEvent(avcTopic, eventKey, avcCloudEvent);
+ }
+
+ private AvcEvent buildAvcEvent(final String attributeName,
+ final String oldAttributeValue,
+ final String newAttributeValue) {
+ final Avc avc = new Avc();
+ avc.setAttributeName(attributeName);
+ avc.setOldAttributeValue(oldAttributeValue);
+ avc.setNewAttributeValue(newAttributeValue);
+
+ final Data payload = new Data();
+ payload.setAttributeValueChange(Collections.singletonList(avc));
+ final AvcEvent avcEvent = new AvcEvent();
+ avcEvent.setData(payload);
+ return avcEvent;
+ }
+
+ private Map<String, String> createAvcEventExtensions(final String eventKey) {
+ final Map<String, String> extensions = new HashMap<>();
+ extensions.put("correlationid", eventKey);
+ return extensions;
+ }
+}
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandler.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandler.java
index fdee28e33..8274772d2 100644
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandler.java
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandler.java
@@ -20,6 +20,7 @@
package org.onap.cps.ncmp.api.impl.events.lcm;
+import java.util.Collection;
import java.util.Map;
import org.onap.cps.ncmp.api.impl.inventory.CmHandleState;
import org.onap.cps.ncmp.api.impl.yangmodels.YangModelCmHandle;
@@ -45,4 +46,10 @@ public interface LcmEventsCmHandleStateHandler {
*/
void updateCmHandleStateBatch(final Map<YangModelCmHandle, CmHandleState> cmHandleStatePerCmHandle);
+ /**
+ * Sets the initial state of cmHandles to ADVISED.
+ *
+ * @param yangModelCmHandles List of Yang Model Cm Handle.
+ */
+ void initiateStateAdvised(Collection<YangModelCmHandle> yangModelCmHandles);
}
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandlerImpl.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandlerImpl.java
index 02368b857..a31332f09 100644
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandlerImpl.java
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandlerImpl.java
@@ -28,6 +28,7 @@ import static org.onap.cps.ncmp.api.impl.inventory.CmHandleState.READY;
import io.micrometer.core.annotation.Timed;
import java.util.ArrayList;
import java.util.Collection;
+import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
@@ -82,6 +83,15 @@ public class LcmEventsCmHandleStateHandlerImpl implements LcmEventsCmHandleState
lcmEventsCmHandleStateHandlerAsyncHelper.publishLcmEventBatchAsynchronously(cmHandleTransitionPairs);
}
+ @Override
+ public void initiateStateAdvised(final Collection<YangModelCmHandle> yangModelCmHandles) {
+ final Map<YangModelCmHandle, CmHandleState> cmHandleStatePerCmHandle = new HashMap<>(yangModelCmHandles.size());
+ for (final YangModelCmHandle yangModelCmHandle : yangModelCmHandles) {
+ cmHandleStatePerCmHandle.put(yangModelCmHandle, ADVISED);
+ }
+ updateCmHandleStateBatch(cmHandleStatePerCmHandle);
+ }
+
private Collection<CmHandleTransitionPair> prepareCmHandleTransitionBatch(
final Map<YangModelCmHandle, CmHandleState> cmHandleStatePerCmHandle) {
final List<CmHandleTransitionPair> cmHandleTransitionPairs = new ArrayList<>(cmHandleStatePerCmHandle.size());
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/trustlevel/DeviceHeartbeatConsumer.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/trustlevel/DeviceHeartbeatConsumer.java
index b6d74d980..45aa631b1 100644
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/trustlevel/DeviceHeartbeatConsumer.java
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/trustlevel/DeviceHeartbeatConsumer.java
@@ -22,22 +22,20 @@ package org.onap.cps.ncmp.api.impl.trustlevel;
import io.cloudevents.CloudEvent;
import io.cloudevents.kafka.impl.KafkaHeaders;
-import java.util.Map;
import lombok.RequiredArgsConstructor;
-import lombok.extern.slf4j.Slf4j;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.onap.cps.ncmp.api.impl.events.mapper.CloudEventMapper;
import org.onap.cps.ncmp.events.trustlevel.DeviceTrustLevel;
import org.springframework.kafka.annotation.KafkaListener;
import org.springframework.stereotype.Component;
-@Slf4j
@Component
@RequiredArgsConstructor
public class DeviceHeartbeatConsumer {
private static final String CLOUD_EVENT_ID_HEADER_NAME = "ce_id";
- private final Map<String, TrustLevel> trustLevelPerCmHandle;
+ private final TrustLevelManager trustLevelManager;
+
/**
* Listening the device heartbeats.
@@ -45,20 +43,17 @@ public class DeviceHeartbeatConsumer {
* @param deviceHeartbeatConsumerRecord Device Heartbeat record.
*/
@KafkaListener(topics = "${app.dmi.device-heartbeat.topic}",
- containerFactory = "cloudEventConcurrentKafkaListenerContainerFactory")
+ containerFactory = "cloudEventConcurrentKafkaListenerContainerFactory")
public void heartbeatListener(final ConsumerRecord<String, CloudEvent> deviceHeartbeatConsumerRecord) {
final String cmHandleId = KafkaHeaders.getParsedKafkaHeader(deviceHeartbeatConsumerRecord.headers(),
- CLOUD_EVENT_ID_HEADER_NAME);
+ CLOUD_EVENT_ID_HEADER_NAME);
final DeviceTrustLevel deviceTrustLevel =
- CloudEventMapper.toTargetEvent(deviceHeartbeatConsumerRecord.value(), DeviceTrustLevel.class);
+ CloudEventMapper.toTargetEvent(deviceHeartbeatConsumerRecord.value(), DeviceTrustLevel.class);
+
+ trustLevelManager.handleUpdateOfTrustLevels(cmHandleId, deviceTrustLevel.getData().getTrustLevel());
- if (cmHandleId != null && deviceTrustLevel != null) {
- final String trustLevel = deviceTrustLevel.getData().getTrustLevel();
- trustLevelPerCmHandle.put(cmHandleId, TrustLevel.valueOf(trustLevel));
- log.debug("Added cmHandleId to trustLevelPerCmHandle map as {}:{}", cmHandleId, trustLevel);
- }
}
}
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/trustlevel/TrustLevelManager.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/trustlevel/TrustLevelManager.java
new file mode 100644
index 000000000..4df6bd237
--- /dev/null
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/trustlevel/TrustLevelManager.java
@@ -0,0 +1,90 @@
+/*
+ * ============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.trustlevel;
+
+import java.util.Map;
+import lombok.RequiredArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+import org.onap.cps.ncmp.api.impl.events.avc.ncmptoclient.AvcEventPublisher;
+import org.springframework.stereotype.Service;
+
+@Slf4j
+@Service
+@RequiredArgsConstructor
+public class TrustLevelManager {
+
+ private final Map<String, TrustLevel> trustLevelPerCmHandle;
+ private final AvcEventPublisher avcEventPublisher;
+ private static final String NO_OLD_VALUE = null;
+ private static final String CHANGED_ATTRIBUTE_NAME = "trustLevel";
+
+
+ /**
+ * Add cmHandles to the cache and publish notification for initial trust level of cmHandles if it is NONE.
+ *
+ * @param cmHandlesToBeCreated a list of cmHandles being created
+ */
+ public void handleInitialRegistrationOfTrustLevels(final Map<String, TrustLevel> cmHandlesToBeCreated) {
+ for (final Map.Entry<String, TrustLevel> entry : cmHandlesToBeCreated.entrySet()) {
+ final String cmHandleId = entry.getKey();
+ if (trustLevelPerCmHandle.containsKey(cmHandleId)) {
+ log.warn("Cm handle: {} already registered", cmHandleId);
+ } else {
+ TrustLevel initialTrustLevel = entry.getValue();
+ if (initialTrustLevel == null) {
+ initialTrustLevel = TrustLevel.COMPLETE;
+ }
+ trustLevelPerCmHandle.put(cmHandleId, initialTrustLevel);
+ if (TrustLevel.NONE.equals(initialTrustLevel)) {
+ sendAvcNotificationEvent(cmHandleId, NO_OLD_VALUE, initialTrustLevel.name());
+ }
+ }
+ }
+ }
+
+
+ /**
+ * Update a cmHandle in the cache and publish notification if the trust level is different.
+ *
+ * @param cmHandleId id of the cmHandle being updated
+ * @param newTrustLevel new trust level of the cmHandle being updated
+ */
+ public void handleUpdateOfTrustLevels(final String cmHandleId, final String newTrustLevel) {
+ final TrustLevel oldTrustLevel = trustLevelPerCmHandle.get(cmHandleId);
+ if (newTrustLevel.equals(oldTrustLevel.name())) {
+ log.debug("The Cm Handle: {} has already the same trust level: {}", cmHandleId, newTrustLevel);
+ } else {
+ trustLevelPerCmHandle.put(cmHandleId, TrustLevel.valueOf(newTrustLevel));
+ sendAvcNotificationEvent(cmHandleId, oldTrustLevel.name(), newTrustLevel);
+ log.info("The new trust level: {} has been updated for Cm Handle: {}", newTrustLevel, cmHandleId);
+ }
+ }
+
+ private void sendAvcNotificationEvent(final String cmHandleId,
+ final String oldTrustLevel,
+ final String newTrustLevel) {
+ avcEventPublisher.publishAvcEvent(cmHandleId,
+ CHANGED_ATTRIBUTE_NAME,
+ oldTrustLevel,
+ newTrustLevel);
+ }
+
+}
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/trustlevel/dmiavailability/DmiPluginWatchDog.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/trustlevel/dmiavailability/DmiPluginWatchDog.java
index b073f1bc3..9fd096dae 100644
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/trustlevel/dmiavailability/DmiPluginWatchDog.java
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/trustlevel/dmiavailability/DmiPluginWatchDog.java
@@ -20,11 +20,14 @@
package org.onap.cps.ncmp.api.impl.trustlevel.dmiavailability;
+import java.util.Collection;
import java.util.Map;
import lombok.RequiredArgsConstructor;
import lombok.extern.slf4j.Slf4j;
+import org.onap.cps.ncmp.api.NetworkCmProxyDataService;
import org.onap.cps.ncmp.api.impl.client.DmiRestClient;
import org.onap.cps.ncmp.api.impl.trustlevel.TrustLevel;
+import org.onap.cps.ncmp.api.impl.trustlevel.TrustLevelManager;
import org.springframework.scheduling.annotation.Scheduled;
import org.springframework.stereotype.Service;
@@ -34,6 +37,8 @@ import org.springframework.stereotype.Service;
public class DmiPluginWatchDog {
private final DmiRestClient dmiRestClient;
+ private final NetworkCmProxyDataService networkCmProxyDataService;
+ private final TrustLevelManager trustLevelManager;
private final Map<String, TrustLevel> trustLevelPerDmiPlugin;
/**
@@ -42,15 +47,36 @@ public class DmiPluginWatchDog {
* The @fixedDelayString is the time interval, in milliseconds, between consecutive checks.
*/
@Scheduled(fixedDelayString = "${ncmp.timers.trust-evel.dmi-availability-watchdog-ms:30000}")
- public void watchDmiPluginTrustLevel() {
- trustLevelPerDmiPlugin.keySet().forEach(dmiKey -> {
- final String dmiHealthStatus = dmiRestClient.getDmiHealthStatus(dmiKey);
+ public void checkDmiAvailability() {
+ trustLevelPerDmiPlugin.entrySet().forEach(entry -> {
+ final TrustLevel newDmiTrustLevel;
+ final TrustLevel oldDmiTrustLevel = entry.getValue();
+ final String dmiServiceName = entry.getKey();
+ final String dmiHealthStatus = getDmiHealthStatus(dmiServiceName);
+ log.debug("The health status for dmi-plugin: {} is {}", dmiServiceName, dmiHealthStatus);
+
if ("UP".equals(dmiHealthStatus)) {
- trustLevelPerDmiPlugin.put(dmiKey, TrustLevel.COMPLETE);
+ newDmiTrustLevel = TrustLevel.COMPLETE;
+ } else {
+ newDmiTrustLevel = TrustLevel.NONE;
+ }
+
+ if (oldDmiTrustLevel.equals(newDmiTrustLevel)) {
+ log.debug("The Dmi Plugin: {} has already the same trust level: {}", dmiServiceName,
+ newDmiTrustLevel);
} else {
- trustLevelPerDmiPlugin.put(dmiKey, TrustLevel.NONE);
+ trustLevelPerDmiPlugin.put(dmiServiceName, newDmiTrustLevel);
+
+ final Collection<String> notificationCandidateCmHandleIds =
+ networkCmProxyDataService.getAllCmHandleIdsByDmiPluginIdentifier(dmiServiceName);
+ for (final String cmHandleId: notificationCandidateCmHandleIds) {
+ trustLevelManager.handleUpdateOfTrustLevels(cmHandleId, newDmiTrustLevel.name());
+ }
}
});
}
+ private String getDmiHealthStatus(final String dmiServiceName) {
+ return dmiRestClient.getDmiHealthStatus(dmiServiceName);
+ }
}
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/models/CmHandleRegistrationResponse.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/models/CmHandleRegistrationResponse.java
index b0aed5763..82283228a 100644
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/models/CmHandleRegistrationResponse.java
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/models/CmHandleRegistrationResponse.java
@@ -116,7 +116,7 @@ public class CmHandleRegistrationResponse {
.status(Status.SUCCESS).build();
}
- public static List<CmHandleRegistrationResponse> createSuccessResponses(final List<String> cmHandleIds) {
+ public static List<CmHandleRegistrationResponse> createSuccessResponses(final Collection<String> cmHandleIds) {
return cmHandleIds.stream().map(CmHandleRegistrationResponse::createSuccessResponse)
.collect(Collectors.toList());
}
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplRegistrationSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplRegistrationSpec.groovy
index 9f15e1fa5..97693a415 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplRegistrationSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplRegistrationSpec.groovy
@@ -21,6 +21,7 @@
package org.onap.cps.ncmp.api.impl
+import org.onap.cps.ncmp.api.impl.trustlevel.TrustLevelManager
import org.onap.cps.ncmp.api.models.UpgradedCmHandles
import static org.onap.cps.ncmp.api.NcmpResponseStatus.CM_HANDLES_NOT_FOUND
@@ -68,8 +69,8 @@ class NetworkCmProxyDataServiceImplRegistrationSpec extends Specification {
def mockLcmEventsCmHandleStateHandler = Mock(LcmEventsCmHandleStateHandler)
def mockCpsDataService = Mock(CpsDataService)
def mockModuleSyncStartedOnCmHandles = Mock(IMap<String, Object>)
- def trustLevelPerCmHandle = [:]
def trustLevelPerDmiPlugin = [:]
+ def mockTrustLevelManager = Mock(TrustLevelManager)
def objectUnderTest = getObjectUnderTest()
def 'DMI Registration: Create, Update, Delete & Upgrade operations are processed in the right order'() {
@@ -208,14 +209,12 @@ class NetworkCmProxyDataServiceImplRegistrationSpec extends Specification {
assert it.cmHandle == 'cmhandle'
}
and: 'state handler is invoked with the expected parameters'
- 1 * mockLcmEventsCmHandleStateHandler.updateCmHandleStateBatch(_) >> {
+ 1 * mockLcmEventsCmHandleStateHandler.initiateStateAdvised(_) >> {
args ->
{
def cmHandleStatePerCmHandle = (args[0] as Map)
cmHandleStatePerCmHandle.each {
- assert (it.key.id == 'cmhandle'
- && it.key.dmiServiceName == 'my-server'
- && it.value == CmHandleState.ADVISED)
+ assert (it.id == 'cmhandle' && it.dmiServiceName == 'my-server')
}
}
}
@@ -237,12 +236,12 @@ class NetworkCmProxyDataServiceImplRegistrationSpec extends Specification {
then: 'a successful response is received'
assert response.createdCmHandles.size() == expectedNumberOfCreatedCmHandles
and: 'trustLevel is set for the created cm-handle'
- assert trustLevelPerCmHandle.get(cmHandleId) == expectedTrustLevel
+ 1 * mockTrustLevelManager.handleInitialRegistrationOfTrustLevels(_)
where:
- scenario | cmHandleId | registrationTrustLevel || expectedNumberOfCreatedCmHandles | expectedTrustLevel
- 'new cmHandleId and trustLevel' | 'ch-new' | TrustLevel.COMPLETE || 2 | TrustLevel.COMPLETE
- 'existing cmHandleId with null trustLevel' | 'ch-1' | null || 1 | TrustLevel.NONE
- 'cmHandleId with null trustLevel' | 'ch-new' | null || 2 | TrustLevel.COMPLETE
+ scenario | cmHandleId | registrationTrustLevel || expectedNumberOfCreatedCmHandles
+ 'new trusted cm handle' | 'ch-new' | TrustLevel.COMPLETE || 2
+ 'existing cm handle without trust level' | 'ch-1' | null || 1
+ 'new cm handle without trust level' | 'ch-new' | null || 2
}
def 'Create CM-Handle Multiple Requests: All cm-handles creation requests are processed with some failures'() {
@@ -253,7 +252,7 @@ class NetworkCmProxyDataServiceImplRegistrationSpec extends Specification {
new NcmpServiceCmHandle(cmHandleId: 'cmhandle3')])
and: 'cm-handle creation is successful for 1st and 3rd; failed for 2nd'
def xpath = "somePathWithId[@id='cmhandle2']"
- mockLcmEventsCmHandleStateHandler.updateCmHandleStateBatch(*_) >> { throw AlreadyDefinedException.forDataNodes([xpath], 'some-context') }
+ mockLcmEventsCmHandleStateHandler.initiateStateAdvised(*_) >> { throw AlreadyDefinedException.forDataNodes([xpath], 'some-context') }
when: 'registration is updated to create cm-handles'
def response = objectUnderTest.updateDmiRegistrationAndSyncModule(dmiPluginRegistration)
then: 'a response is received for all cm-handles'
@@ -272,7 +271,7 @@ class NetworkCmProxyDataServiceImplRegistrationSpec extends Specification {
def dmiPluginRegistration = new DmiPluginRegistration(dmiPlugin: 'my-server')
dmiPluginRegistration.createdCmHandles = [new NcmpServiceCmHandle(cmHandleId: 'cmhandle')]
and: 'cm-handler registration fails: #scenario'
- mockLcmEventsCmHandleStateHandler.updateCmHandleStateBatch(*_) >> { throw exception }
+ mockLcmEventsCmHandleStateHandler.initiateStateAdvised(*_) >> { throw exception }
when: 'registration is updated'
def response = objectUnderTest.updateDmiRegistrationAndSyncModule(dmiPluginRegistration)
then: 'a failure response is received'
@@ -439,7 +438,7 @@ class NetworkCmProxyDataServiceImplRegistrationSpec extends Specification {
return Spy(new NetworkCmProxyDataServiceImpl(spiedJsonObjectMapper, mockDmiDataOperations,
mockNetworkCmProxyDataServicePropertyHandler, mockInventoryPersistence, mockCmHandleQueries,
stubbedNetworkCmProxyCmHandlerQueryService, mockLcmEventsCmHandleStateHandler, mockCpsDataService,
- mockModuleSyncStartedOnCmHandles, trustLevelPerCmHandle, trustLevelPerDmiPlugin))
+ mockModuleSyncStartedOnCmHandles, trustLevelPerDmiPlugin, mockTrustLevelManager))
}
def addPersistedYangModelCmHandles(ids) {
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplSpec.groovy
index 71511cc16..4f7b72698 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplSpec.groovy
@@ -22,7 +22,6 @@
*/
package org.onap.cps.ncmp.api.impl
-
import static org.onap.cps.ncmp.api.impl.ncmppersistence.NcmpPersistence.NFP_OPERATIONAL_DATASTORE_DATASPACE_NAME
import static org.onap.cps.ncmp.api.impl.ncmppersistence.NcmpPersistence.NCMP_DATASPACE_NAME
import static org.onap.cps.ncmp.api.impl.ncmppersistence.NcmpPersistence.NCMP_DMI_REGISTRY_ANCHOR
@@ -36,6 +35,7 @@ import com.hazelcast.map.IMap
import org.onap.cps.ncmp.api.NetworkCmProxyCmHandleQueryService
import org.onap.cps.ncmp.api.impl.events.lcm.LcmEventsCmHandleStateHandler
import org.onap.cps.ncmp.api.impl.trustlevel.TrustLevel
+import org.onap.cps.ncmp.api.impl.trustlevel.TrustLevelManager
import org.onap.cps.ncmp.api.impl.yangmodels.YangModelCmHandle
import org.onap.cps.ncmp.api.impl.inventory.CmHandleQueries
import org.onap.cps.ncmp.api.impl.inventory.CmHandleState
@@ -76,8 +76,8 @@ class NetworkCmProxyDataServiceImplSpec extends Specification {
def mockCpsCmHandlerQueryService = Mock(NetworkCmProxyCmHandleQueryService)
def mockLcmEventsCmHandleStateHandler = Mock(LcmEventsCmHandleStateHandler)
def stubModuleSyncStartedOnCmHandles = Stub(IMap<String, Object>)
- def stubTrustLevelPerCmHandle = Stub(Map<String, TrustLevel>)
def stubTrustLevelPerDmiPlugin = Stub(Map<String, TrustLevel>)
+ def mockTrustLevelManager = Mock(TrustLevelManager)
def NO_TOPIC = null
def NO_REQUEST_ID = null
@@ -96,8 +96,8 @@ class NetworkCmProxyDataServiceImplSpec extends Specification {
mockLcmEventsCmHandleStateHandler,
mockCpsDataService,
stubModuleSyncStartedOnCmHandles,
- stubTrustLevelPerCmHandle,
- stubTrustLevelPerDmiPlugin)
+ stubTrustLevelPerDmiPlugin,
+ mockTrustLevelManager)
def cmHandleXPath = "/dmi-registry/cm-handles[@id='testCmHandle']"
@@ -265,11 +265,11 @@ class NetworkCmProxyDataServiceImplSpec extends Specification {
when: 'parse and create cm handle in dmi registration then sync module'
objectUnderTest.parseAndProcessCreatedCmHandlesInRegistration(mockDmiPluginRegistration)
then: 'system persists the cm handle state'
- 1 * mockLcmEventsCmHandleStateHandler.updateCmHandleStateBatch(_) >> {
+ 1 * mockLcmEventsCmHandleStateHandler.initiateStateAdvised(_) >> {
args -> {
- def cmHandleStatePerCmHandle = (args[0] as Map)
+ def cmHandleStatePerCmHandle = (args[0] as Collection)
cmHandleStatePerCmHandle.each {
- assert it.key.id == 'test-cm-handle-id' && it.value == CmHandleState.ADVISED
+ assert it.id == 'test-cm-handle-id'
}
}
}
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/client/DmiRestClientSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/client/DmiRestClientSpec.groovy
index c9ba5645f..0176de714 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/client/DmiRestClientSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/client/DmiRestClientSpec.groovy
@@ -96,7 +96,7 @@ class DmiRestClientSpec extends Specification {
mockRestTemplate.getForObject(*_) >> {jsonNode}
when: 'get trust level of the dmi plugin'
def result = objectUnderTest.getDmiHealthStatus('some url')
- then: 'the correct trust level is returned'
+ then: 'the status value from the json is return'
assert result == 'my status'
}
@@ -105,9 +105,9 @@ class DmiRestClientSpec extends Specification {
mockRestTemplate.getForObject(*_) >> healthStatusResponse
when: 'attempt to get health status of the dmi plugin'
def result = objectUnderTest.getDmiHealthStatus('some url')
- then: 'result will be EMPTY_STRING "" '
+ then: 'result will be empty'
assert result == ''
- where: 'the following values are used'
+ where: 'the following responses are used'
scenario | healthStatusResponse
'null' | null
'exception' | {throw new Exception()}
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avc/ncmptoclient/AvcEventPublisherSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avc/ncmptoclient/AvcEventPublisherSpec.groovy
new file mode 100644
index 000000000..a614fd2a1
--- /dev/null
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/avc/ncmptoclient/AvcEventPublisherSpec.groovy
@@ -0,0 +1,63 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (c) 2023 Nordix Foundation.
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an 'AS IS' BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ * SPDX-License-Identifier: Apache-2.0
+ * ============LICENSE_END=========================================================
+ */
+
+package org.onap.cps.ncmp.api.impl.events.avc.ncmptoclient
+
+import com.fasterxml.jackson.databind.ObjectMapper
+import io.cloudevents.CloudEvent
+import org.onap.cps.ncmp.api.impl.events.EventsPublisher
+import org.onap.cps.ncmp.api.impl.utils.context.CpsApplicationContext
+import org.onap.cps.ncmp.api.kafka.MessagingBaseSpec
+import org.onap.cps.ncmp.events.avc.ncmp_to_client.Avc
+import org.onap.cps.ncmp.events.avc.ncmp_to_client.AvcEvent
+import org.onap.cps.utils.JsonObjectMapper
+import org.springframework.test.context.ContextConfiguration
+
+import static org.onap.cps.ncmp.api.impl.events.mapper.CloudEventMapper.toTargetEvent
+
+@ContextConfiguration(classes = [CpsApplicationContext, ObjectMapper, JsonObjectMapper])
+class AvcEventPublisherSpec extends MessagingBaseSpec {
+
+ def mockEventsPublisher = Mock(EventsPublisher<CloudEvent>)
+ def objectUnderTest = new AvcEventPublisher(mockEventsPublisher)
+
+ def 'Publish an attribute value change event'() {
+ given: 'the event key'
+ def someEventKey = 'someEventKey'
+ and: 'the name of the attribute being changed'
+ def someAttributeName = 'someAttributeName'
+ and: 'the old value of the attribute'
+ def someOldAttributeValue = 'someOldAttributeValue'
+ and: 'the new value of the attribute'
+ def someNewAttributeValue = 'someNewAttributeValue'
+ when: 'an attribute value change event is published'
+ objectUnderTest.publishAvcEvent(someEventKey, someAttributeName, someOldAttributeValue, someNewAttributeValue)
+ then: 'the cloud event publisher is invoked with the correct data'
+ 1 * mockEventsPublisher.publishCloudEvent(_, someEventKey,
+ cloudEvent -> {
+ def actualAvcs = toTargetEvent(cloudEvent, AvcEvent.class).data.attributeValueChange
+ def expectedAvc = new Avc(attributeName: someAttributeName,
+ oldAttributeValue: someOldAttributeValue,
+ newAttributeValue: someNewAttributeValue)
+ assert actualAvcs == [expectedAvc]
+ })
+ }
+
+}
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandlerImplSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandlerImplSpec.groovy
index 0ec73a26a..3ae95209f 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandlerImplSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandlerImplSpec.groovy
@@ -169,9 +169,9 @@ class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
def 'Batch of new cm handles provided'() {
given: 'A batch of new cm handles'
- def cmHandleStateMap = setupBatch('NEW')
- when: 'updating a batch of changes'
- objectUnderTest.updateCmHandleStateBatch(cmHandleStateMap)
+ def yangModelCmHandlesToBeCreated = setupBatch('NEW')
+ when: 'instantiating a batch of new cm handles'
+ objectUnderTest.initiateStateAdvised(yangModelCmHandlesToBeCreated)
then: 'new cm handles are saved using inventory persistence'
1 * mockInventoryPersistence.saveCmHandleBatch(_) >> {
args -> {
@@ -180,7 +180,6 @@ class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
}
and: 'event service is called to publish events'
2 * mockLcmEventsService.publishLcmEvent(_, _, _)
-
}
def 'Batch of existing cm handles is updated'() {
@@ -219,7 +218,7 @@ class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
def yangModelCmHandle2 = new YangModelCmHandle(id: 'cmhandle2', dmiProperties: [], publicProperties: [])
if ('NEW' == type) {
- return [(yangModelCmHandle1): ADVISED, (yangModelCmHandle2): ADVISED]
+ return [yangModelCmHandle1, yangModelCmHandle2]
}
if ('DELETED' == type) {
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/DeviceHeartbeatConsumerSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/DeviceHeartbeatConsumerSpec.groovy
index 80778b9c7..8886fc189 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/DeviceHeartbeatConsumerSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/DeviceHeartbeatConsumerSpec.groovy
@@ -21,7 +21,6 @@
package org.onap.cps.ncmp.api.impl.trustlevel
import com.fasterxml.jackson.databind.ObjectMapper
-import com.hazelcast.map.IMap
import io.cloudevents.CloudEvent
import io.cloudevents.core.builder.CloudEventBuilder
import org.apache.kafka.clients.consumer.ConsumerRecord
@@ -34,9 +33,9 @@ import spock.lang.Specification
@SpringBootTest(classes = [ObjectMapper, JsonObjectMapper])
class DeviceHeartbeatConsumerSpec extends Specification {
- def mockTrustLevelPerCmHandle = Mock(Map<String, TrustLevel>)
+ def mockTrustLevelManager = Mock(TrustLevelManager)
- def objectUnderTest = new DeviceHeartbeatConsumer(mockTrustLevelPerCmHandle)
+ def objectUnderTest = new DeviceHeartbeatConsumer(mockTrustLevelManager)
def objectMapper = new ObjectMapper()
@Autowired
@@ -54,29 +53,7 @@ class DeviceHeartbeatConsumerSpec extends Specification {
when: 'the event is consumed'
objectUnderTest.heartbeatListener(consumerRecord)
then: 'cm handles are stored with correct trust level'
- 1 * mockTrustLevelPerCmHandle.put('"cmhandle1"', TrustLevel.COMPLETE)
- }
-
- def 'Consume trustlevel event without cloud event id'() {
- given: 'an event from dmi'
- def payload = jsonObjectMapper.convertJsonString(trustLevelString, DeviceTrustLevel.class)
- def eventFromDmi = createTrustLevelEvent(payload)
- and: 'transformed to a consumer record WITHOUT Cloud event ID (ce_id)'
- def consumerRecord = new ConsumerRecord<String, CloudEvent>('test-device-heartbeat', 0, 0, 'sample-message-key', eventFromDmi)
- when: 'the event is consumed'
- objectUnderTest.heartbeatListener(consumerRecord)
- then: 'no cm handle has been stored in the map'
- 0 * mockTrustLevelPerCmHandle.put(*_)
- }
-
- def 'Consume a trust level event without payload'() {
- given: 'a consumer record with ce_id header but without payload'
- def consumerRecord = new ConsumerRecord<String, CloudEvent>('test-device-heartbeat', 0, 0, 'cmhandle1', createTrustLevelEvent(null))
- consumerRecord.headers().add('some_other_header_value', objectMapper.writeValueAsBytes('cmhandle1'))
- when: 'the event is consumed'
- objectUnderTest.heartbeatListener(consumerRecord)
- then: 'no cm handle has been stored in the map'
- 0 * mockTrustLevelPerCmHandle.put(*_)
+ 1 * mockTrustLevelManager.handleUpdateOfTrustLevels('"cmhandle1"', 'COMPLETE')
}
def createTrustLevelEvent(eventPayload) {
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/TrustLevelManagerSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/TrustLevelManagerSpec.groovy
new file mode 100644
index 000000000..b3559e41e
--- /dev/null
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/TrustLevelManagerSpec.groovy
@@ -0,0 +1,73 @@
+/*
+ * ============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.trustlevel
+
+import org.onap.cps.ncmp.api.impl.events.avc.ncmptoclient.AvcEventPublisher
+import spock.lang.Specification
+
+class TrustLevelManagerSpec extends Specification {
+
+ def trustLevelPerCmHandle = [:]
+ def mockAttributeValueChangeEventPublisher = Mock(AvcEventPublisher)
+ def objectUnderTest = new TrustLevelManager(trustLevelPerCmHandle, mockAttributeValueChangeEventPublisher)
+
+ def 'Initial cm handle registration'() {
+ given: 'two cm handles: one with no trust level and one trusted'
+ def cmHandleModelsToBeCreated = ['ch-1': null, 'ch-2': TrustLevel.COMPLETE]
+ when: 'the initial registration handled'
+ objectUnderTest.handleInitialRegistrationOfTrustLevels(cmHandleModelsToBeCreated)
+ then: 'no notification sent'
+ 0 * mockAttributeValueChangeEventPublisher.publishAvcEvent(*_)
+ and: 'both cm handles are in the cache and are trusted'
+ assert trustLevelPerCmHandle.get('ch-1') == TrustLevel.COMPLETE
+ assert trustLevelPerCmHandle.get('ch-2') == TrustLevel.COMPLETE
+ }
+
+ def 'Initial cm handle registration with a cm handle that is not trusted'() {
+ given: 'a not trusted cm handle'
+ def cmHandleModelsToBeCreated = ['ch-2': TrustLevel.NONE]
+ when: 'the initial registration handled'
+ objectUnderTest.handleInitialRegistrationOfTrustLevels(cmHandleModelsToBeCreated)
+ then: 'notification is sent'
+ 1 * mockAttributeValueChangeEventPublisher.publishAvcEvent(*_)
+ }
+
+ def 'Trust level updated'() {
+ given: 'a not trusted cm handle'
+ trustLevelPerCmHandle.put('ch-1', TrustLevel.NONE)
+ when: 'the update is handled'
+ objectUnderTest.handleUpdateOfTrustLevels('ch-1', 'COMPLETE')
+ then: 'notification is sent'
+ 1 * mockAttributeValueChangeEventPublisher.publishAvcEvent('ch-1', 'trustLevel', 'NONE', 'COMPLETE')
+ and: 'the cm handle in the cache is trusted'
+ assert trustLevelPerCmHandle.get('ch-1') == TrustLevel.COMPLETE
+ }
+
+ def 'Trust level updated with same value'() {
+ given: 'a trusted cm handle'
+ trustLevelPerCmHandle.put('ch-1', TrustLevel.COMPLETE)
+ when: 'the update is handled'
+ objectUnderTest.handleUpdateOfTrustLevels('ch-1', 'COMPLETE')
+ then: 'no notification is sent'
+ 0 * mockAttributeValueChangeEventPublisher.publishAvcEvent(*_)
+ }
+
+}
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/TrustLevelTest.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/TrustLevelSpec.groovy
index 9971f6307..f2521b560 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/TrustLevelTest.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/TrustLevelSpec.groovy
@@ -22,7 +22,7 @@ package org.onap.cps.ncmp.api.impl.trustlevel
import spock.lang.Specification
-class TrustLevelTest extends Specification {
+class TrustLevelSpec extends Specification {
def 'Get effective trust level between this and other.'() {
expect: 'the lower of two is returned'
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/dmiavailability/DmiPluginWatchDogSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/dmiavailability/DmiPluginWatchDogSpec.groovy
index 2771c4df1..ec1d8e8f3 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/dmiavailability/DmiPluginWatchDogSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/trustlevel/dmiavailability/DmiPluginWatchDogSpec.groovy
@@ -20,31 +20,45 @@
package org.onap.cps.ncmp.api.impl.trustlevel.dmiavailability
+import org.onap.cps.ncmp.api.NetworkCmProxyDataService
import org.onap.cps.ncmp.api.impl.client.DmiRestClient
import org.onap.cps.ncmp.api.impl.trustlevel.TrustLevel
+import org.onap.cps.ncmp.api.impl.trustlevel.TrustLevelManager
import spock.lang.Specification
class DmiPluginWatchDogSpec extends Specification {
def mockDmiRestClient = Mock(DmiRestClient)
+ def mockNetworkCmProxyDataService = Mock(NetworkCmProxyDataService)
+ def mockTrustLevelManager = Mock(TrustLevelManager)
def trustLevelPerDmiPlugin = [:]
- def objectUnderTest = new DmiPluginWatchDog(mockDmiRestClient, trustLevelPerDmiPlugin)
+
+ def objectUnderTest = new DmiPluginWatchDog(mockDmiRestClient,
+ mockNetworkCmProxyDataService,
+ mockTrustLevelManager,
+ trustLevelPerDmiPlugin)
def 'watch dmi plugin health status for #dmiHealhStatus'() {
given: 'the cache has been initialised and "knows" about dmi-1'
- trustLevelPerDmiPlugin.put('dmi-1',null)
+ trustLevelPerDmiPlugin.put('dmi-1', dmiOldTrustLevel)
and: 'dmi client returns health status #dmiHealhStatus'
mockDmiRestClient.getDmiHealthStatus('dmi-1') >> dmiHealhStatus
+ and: 'network cm proxy data returns a list of all cm handle ids belonging to a dmi'
+ mockNetworkCmProxyDataService.getAllCmHandleIdsByDmiPluginIdentifier('dmi-1') >> ['ch-1']
when: 'dmi watch dog method runs'
- objectUnderTest.watchDmiPluginTrustLevel()
+ objectUnderTest.checkDmiAvailability()
then: 'the result is as expected'
- assert trustLevelPerDmiPlugin.get('dmi-1') == expectedResult
- where: 'the following health status is used'
- dmiHealhStatus || expectedResult
- 'UP' || TrustLevel.COMPLETE
- 'Other' || TrustLevel.NONE
- null || TrustLevel.NONE
+ assert trustLevelPerDmiPlugin.get('dmi-1') == newDmiTrustLevel
+ and: 'the update delegated to manager'
+ times * mockTrustLevelManager.handleUpdateOfTrustLevels(*_)
+ where: 'the following parameters are used'
+ dmiHealhStatus | dmiOldTrustLevel || newDmiTrustLevel || times
+ 'UP' | TrustLevel.COMPLETE || TrustLevel.COMPLETE || 0
+ 'DOWN' | TrustLevel.COMPLETE || TrustLevel.NONE || 1
+ 'DOWN' | TrustLevel.NONE || TrustLevel.NONE || 0
+ 'UP' | TrustLevel.NONE || TrustLevel.COMPLETE || 1
+ '' | TrustLevel.COMPLETE || TrustLevel.NONE || 1
}
}