then: 'an HTTP response is returned with correct message and details'
assertTestResponse(response, expectedErrorCode, expectedErrorMessage, expectedErrorDetails)
where:
- scenario | exception || expectedErrorCode | expectedErrorMessage | expectedErrorDetails
- 'CPS' | new CpsException(sampleErrorMessage, sampleErrorDetails) || INTERNAL_SERVER_ERROR | sampleErrorMessage | sampleErrorDetails
- 'NCMP-server' | new ServerNcmpException(sampleErrorMessage, sampleErrorDetails) || INTERNAL_SERVER_ERROR | sampleErrorMessage | null
- 'DMI Request' | new DmiRequestException(sampleErrorMessage, sampleErrorDetails) || BAD_REQUEST | sampleErrorMessage | null
- 'Invalid Operation' | new InvalidOperationException('some reason') || BAD_REQUEST | 'some reason' | null
- 'Unsupported Operation' | new OperationNotSupportedException('not yet') || BAD_REQUEST | 'not yet' | null
- 'DataNode Validation' | new DataNodeNotFoundException('myDataspaceName', 'myAnchorName') || NOT_FOUND | 'DataNode not found' | null
- 'other' | new IllegalStateException(sampleErrorMessage) || INTERNAL_SERVER_ERROR | sampleErrorMessage | null
- 'Data Node Not Found' | new DataNodeNotFoundException('myDataspaceName', 'myAnchorName') || NOT_FOUND | 'DataNode not found' | 'DataNode not found'
- 'Existing entry' | new AlreadyDefinedException('name',null) || CONFLICT | 'Already defined exception' | 'name already exists'
- 'Existing entries' | AlreadyDefinedException.forDataNodes(['A', 'B'], 'myAnchorName') || CONFLICT | 'Already defined exception' | '2 data node(s) already exist'
- 'Operation too large' | new PayloadTooLargeException(sampleErrorMessage) || PAYLOAD_TOO_LARGE | sampleErrorMessage | 'Check logs'
- 'Policy Executor' | new PolicyExecutorException(sampleErrorMessage, sampleErrorDetails, null) || CONFLICT | sampleErrorMessage | sampleErrorDetails
- 'Invalid Path' | new ProvMnSException('not a valid path' ,'some invalid path not a valid path') || UNPROCESSABLE_ENTITY | 'not a valid path' | 'some invalid path not a valid path'
- 'Invalid Path' | new ProvMnSException('some invalid path not a valid path') || UNPROCESSABLE_ENTITY | 'some invalid path not a valid path' | null
+ scenario | exception || expectedErrorCode | expectedErrorMessage | expectedErrorDetails
+ 'CPS' | new CpsException(sampleErrorMessage, sampleErrorDetails) || INTERNAL_SERVER_ERROR | sampleErrorMessage | sampleErrorDetails
+ 'NCMP-server' | new ServerNcmpException(sampleErrorMessage, sampleErrorDetails) || INTERNAL_SERVER_ERROR | sampleErrorMessage | null
+ 'DMI Request' | new DmiRequestException(sampleErrorMessage, sampleErrorDetails) || BAD_REQUEST | sampleErrorMessage | null
+ 'Invalid Operation' | new InvalidOperationException('some reason') || BAD_REQUEST | 'some reason' | null
+ 'Unsupported Operation' | new OperationNotSupportedException('not yet') || BAD_REQUEST | 'not yet' | null
+ 'DataNode Validation' | new DataNodeNotFoundException('myDataspaceName', 'myAnchorName') || NOT_FOUND | 'DataNode not found' | null
+ 'other' | new IllegalStateException(sampleErrorMessage) || INTERNAL_SERVER_ERROR | sampleErrorMessage | null
+ 'Data Node Not Found' | new DataNodeNotFoundException('myDataspaceName', 'myAnchorName') || NOT_FOUND | 'DataNode not found' | 'DataNode not found'
+ 'Existing entry' | new AlreadyDefinedException('name',null) || CONFLICT | 'Already defined exception' | 'name already exists'
+ 'Existing entries' | AlreadyDefinedException.forDataNodes(['A', 'B'], 'myAnchorName') || CONFLICT | 'Already defined exception' | '2 data node(s) already exist'
+ 'Operation too large' | new PayloadTooLargeException(sampleErrorMessage) || PAYLOAD_TOO_LARGE | sampleErrorMessage | 'Check logs'
+ 'Policy Executor' | new PolicyExecutorException(sampleErrorMessage, sampleErrorDetails, null) || CONFLICT | sampleErrorMessage | sampleErrorDetails
+ 'Invalid Path' | new ProvMnSException('not a valid path' ,'some details') || UNPROCESSABLE_ENTITY | 'not a valid path' | 'some details'
}
def 'Post request with exception returns correct HTTP Status.'() {
super(message, details);
}
- /**
- * Constructor.
- *
- * @param message exception message
- */
- public ProvMnSException(final String message) {
- super(message, null);
- }
-
}
/*
* ============LICENSE_START=======================================================
* Copyright (C) 2022 Bell Canada
- * Modifications Copyright (C) 2022-2023 Nordix Foundation.
+ * Modifications Copyright (C) 2022-2025 OpenInfra Foundation Europe.
* ================================================================================
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
}
for (final DataNode stateChildNode : dataNode.getChildDataNodes()) {
if (stateChildNode.getXpath().endsWith("/lock-reason")) {
- this.lockReason = getLockReason(stateChildNode);
+ this.lockReason = toLockReason(stateChildNode);
}
if (stateChildNode.getXpath().endsWith("/datastores")) {
for (final DataNode dataStoreNodes : stateChildNode.getChildDataNodes()) {
Operational operationalDataStore = null;
if (dataStoreNodes.getXpath().contains("/operational")) {
- operationalDataStore = getOperationalDataStore(dataStoreNodes);
+ operationalDataStore = toOperationalDataStore(dataStoreNodes);
}
this.datastores = DataStores.builder().operationalDataStore(operationalDataStore).build();
}
return this;
}
- private Operational getOperationalDataStore(final DataNode dataStoreNodes) {
+ private Operational toOperationalDataStore(final DataNode dataStoreNodes) {
return Operational.builder()
.dataStoreSyncState(DataStoreSyncState.valueOf((String) dataStoreNodes.getLeaves().get("sync-state")))
.lastSyncTime((String) dataStoreNodes.getLeaves().get("last-sync-time"))
.build();
}
- private LockReason getLockReason(final DataNode stateChildNode) {
+ private LockReason toLockReason(final DataNode stateChildNode) {
final boolean isLockReasonExists = stateChildNode.getLeaves().containsKey("reason");
return new LockReason(isLockReasonExists
? LockReasonCategory.valueOf((String) stateChildNode.getLeaves().get("reason"))
}
}
+
+
@Override
public Collection<ModuleDefinition> getModuleDefinitionsByCmHandleAndModule(final String cmHandleReference,
final String moduleName,
import org.onap.cps.ncmp.api.inventory.models.CmHandleRegistrationResponse;
import org.onap.cps.ncmp.api.inventory.models.NcmpServiceCmHandle;
import org.onap.cps.ncmp.impl.inventory.models.YangModelCmHandle;
+import org.onap.cps.ncmp.impl.inventory.sync.lcm.CmHandleTransitionPair;
import org.onap.cps.ncmp.impl.inventory.sync.lcm.LcmEventsHelper;
import org.onap.cps.ncmp.impl.utils.YangDataConverter;
import org.onap.cps.utils.ContentType;
}
private void updateDataProducerIdentifier(final DataNode cmHandleDataNode,
- final NcmpServiceCmHandle ncmpServiceCmHandle) {
+ final NcmpServiceCmHandle ncmpServiceCmHandle) {
final String targetDataProducerIdentifier = ncmpServiceCmHandle.getDataProducerIdentifier();
final String cmHandleId = ncmpServiceCmHandle.getCmHandleId();
return;
}
- final YangModelCmHandle existingYangModelCmHandle = YangDataConverter.toYangModelCmHandle(cmHandleDataNode);
- final String existingDataProducerIdentifier = existingYangModelCmHandle.getDataProducerIdentifier();
+ final YangModelCmHandle currentYangModelCmHandle = YangDataConverter.toYangModelCmHandle(cmHandleDataNode);
+ final String currentDataProducerIdentifier = currentYangModelCmHandle.getDataProducerIdentifier();
- if (existingDataProducerIdentifier.equals(targetDataProducerIdentifier)) {
+ if (currentDataProducerIdentifier.equals(targetDataProducerIdentifier)) {
log.debug("Ignoring update as dataProducerIdentifier for cmHandle {} is already set to {}.", cmHandleId,
targetDataProducerIdentifier);
return;
}
-
setAndUpdateCmHandleField(cmHandleId, "data-producer-identifier", targetDataProducerIdentifier);
log.debug("dataProducerIdentifier for cmHandle {} updated from {} to {}", cmHandleId,
- existingDataProducerIdentifier, targetDataProducerIdentifier);
- sendLcmEventForDataProducerIdentifier(cmHandleId, existingYangModelCmHandle);
+ currentDataProducerIdentifier, targetDataProducerIdentifier);
+ sendLcmEventForDataProducerIdentifier(cmHandleId, currentYangModelCmHandle);
}
private void sendLcmEventForDataProducerIdentifier(final String cmHandleId,
- final YangModelCmHandle existingYangModelCmHandle) {
+ final YangModelCmHandle currentYangModelCmHandle) {
final YangModelCmHandle updatedYangModelCmHandle = inventoryPersistence.getYangModelCmHandle(cmHandleId);
- final NcmpServiceCmHandle existingNcmpServiceCmHandle =
- YangDataConverter.toNcmpServiceCmHandle(existingYangModelCmHandle);
- final NcmpServiceCmHandle updatedNcmpServiceCmHandle =
- YangDataConverter.toNcmpServiceCmHandle(updatedYangModelCmHandle);
-
- lcmEventsHelper.sendLcmEventAsynchronously(updatedNcmpServiceCmHandle,
- existingNcmpServiceCmHandle);
+ final CmHandleTransitionPair cmHandleTransitionPair =
+ new CmHandleTransitionPair(currentYangModelCmHandle, updatedYangModelCmHandle);
+ lcmEventsHelper.sendLcmEventBatchAsynchronously(List.of(cmHandleTransitionPair));
}
private void updateProperties(final DataNode existingCmHandleDataNode, final PropertyType propertyType,
import org.onap.cps.ncmp.api.inventory.models.CmHandleState;
import org.onap.cps.ncmp.api.inventory.models.CompositeState;
import org.onap.cps.ncmp.impl.inventory.CmHandleQueryService;
-import org.onap.cps.ncmp.impl.inventory.sync.lcm.LcmEventsCmHandleStateHandlerImpl.CmHandleTransitionPair;
import org.onap.cps.ncmp.utils.events.NcmpInventoryModelOnboardingFinishedEvent;
import org.springframework.context.annotation.DependsOn;
import org.springframework.context.event.EventListener;
}
private void updateMetricWithStateChange(final CmHandleTransitionPair cmHandleTransitionPair) {
- final CmHandleState targetCmHandleState = cmHandleTransitionPair.getTargetYangModelCmHandle()
+ final CmHandleState targetCmHandleState = cmHandleTransitionPair.targetYangModelCmHandle()
.getCompositeState().getCmHandleState();
- if (isNew(cmHandleTransitionPair.getCurrentYangModelCmHandle().getCompositeState())) {
+ if (isNew(cmHandleTransitionPair.currentYangModelCmHandle().getCompositeState())) {
updateTargetStateCount(targetCmHandleState);
} else {
- final CmHandleState previousCmHandleState = cmHandleTransitionPair.getCurrentYangModelCmHandle()
+ final CmHandleState previousCmHandleState = cmHandleTransitionPair.currentYangModelCmHandle()
.getCompositeState().getCmHandleState();
updatePreviousStateCount(previousCmHandleState);
updateTargetStateCount(targetCmHandleState);
cmHandlesByState.executeOnKey(keyName, new IncreasingEntryProcessor());
}
- private boolean isNew(final CompositeState existingCompositeState) {
- return (existingCompositeState == null);
+ private boolean isNew(final CompositeState currentCompositeState) {
+ return (currentCompositeState == null);
}
static class DecreasingEntryProcessor implements EntryProcessor<String, Integer, Void> {
--- /dev/null
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2025 OpenInfra Foundation Europe. All rights reserved.
+ * ================================================================================
+ * 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.impl.inventory.sync.lcm;
+
+import org.onap.cps.ncmp.impl.inventory.models.YangModelCmHandle;
+
+public record CmHandleTransitionPair(YangModelCmHandle currentYangModelCmHandle,
+ YangModelCmHandle targetYangModelCmHandle
+) {}
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
-import lombok.Getter;
-import lombok.NoArgsConstructor;
import lombok.RequiredArgsConstructor;
-import lombok.Setter;
import lombok.extern.slf4j.Slf4j;
import org.onap.cps.ncmp.api.inventory.models.CmHandleState;
import org.onap.cps.ncmp.api.inventory.models.CompositeState;
import org.onap.cps.ncmp.impl.inventory.models.YangModelCmHandle;
import org.springframework.stereotype.Service;
+
@Slf4j
@Service
@RequiredArgsConstructor
@Override
@Timed(value = "cps.ncmp.cmhandle.state.update.batch",
description = "Time taken to update a batch of cm handle states")
- public void updateCmHandleStateBatch(final Map<YangModelCmHandle, CmHandleState> cmHandleStatePerCmHandle) {
+ public void updateCmHandleStateBatch(final Map<YangModelCmHandle, CmHandleState> targetCmHandleStatePerCmHandle) {
final Collection<CmHandleTransitionPair> cmHandleTransitionPairs =
- prepareCmHandleTransitionBatch(cmHandleStatePerCmHandle);
+ prepareCmHandleTransitionBatch(targetCmHandleStatePerCmHandle);
persistCmHandleBatch(cmHandleTransitionPairs);
lcmEventsHelper.sendLcmEventBatchAsynchronously(cmHandleTransitionPairs);
cmHandleStateMonitor.updateCmHandleStateMetrics(cmHandleTransitionPairs);
@Override
public void initiateStateAdvised(final Collection<YangModelCmHandle> yangModelCmHandles) {
- final Map<YangModelCmHandle, CmHandleState> cmHandleStatePerCmHandle = new HashMap<>(yangModelCmHandles.size());
+ final Map<YangModelCmHandle, CmHandleState> targetCmHandleStatePerCmHandle
+ = new HashMap<>(yangModelCmHandles.size());
for (final YangModelCmHandle yangModelCmHandle : yangModelCmHandles) {
- cmHandleStatePerCmHandle.put(yangModelCmHandle, ADVISED);
+ targetCmHandleStatePerCmHandle.put(yangModelCmHandle, ADVISED);
}
- updateCmHandleStateBatch(cmHandleStatePerCmHandle);
+ updateCmHandleStateBatch(targetCmHandleStatePerCmHandle);
}
private Collection<CmHandleTransitionPair> prepareCmHandleTransitionBatch(
- final Map<YangModelCmHandle, CmHandleState> cmHandleStatePerCmHandle) {
- final List<CmHandleTransitionPair> cmHandleTransitionPairs = new ArrayList<>(cmHandleStatePerCmHandle.size());
- cmHandleStatePerCmHandle.forEach((yangModelCmHandle, targetCmHandleState) -> {
-
- final CompositeState compositeState = yangModelCmHandle.getCompositeState();
-
- if (isCompositeStateSame(compositeState, targetCmHandleState)) {
- log.debug("CmHandle with id : {} already in state : {}", yangModelCmHandle.getId(),
- targetCmHandleState);
+ final Map<YangModelCmHandle, CmHandleState> targetCmHandleStatePerCmHandle) {
+ final List<CmHandleTransitionPair> cmHandleTransitionPairs
+ = new ArrayList<>(targetCmHandleStatePerCmHandle.size());
+ targetCmHandleStatePerCmHandle.forEach((yangModelCmHandle, targetCmHandleState) -> {
+ final CompositeState currentCmHandleState = yangModelCmHandle.getCompositeState();
+ if (isCompositeStateSame(currentCmHandleState, targetCmHandleState)) {
+ log.debug("CmHandle: {} already in state: {}", yangModelCmHandle.getId(), targetCmHandleState);
} else {
- final CmHandleTransitionPair cmHandleTransitionPair = new CmHandleTransitionPair();
- cmHandleTransitionPair.setCurrentYangModelCmHandle(YangModelCmHandle.deepCopyOf(yangModelCmHandle));
- updateToSpecifiedCmHandleState(yangModelCmHandle, targetCmHandleState);
- cmHandleTransitionPair.setTargetYangModelCmHandle(yangModelCmHandle);
+ final YangModelCmHandle oldYangModelCmHandle = YangModelCmHandle.deepCopyOf(yangModelCmHandle);
+ updateCmHandleState(yangModelCmHandle, targetCmHandleState);
+ final CmHandleTransitionPair cmHandleTransitionPair = new CmHandleTransitionPair(
+ oldYangModelCmHandle, yangModelCmHandle);
cmHandleTransitionPairs.add(cmHandleTransitionPair);
}
});
final Map<String, CompositeState> compositeStatePerCmHandleId = new LinkedHashMap<>();
cmHandleTransitionPairs.forEach(cmHandleTransitionPair -> {
- if (isNew(cmHandleTransitionPair.getCurrentYangModelCmHandle().getCompositeState())) {
- newCmHandles.add(cmHandleTransitionPair.getTargetYangModelCmHandle());
- } else if (!isDeleted(cmHandleTransitionPair.getTargetYangModelCmHandle().getCompositeState())) {
- compositeStatePerCmHandleId.put(cmHandleTransitionPair.getTargetYangModelCmHandle().getId(),
- cmHandleTransitionPair.getTargetYangModelCmHandle().getCompositeState());
+ if (isNew(cmHandleTransitionPair.currentYangModelCmHandle().getCompositeState())) {
+ newCmHandles.add(cmHandleTransitionPair.targetYangModelCmHandle());
+ } else if (!isDeleted(cmHandleTransitionPair.targetYangModelCmHandle().getCompositeState())) {
+ compositeStatePerCmHandleId.put(cmHandleTransitionPair.targetYangModelCmHandle().getId(),
+ cmHandleTransitionPair.targetYangModelCmHandle().getCompositeState());
}
});
-
inventoryPersistence.saveCmHandleBatch(newCmHandles);
inventoryPersistence.saveCmHandleStateBatch(compositeStatePerCmHandleId);
-
logCmHandleStateChanges(cmHandleTransitionPairs);
}
- private void updateToSpecifiedCmHandleState(final YangModelCmHandle yangModelCmHandle,
- final CmHandleState targetCmHandleState) {
-
+ private void updateCmHandleState(final YangModelCmHandle yangModelCmHandle,
+ final CmHandleState targetCmHandleState) {
if (READY == targetCmHandleState) {
- setInitialStates(yangModelCmHandle);
+ setInitialState(yangModelCmHandle);
} else if (ADVISED == targetCmHandleState) {
if (yangModelCmHandle.getCompositeState() == null) {
registerNewCmHandle(yangModelCmHandle);
}
}
- private void setInitialStates(final YangModelCmHandle yangModelCmHandle) {
+ private void setInitialState(final YangModelCmHandle yangModelCmHandle) {
CompositeStateUtils.setInitialDataStoreSyncState(yangModelCmHandle.getCompositeState());
CompositeStateUtils.setCompositeState(READY, yangModelCmHandle.getCompositeState());
}
private static void logCmHandleStateChanges(final Collection<CmHandleTransitionPair> cmHandleTransitionPairs) {
cmHandleTransitionPairs.stream()
- .map(CmHandleTransitionPair::getTargetYangModelCmHandle)
+ .map(CmHandleTransitionPair::targetYangModelCmHandle)
.forEach(yangModelCmHandle -> log.debug("{} is now in {} state", yangModelCmHandle.getId(),
yangModelCmHandle.getCompositeState().getCmHandleState().name()));
}
- @Getter
- @Setter
- @NoArgsConstructor
- public static class CmHandleTransitionPair {
- private YangModelCmHandle currentYangModelCmHandle;
- private YangModelCmHandle targetYangModelCmHandle;
- }
}
import org.onap.cps.ncmp.events.lcm.v1.LcmEvent;
import org.onap.cps.ncmp.events.lcm.v1.LcmEventHeader;
import org.onap.cps.ncmp.impl.inventory.models.YangModelCmHandle;
-import org.onap.cps.ncmp.impl.inventory.sync.lcm.LcmEventsCmHandleStateHandlerImpl.CmHandleTransitionPair;
import org.onap.cps.ncmp.impl.utils.YangDataConverter;
import org.springframework.scheduling.annotation.Async;
import org.springframework.stereotype.Service;
@Async("notificationExecutor")
public void sendLcmEventBatchAsynchronously(final Collection<CmHandleTransitionPair> cmHandleTransitionPairs) {
cmHandleTransitionPairs.forEach(cmHandleTransitionPair -> sendLcmEvent(
- toNcmpServiceCmHandle(cmHandleTransitionPair.getTargetYangModelCmHandle()),
- toNcmpServiceCmHandle(cmHandleTransitionPair.getCurrentYangModelCmHandle())));
+ toNcmpServiceCmHandle(cmHandleTransitionPair.currentYangModelCmHandle()),
+ toNcmpServiceCmHandle(cmHandleTransitionPair.targetYangModelCmHandle())
+ ));
}
- /**
- * Sends LcmEvent asynchronously.
- *
- * @param targetNcmpServiceCmHandle target Ncmp Service Cm Handle
- * @param existingNcmpServiceCmHandle existing Ncmp Service Cm Handle
- */
- @Async("notificationExecutor")
- public void sendLcmEventAsynchronously(final NcmpServiceCmHandle targetNcmpServiceCmHandle,
- final NcmpServiceCmHandle existingNcmpServiceCmHandle) {
- sendLcmEvent(targetNcmpServiceCmHandle, existingNcmpServiceCmHandle);
- }
-
- private void sendLcmEvent(final NcmpServiceCmHandle targetNcmpServiceCmHandle,
- final NcmpServiceCmHandle existingNcmpServiceCmHandle) {
+ private void sendLcmEvent(final NcmpServiceCmHandle currentNcmpServiceCmHandle,
+ final NcmpServiceCmHandle targetNcmpServiceCmHandle) {
final String cmHandleId = targetNcmpServiceCmHandle.getCmHandleId();
final LcmEventHeader lcmEventHeader =
- lcmEventsProducerHelper.populateLcmEventHeader(cmHandleId, targetNcmpServiceCmHandle,
- existingNcmpServiceCmHandle);
+ lcmEventsProducerHelper.createLcmEventHeader(cmHandleId, currentNcmpServiceCmHandle,
+ targetNcmpServiceCmHandle
+ );
final LcmEvent lcmEvent =
- lcmEventsProducerHelper.populateLcmEvent(cmHandleId, targetNcmpServiceCmHandle,
- existingNcmpServiceCmHandle);
+ lcmEventsProducerHelper.createLcmEvent(cmHandleId, currentNcmpServiceCmHandle,
+ targetNcmpServiceCmHandle
+ );
lcmEventsProducer.sendLcmEvent(cmHandleId, lcmEvent, lcmEventHeader);
}
private final LcmEventHeaderMapper lcmEventHeaderMapper;
/**
- * Populate Lifecycle Management Event.
+ * Create Lifecycle Management Event.
*
* @param cmHandleId cm handle identifier
+ * @param currentNcmpServiceCmHandle current ncmp service cmhandle
* @param targetNcmpServiceCmHandle target ncmp service cmhandle
- * @param existingNcmpServiceCmHandle existing ncmp service cmhandle
* @return Populated LcmEvent
*/
- public LcmEvent populateLcmEvent(final String cmHandleId, final NcmpServiceCmHandle targetNcmpServiceCmHandle,
- final NcmpServiceCmHandle existingNcmpServiceCmHandle) {
- return createLcmEvent(cmHandleId, targetNcmpServiceCmHandle, existingNcmpServiceCmHandle);
+ public LcmEvent createLcmEvent(final String cmHandleId,
+ final NcmpServiceCmHandle currentNcmpServiceCmHandle,
+ final NcmpServiceCmHandle targetNcmpServiceCmHandle) {
+ final LcmEventType lcmEventType =
+ determineEventType(currentNcmpServiceCmHandle, targetNcmpServiceCmHandle);
+ final LcmEvent lcmEvent = createLcmEventWithHeaderDetails(cmHandleId, lcmEventType);
+ final Event event = new Event();
+ event.setCmHandleId(cmHandleId);
+ event.setAlternateId(targetNcmpServiceCmHandle.getAlternateId());
+ event.setModuleSetTag(targetNcmpServiceCmHandle.getModuleSetTag());
+ event.setDataProducerIdentifier(targetNcmpServiceCmHandle.getDataProducerIdentifier());
+ final CmHandleValuesHolder cmHandleValuesHolder =
+ determineEventValues(lcmEventType, currentNcmpServiceCmHandle, targetNcmpServiceCmHandle);
+ event.setOldValues(cmHandleValuesHolder.getOldValues());
+ event.setNewValues(cmHandleValuesHolder.getNewValues());
+ lcmEvent.setEvent(event);
+ return lcmEvent;
}
/**
- * Populate Lifecycle Management Event Header.
+ * Create Lifecycle Management Event Header.
*
- * @param cmHandleId cm handle identifier
- * @param targetNcmpServiceCmHandle target ncmp service cmhandle
- * @param existingNcmpServiceCmHandle existing ncmp service cmhandle
+ * @param cmHandleId cm handle identifier
+ * @param currentNcmpServiceCmHandle current ncmp service cmhandle
+ * @param targetNcmpServiceCmHandle target ncmp service cmhandle
* @return Populated LcmEventHeader
*/
- public LcmEventHeader populateLcmEventHeader(final String cmHandleId,
- final NcmpServiceCmHandle targetNcmpServiceCmHandle,
- final NcmpServiceCmHandle existingNcmpServiceCmHandle) {
- return createLcmEventHeader(cmHandleId, targetNcmpServiceCmHandle, existingNcmpServiceCmHandle);
- }
-
- private LcmEvent createLcmEvent(final String cmHandleId, final NcmpServiceCmHandle targetNcmpServiceCmHandle,
- final NcmpServiceCmHandle existingNcmpServiceCmHandle) {
- final LcmEventType lcmEventType =
- determineEventType(targetNcmpServiceCmHandle, existingNcmpServiceCmHandle);
- final LcmEvent lcmEvent = lcmEventHeader(cmHandleId, lcmEventType);
- lcmEvent.setEvent(
- lcmEventPayload(cmHandleId, targetNcmpServiceCmHandle, existingNcmpServiceCmHandle, lcmEventType));
- return lcmEvent;
- }
-
- private LcmEventHeader createLcmEventHeader(final String cmHandleId,
- final NcmpServiceCmHandle targetNcmpServiceCmHandle,
- final NcmpServiceCmHandle existingNcmpServiceCmHandle) {
+ public LcmEventHeader createLcmEventHeader(final String cmHandleId,
+ final NcmpServiceCmHandle currentNcmpServiceCmHandle,
+ final NcmpServiceCmHandle targetNcmpServiceCmHandle) {
final LcmEventType lcmEventType =
- determineEventType(targetNcmpServiceCmHandle, existingNcmpServiceCmHandle);
- final LcmEvent lcmEventWithHeaderInformation = lcmEventHeader(cmHandleId, lcmEventType);
- return lcmEventHeaderMapper.toLcmEventHeader(lcmEventWithHeaderInformation);
+ determineEventType(currentNcmpServiceCmHandle, targetNcmpServiceCmHandle);
+ final LcmEvent lcmEventWithHeaderDetails = createLcmEventWithHeaderDetails(cmHandleId, lcmEventType);
+ return lcmEventHeaderMapper.toLcmEventHeader(lcmEventWithHeaderDetails);
}
- private static LcmEventType determineEventType(final NcmpServiceCmHandle targetNcmpServiceCmHandle,
- final NcmpServiceCmHandle existingNcmpServiceCmHandle) {
+ private static LcmEventType determineEventType(final NcmpServiceCmHandle currentNcmpServiceCmHandle,
+ final NcmpServiceCmHandle targetNcmpServiceCmHandle) {
- if (existingNcmpServiceCmHandle.getCompositeState() == null) {
+ if (currentNcmpServiceCmHandle.getCompositeState() == null) {
return CREATE;
} else if (targetNcmpServiceCmHandle.getCompositeState().getCmHandleState() == DELETED) {
return DELETE;
return UPDATE;
}
- private static CmHandleValuesHolder determineEventValues(
- final NcmpServiceCmHandle targetNcmpServiceCmHandle, final NcmpServiceCmHandle existingNcmpServiceCmHandle,
- final LcmEventType lcmEventType) {
-
+ private static CmHandleValuesHolder determineEventValues(final LcmEventType lcmEventType,
+ final NcmpServiceCmHandle currentNcmpServiceCmHandle,
+ final NcmpServiceCmHandle targetNcmpServiceCmHandle) {
if (CREATE == lcmEventType) {
return determineCreateEventValues(targetNcmpServiceCmHandle);
} else if (UPDATE == lcmEventType) {
- return determineUpdateEventValues(targetNcmpServiceCmHandle, existingNcmpServiceCmHandle);
+ return determineUpdateEventValues(targetNcmpServiceCmHandle, currentNcmpServiceCmHandle);
}
return new CmHandleValuesHolder();
}
- private Event lcmEventPayload(final String eventCorrelationId, final NcmpServiceCmHandle targetNcmpServiceCmHandle,
- final NcmpServiceCmHandle existingNcmpServiceCmHandle, final LcmEventType lcmEventType) {
- final Event event = new Event();
- event.setCmHandleId(eventCorrelationId);
- event.setAlternateId(targetNcmpServiceCmHandle.getAlternateId());
- event.setModuleSetTag(targetNcmpServiceCmHandle.getModuleSetTag());
- event.setDataProducerIdentifier(targetNcmpServiceCmHandle.getDataProducerIdentifier());
- final CmHandleValuesHolder cmHandleValuesHolder =
- determineEventValues(targetNcmpServiceCmHandle, existingNcmpServiceCmHandle,
- lcmEventType);
- event.setOldValues(cmHandleValuesHolder.getOldValues());
- event.setNewValues(cmHandleValuesHolder.getNewValues());
-
- return event;
- }
-
- private LcmEvent lcmEventHeader(final String eventCorrelationId, final LcmEventType lcmEventType) {
+ private LcmEvent createLcmEventWithHeaderDetails(final String eventCorrelationId, final LcmEventType lcmEventType) {
final LcmEvent lcmEvent = new LcmEvent();
lcmEvent.setEventId(UUID.randomUUID().toString());
lcmEvent.setEventCorrelationId(eventCorrelationId);
}
- private static CmHandleValuesHolder determineCreateEventValues(
- final NcmpServiceCmHandle ncmpServiceCmHandle) {
+ private static CmHandleValuesHolder determineCreateEventValues(final NcmpServiceCmHandle ncmpServiceCmHandle) {
final CmHandleValuesHolder cmHandleValuesHolder = new CmHandleValuesHolder();
cmHandleValuesHolder.setNewValues(new Values());
cmHandleValuesHolder.getNewValues().setDataSyncEnabled(getDataSyncEnabledFlag(ncmpServiceCmHandle));
return cmHandleValuesHolder;
}
- private static CmHandleValuesHolder determineUpdateEventValues(
- final NcmpServiceCmHandle targetNcmpServiceCmHandle,
- final NcmpServiceCmHandle existingNcmpServiceCmHandle) {
-
+ private static CmHandleValuesHolder determineUpdateEventValues(final NcmpServiceCmHandle targetNcmpServiceCmHandle,
+ final NcmpServiceCmHandle currentNcmpServiceCmHandle) {
final boolean hasDataSyncFlagEnabledChanged =
- hasDataSyncEnabledFlagChanged(targetNcmpServiceCmHandle, existingNcmpServiceCmHandle);
+ hasDataSyncEnabledFlagChanged(currentNcmpServiceCmHandle, targetNcmpServiceCmHandle);
final boolean hasCmHandleStateChanged =
- hasCmHandleStateChanged(targetNcmpServiceCmHandle, existingNcmpServiceCmHandle);
+ hasCmHandleStateChanged(currentNcmpServiceCmHandle, targetNcmpServiceCmHandle);
final boolean arePublicCmHandlePropertiesEqual =
- arePublicCmHandlePropertiesEqual(targetNcmpServiceCmHandle.getPublicProperties(),
- existingNcmpServiceCmHandle.getPublicProperties());
+ arePublicCmHandlePropertiesEqual(currentNcmpServiceCmHandle.getPublicProperties(),
+ targetNcmpServiceCmHandle.getPublicProperties()
+ );
final CmHandleValuesHolder cmHandleValuesHolder = new CmHandleValuesHolder();
}
if (hasDataSyncFlagEnabledChanged) {
- setDataSyncEnabledFlag(targetNcmpServiceCmHandle, existingNcmpServiceCmHandle, cmHandleValuesHolder);
+ setDataSyncEnabledFlag(currentNcmpServiceCmHandle, targetNcmpServiceCmHandle, cmHandleValuesHolder);
}
if (hasCmHandleStateChanged) {
- setCmHandleStateChange(targetNcmpServiceCmHandle, existingNcmpServiceCmHandle, cmHandleValuesHolder);
+ setCmHandleStateChange(currentNcmpServiceCmHandle, targetNcmpServiceCmHandle, cmHandleValuesHolder);
}
if (!arePublicCmHandlePropertiesEqual) {
- setPublicCmHandlePropertiesChange(targetNcmpServiceCmHandle, existingNcmpServiceCmHandle,
- cmHandleValuesHolder);
+ setPublicCmHandlePropertiesChange(currentNcmpServiceCmHandle, targetNcmpServiceCmHandle,
+ cmHandleValuesHolder);
}
return cmHandleValuesHolder;
}
- private static void setDataSyncEnabledFlag(final NcmpServiceCmHandle targetNcmpServiceCmHandle,
- final NcmpServiceCmHandle existingNcmpServiceCmHandle,
+ private static void setDataSyncEnabledFlag(final NcmpServiceCmHandle currentNcmpServiceCmHandle,
+ final NcmpServiceCmHandle targetNcmpServiceCmHandle,
final CmHandleValuesHolder cmHandleValuesHolder) {
-
- cmHandleValuesHolder.getOldValues().setDataSyncEnabled(getDataSyncEnabledFlag(existingNcmpServiceCmHandle));
+ cmHandleValuesHolder.getOldValues().setDataSyncEnabled(getDataSyncEnabledFlag(currentNcmpServiceCmHandle));
cmHandleValuesHolder.getNewValues().setDataSyncEnabled(getDataSyncEnabledFlag(targetNcmpServiceCmHandle));
}
- private static void setCmHandleStateChange(final NcmpServiceCmHandle targetNcmpServiceCmHandle,
- final NcmpServiceCmHandle existingNcmpServiceCmHandle,
+ private static void setCmHandleStateChange(final NcmpServiceCmHandle currentNcmpServiceCmHandle,
+ final NcmpServiceCmHandle targetNcmpServiceCmHandle,
final CmHandleValuesHolder cmHandleValuesHolder) {
cmHandleValuesHolder.getOldValues()
- .setCmHandleState(mapCmHandleStateToLcmEventCmHandleState(existingNcmpServiceCmHandle));
+ .setCmHandleState(mapCmHandleStateToLcmEventCmHandleState(currentNcmpServiceCmHandle));
cmHandleValuesHolder.getNewValues()
.setCmHandleState(mapCmHandleStateToLcmEventCmHandleState(targetNcmpServiceCmHandle));
}
- private static void setPublicCmHandlePropertiesChange(final NcmpServiceCmHandle targetNcmpServiceCmHandle,
- final NcmpServiceCmHandle existingNcmpServiceCmHandle,
+ private static void setPublicCmHandlePropertiesChange(final NcmpServiceCmHandle currentNcmpServiceCmHandle,
+ final NcmpServiceCmHandle targetNcmpServiceCmHandle,
final CmHandleValuesHolder cmHandleValuesHolder) {
final Map<String, Map<String, String>> publicCmHandlePropertiesDifference =
- getPublicCmHandlePropertiesDifference(targetNcmpServiceCmHandle.getPublicProperties(),
- existingNcmpServiceCmHandle.getPublicProperties());
+ getPublicCmHandlePropertiesDifference(currentNcmpServiceCmHandle.getPublicProperties(),
+ targetNcmpServiceCmHandle.getPublicProperties()
+ );
cmHandleValuesHolder.getOldValues()
.setCmHandleProperties(List.of(publicCmHandlePropertiesDifference.get("oldValues")));
cmHandleValuesHolder.getNewValues()
return ncmpServiceCmHandle.getCompositeState().getDataSyncEnabled();
}
- private static boolean hasDataSyncEnabledFlagChanged(final NcmpServiceCmHandle targetNcmpServiceCmHandle,
- final NcmpServiceCmHandle existingNcmpServiceCmHandle) {
-
+ private static boolean hasDataSyncEnabledFlagChanged(final NcmpServiceCmHandle currentNcmpServiceCmHandle,
+ final NcmpServiceCmHandle targetNcmpServiceCmHandle) {
+ final Boolean currentDataSyncFlag = currentNcmpServiceCmHandle.getCompositeState().getDataSyncEnabled();
final Boolean targetDataSyncFlag = targetNcmpServiceCmHandle.getCompositeState().getDataSyncEnabled();
- final Boolean existingDataSyncFlag = existingNcmpServiceCmHandle.getCompositeState().getDataSyncEnabled();
if (targetDataSyncFlag == null) {
- return existingDataSyncFlag != null;
+ return currentDataSyncFlag != null;
}
- return !targetDataSyncFlag.equals(existingDataSyncFlag);
+ return !targetDataSyncFlag.equals(currentDataSyncFlag);
}
- private static boolean hasCmHandleStateChanged(final NcmpServiceCmHandle targetNcmpServiceCmHandle,
- final NcmpServiceCmHandle existingNcmpServiceCmHandle) {
-
+ private static boolean hasCmHandleStateChanged(final NcmpServiceCmHandle currentNcmpServiceCmHandle,
+ final NcmpServiceCmHandle targetNcmpServiceCmHandle) {
return targetNcmpServiceCmHandle.getCompositeState().getCmHandleState()
- != existingNcmpServiceCmHandle.getCompositeState().getCmHandleState();
+ != currentNcmpServiceCmHandle.getCompositeState().getCmHandleState();
}
- private static boolean arePublicCmHandlePropertiesEqual(final Map<String, String> targetCmHandleProperties,
- final Map<String, String> existingCmHandleProperties) {
- if (targetCmHandleProperties.size() != existingCmHandleProperties.size()) {
+ private static boolean arePublicCmHandlePropertiesEqual(final Map<String, String> currentCmHandleProperties,
+ final Map<String, String> targetCmHandleProperties) {
+ if (targetCmHandleProperties.size() != currentCmHandleProperties.size()) {
return false;
}
-
- return targetCmHandleProperties.equals(existingCmHandleProperties);
+ return targetCmHandleProperties.equals(currentCmHandleProperties);
}
private static Map<String, Map<String, String>> getPublicCmHandlePropertiesDifference(
- final Map<String, String> targetCmHandleProperties, final Map<String, String> existingCmHandleProperties) {
+ final Map<String, String> currentCmHandleProperties,
+ final Map<String, String> targetCmHandleProperties) {
final Map<String, Map<String, String>> oldAndNewPropertiesDifferenceMap = new HashMap<>(2);
final MapDifference<String, String> cmHandlePropertiesDifference =
- Maps.difference(targetCmHandleProperties, existingCmHandleProperties);
+ Maps.difference(targetCmHandleProperties, currentCmHandleProperties);
- final Map<String, String> newValues = new HashMap<>(cmHandlePropertiesDifference.entriesOnlyOnLeft());
final Map<String, String> oldValues = new HashMap<>(cmHandlePropertiesDifference.entriesOnlyOnRight());
+ final Map<String, String> newValues = new HashMap<>(cmHandlePropertiesDifference.entriesOnlyOnLeft());
cmHandlePropertiesDifference.entriesDiffering().keySet().forEach(cmHandlePropertyName -> {
- oldValues.put(cmHandlePropertyName, existingCmHandleProperties.get(cmHandlePropertyName));
+ oldValues.put(cmHandlePropertyName, currentCmHandleProperties.get(cmHandlePropertyName));
newValues.put(cmHandlePropertyName, targetCmHandleProperties.get(cmHandlePropertyName));
});
return oldAndNewPropertiesDifferenceMap;
}
-
@NoArgsConstructor
@Getter
@Setter
static class CmHandleValuesHolder {
-
private Values oldValues;
private Values newValues;
}
-
}
--- /dev/null
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2025 OpenInfra Foundation Europe. All rights reserved.
+ * ================================================================================
+ * 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.datajobs.models
+
+import spock.lang.Specification
+
+class DataJobRequestSpec extends Specification {
+
+ def dataJobMetaData = new DataJobMetadata('some destination', 'some accept type', 'some content type')
+ def writeOperation = new WriteOperation('some path', 'some operation', 'some id', 'some value')
+ def dataJobWriteRequest = new DataJobWriteRequest([writeOperation])
+
+ def objectUnderTest = new DataJobRequest(dataJobMetaData, dataJobWriteRequest)
+
+ //TODO This class is only used for a test Controller. Maybe it can be removed, see https://lf-onap.atlassian.net/browse/CPS-3062
+ def 'a Data Job Request.'() {
+ expect: 'a data job request consisting out of meta data and a write request'
+ assert objectUnderTest.dataJobMetadata == dataJobMetaData
+ assert objectUnderTest.dataJobWriteRequest == dataJobWriteRequest
+ }
+}
--- /dev/null
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2025 OpenInfra Foundation Europe. All rights reserved.
+ * ================================================================================
+ * 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.exceptions
+
+import spock.lang.Specification
+
+class ServerNcmpExceptionSpec extends Specification {
+
+ def objectUnderTest = new ServerNcmpException('my message', 'my details')
+
+ def 'A Server ncmp exception.'() {
+ expect: 'the exception has the correct message'
+ objectUnderTest.message == 'my message'
+ and: 'the exception has the correct details'
+ objectUnderTest.details == 'my details'
+ }
+
+}
/*
* ============LICENSE_START=======================================================
* Copyright (C) 2022 Bell Canada
- * Modifications Copyright (C) 2022-2023 Nordix Foundation.
+ * Modifications Copyright (C) 2022-2025 OpenInfra Foundation Europe.
* ================================================================================
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
package org.onap.cps.ncmp.api.inventory.models
-
-import org.onap.cps.ncmp.api.inventory.DataStoreSyncState
import org.onap.cps.api.model.DataNode
import org.onap.cps.impl.DataNodeBuilder
+import org.onap.cps.ncmp.api.inventory.DataStoreSyncState
import spock.lang.Specification
import java.time.OffsetDateTime
def formattedDateAndTime = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZ")
.format(OffsetDateTime.of(2022, 12, 31, 20, 30, 40, 1, ZoneOffset.UTC))
- def static cmHandleId = 'myHandle1'
- def static cmHandleXpath = "/dmi-registry/cm-handles[@id='${cmHandleId}/state']"
- def static stateDataNodes = [new DataNodeBuilder().withXpath("/dmi-registry/cm-handles[@id='${cmHandleId}']/state/lock-reason")
+ def cmHandleId = 'myHandle1'
+ def cmHandleXpath = "/dmi-registry/cm-handles[@id='${cmHandleId}/state']"
+ def stateDataNodes = [new DataNodeBuilder().withXpath("/dmi-registry/cm-handles[@id='${cmHandleId}']/state/lock-reason")
.withLeaves(['reason': 'MODULE_SYNC_FAILED', 'details': 'lock details']).build(),
new DataNodeBuilder().withXpath("/dmi-registry/cm-handles[@id='${cmHandleId}']/state/datastores")
.withChildDataNodes(Arrays.asList(new DataNodeBuilder()
.withXpath("/dmi-registry/cm-handles[@id='${cmHandleId}']/state/datastores/operational")
.withLeaves(['sync-state': 'UNSYNCHRONIZED']).build())).build()]
- def static cmHandleDataNode = new DataNode(xpath: cmHandleXpath, childDataNodes: stateDataNodes, leaves: ['cm-handle-state': 'ADVISED'])
+ def cmHandleDataNode = new DataNode(xpath: cmHandleXpath, childDataNodes: stateDataNodes, leaves: ['cm-handle-state': 'READY'])
+
+ def objectUnderTest = new CompositeStateBuilder()
- def "Composite State Specification"() {
- when: 'using composite state builder '
- def compositeState = new CompositeStateBuilder().withCmHandleState(CmHandleState.ADVISED)
+ def 'Composite State Specification.'() {
+ when: 'using composite state builder'
+ def compositeState = objectUnderTest.withCmHandleState(CmHandleState.ADVISED)
.withLockReason(LockReasonCategory.MODULE_SYNC_FAILED,"").withOperationalDataStores(DataStoreSyncState.UNSYNCHRONIZED,
formattedDateAndTime.toString()).withLastUpdatedTime(formattedDateAndTime).build()
then: 'it matches expected cm handle state and data store sync state'
assert compositeState.dataStores.operationalDataStore.dataStoreSyncState == DataStoreSyncState.UNSYNCHRONIZED
}
- def "Build composite state from DataNode "() {
- given: "a Data Node "
- new DataNode(leaves: ['cm-handle-state': 'ADVISED'])
+ def 'Build composite state from DataNode.'() {
when: 'build from data node function is invoked'
- def compositeState = new CompositeStateBuilder().fromDataNode(cmHandleDataNode).build()
+ def compositeState = objectUnderTest.fromDataNode(cmHandleDataNode).build()
then: 'it matches expected state model as JSON'
- assert compositeState.cmHandleState == CmHandleState.ADVISED
+ assert compositeState.cmHandleState == CmHandleState.READY
}
def 'CompositeStateBuilder build'() {
given: 'A CompositeStateBuilder with all private fields set'
- def finalCompositeStateBuilder = new CompositeStateBuilder()
+ def finalCompositeStateBuilder = objectUnderTest
.withCmHandleState(CmHandleState.ADVISED)
.withLastUpdatedTime(formattedDateAndTime.toString())
.withLockReason(LockReasonCategory.MODULE_SYNC_FAILED, 'locked details')
assert result.getDataStores().getOperationalDataStore().getLastSyncTime() == formattedDateAndTime
}
+ def 'Get lock reason without leaf for reason.'() {
+ given: 'a data node with details but no reason'
+ def dataNodeWithJustDetailsLeaf = new DataNode(leaves:[details:'my details'])
+ when: 'convert it to a lock reason'
+ def result = getObjectUnderTest().toLockReason(dataNodeWithJustDetailsLeaf)
+ then: 'the result has no reason category'
+ assert result.lockReasonCategory == null
+ and: 'the result has the correct details'
+ assert result.details == 'my details'
+ }
+
}
import com.hazelcast.map.IMap
import org.springframework.beans.factory.annotation.Autowired
import org.springframework.boot.test.context.SpringBootTest
-import org.springframework.test.context.ContextConfiguration;
-import spock.lang.Specification;
+import spock.lang.Specification
@SpringBootTest(classes = [AdminCacheConfig])
class AdminCacheConfigSpec extends Specification {
import org.springframework.beans.factory.annotation.Autowired
import org.springframework.beans.factory.annotation.Qualifier
import org.springframework.boot.test.context.SpringBootTest
-import org.springframework.test.context.ContextConfiguration
import spock.lang.Specification
@SpringBootTest(classes = [AlternateIdCacheConfig])
+++ /dev/null
-/*
- * ============LICENSE_START=======================================================
- * Copyright (C) 2025 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.impl.datajobs.subscription.cache
-
-import com.hazelcast.core.Hazelcast
-import com.hazelcast.map.IMap
-import org.onap.cps.ncmp.impl.datajobs.subscription.models.DmiCmSubscriptionDetails
-import org.springframework.beans.factory.annotation.Autowired
-import org.springframework.boot.test.context.SpringBootTest
-import org.springframework.test.context.ContextConfiguration
-import spock.lang.Specification
-
-@SpringBootTest(classes = [CmSubscriptionConfig])
-class CmSubscriptionConfigSpec extends Specification {
-
- @Autowired
- IMap<String, Map<String, DmiCmSubscriptionDetails>> cmNotificationSubscriptionCache
-
- def cleanupSpec() {
- Hazelcast.getHazelcastInstanceByName('cps-and-ncmp-hazelcast-instance-test-config').shutdown()
- }
-
- def 'Hazelcast cache for cm subscriptions.'() {
- expect: 'system is able to create an instance of the cm subscription cache'
- assert null != cmNotificationSubscriptionCache
- and: 'there is at least 1 instance'
- assert Hazelcast.allHazelcastInstances.size() > 0
- and: 'Hazelcast cache instance for cm subscriptions present'
- assert Hazelcast.getHazelcastInstanceByName('cps-and-ncmp-hazelcast-instance-test-config').getMap('cmNotificationSubscriptionCache') != null
- }
-}
import com.fasterxml.jackson.databind.ObjectMapper
import com.hazelcast.map.IMap
import org.onap.cps.api.CpsDataService
-import org.onap.cps.ncmp.api.inventory.models.NcmpServiceCmHandle
import org.onap.cps.api.exceptions.DataNodeNotFoundException
import org.onap.cps.api.exceptions.DataValidationException
import org.onap.cps.api.model.DataNode
import org.onap.cps.impl.DataNodeBuilder
+import org.onap.cps.ncmp.api.inventory.models.NcmpServiceCmHandle
import org.onap.cps.ncmp.impl.inventory.models.YangModelCmHandle
import org.onap.cps.ncmp.impl.inventory.sync.lcm.LcmEventsHelper
import org.onap.cps.utils.ContentType
assert args[3].contains('New Data Producer Identifier')
}
and: 'LCM event is sent'
- 1 * mockLcmEventsHelper.sendLcmEventAsynchronously(_, _) >> { args ->
- assert args[0].dataProducerIdentifier == 'New Data Producer Identifier'
- }
+ 1 * mockLcmEventsHelper.sendLcmEventBatchAsynchronously({ cmHandleTransitionPairs ->
+ assert cmHandleTransitionPairs[0].targetYangModelCmHandle.dataProducerIdentifier == 'New Data Producer Identifier'
+ })
where: 'the following scenarios are used'
scenario | oldDataProducerIdentifier
'null to something' | null
objectUnderTest.updateDataProducerIdentifier(existingCmHandleDataNode, ncmpServiceCmHandle)
then: 'the update node leaves method is not invoked'
0 * mockCpsDataService.updateNodeLeaves(*_)
- and: 'LCM event is not sent'
- 0 * mockLcmEventsHelper.sendLcmEventAsynchronously(*_)
+ and: 'No LCM events are sent'
+ 0 * mockLcmEventsHelper.sendLcmEventBatchAsynchronously(*_)
and: 'debug information is logged'
def loggingEvent = logger.list[0]
assert loggingEvent.level == Level.DEBUG
assert args[3].contains('newDataProducerIdentifier')
}
and: 'LCM event is sent'
- 1 * mockLcmEventsHelper.sendLcmEventAsynchronously(_, _) >> { args ->
- assert args[0].dataProducerIdentifier == 'newDataProducerIdentifier'
- assert args[1].dataProducerIdentifier == 'oldDataProducerIdentifier'
- }
+ 1 * mockLcmEventsHelper.sendLcmEventBatchAsynchronously( { cmHandleTransitionPairs ->
+ assert cmHandleTransitionPairs[0].targetYangModelCmHandle.dataProducerIdentifier == 'newDataProducerIdentifier'
+ assert cmHandleTransitionPairs[0].currentYangModelCmHandle.dataProducerIdentifier == 'oldDataProducerIdentifier'
+ })
and: 'correct information is logged'
def loggingEvent = logger.list[1]
assert loggingEvent.level == Level.DEBUG
objectUnderTest.updateDataProducerIdentifier(existingCmHandleDataNode, ncmpServiceCmHandle)
then: 'the update node leaves method is not invoked'
0 * mockCpsDataService.updateNodeLeaves(*_)
- and: 'LCM event is not sent'
- 0 * mockLcmEventsHelper.sendLcmEventAsynchronously(*_)
+ and: 'No LCM events are sent'
+ 0 * mockLcmEventsHelper.sendLcmEventBatchAsynchronously(*_)
and: 'warning is logged'
def lastLoggingEvent = logger.list[0]
assert lastLoggingEvent.level == Level.WARN
then: 'The call is delegated to the anchor service with teh correct parameters'
mockCpsAnchorService.deleteAnchors(NCMP_DATASPACE_NAME ,['anchor1' ,'anchor2'])
}
+
+ def 'Get Yang Model CM Handles without properties.'() {
+ given: 'the cps data service returns 2 data nodes from the DMI registry (omitting descendants)'
+ def dataNodes = [new DataNode(xpath: xpath, leaves: ['id': cmHandleId]), new DataNode(xpath: xpath2, leaves: ['id': cmHandleId2])]
+ mockCpsDataService.getDataNodesForMultipleXpaths(NCMP_DATASPACE_NAME, NCMP_DMI_REGISTRY_ANCHOR, [xpath, xpath2] , OMIT_DESCENDANTS) >> dataNodes
+ when: 'retrieving cm handles without properties'
+ def result = objectUnderTest.getYangModelCmHandlesWithoutProperties([cmHandleId, cmHandleId2])
+ then: 'The cm handles from the data service are returned'
+ assert result.size() == 2
+ assert result.id.containsAll([cmHandleId, cmHandleId2])
+ }
}
import com.fasterxml.jackson.databind.ObjectMapper
import org.onap.cps.api.exceptions.DataValidationException
import org.onap.cps.api.model.ConditionProperties
+import org.onap.cps.ncmp.api.exceptions.CmHandleNotFoundException
import org.onap.cps.ncmp.api.inventory.DataStoreSyncState
import org.onap.cps.ncmp.api.inventory.models.CmHandleQueryApiParameters
import org.onap.cps.ncmp.api.inventory.models.CmHandleQueryServiceParameters
+import org.onap.cps.ncmp.api.inventory.models.CmHandleState
import org.onap.cps.ncmp.api.inventory.models.CompositeState
import org.onap.cps.ncmp.api.inventory.models.ConditionApiProperties
import org.onap.cps.ncmp.api.inventory.models.DmiPluginRegistration
+import org.onap.cps.ncmp.api.inventory.models.LockReasonCategory
import org.onap.cps.ncmp.api.inventory.models.NcmpServiceCmHandle
import org.onap.cps.ncmp.api.inventory.models.TrustLevel
import org.onap.cps.ncmp.impl.NetworkCmProxyInventoryFacadeImpl
-import org.onap.cps.ncmp.api.inventory.models.CmHandleState
-import org.onap.cps.ncmp.api.inventory.models.LockReasonCategory
import org.onap.cps.ncmp.impl.inventory.models.YangModelCmHandle
import org.onap.cps.ncmp.impl.inventory.trustlevel.TrustLevelManager
import org.onap.cps.ncmp.impl.utils.AlternateIdMatcher
def 'Getting Yang Resources for a given #scenario'() {
when: 'yang resources is called'
objectUnderTest.getYangResourcesModuleReferences(cmHandleRef)
- then: 'alternate id matcher is called'
+ then: 'alternate id matcher can find it'
mockAlternateIdMatcher.getCmHandleId(cmHandleRef) >> 'some-cm-handle'
and: 'CPS module services is invoked for the correct cm handle'
1 * mockInventoryPersistence.getYangResourcesModuleReferences('some-cm-handle')
'Cm Handle Reference as alternate-id' | 'some-alternate-id'
}
+ def 'Getting Yang Resources with exception.'() {
+ given: 'alternate id matcher can always find a cm handle'
+ mockAlternateIdMatcher.getCmHandleId(_) >> 'some id'
+ and: 'CPS module services throws a not found exception'
+ mockInventoryPersistence.getYangResourcesModuleReferences(_) >> { throw new CmHandleNotFoundException('') }
+ when: 'attempt to get the yang resources'
+ def result = objectUnderTest.getYangResourcesModuleReferences('some id')
+ then: 'the result is an empty collection'
+ assert result == []
+ }
+
+
def 'Get a cm handle details using #scenario'() {
given: 'the system returns a yang modelled cm handle'
def dmiServiceName = 'some service name'
'Cm Handle Reference as alternate-id' | 'some-alternate-id'
}
- def 'Getting module definitions for a given #scenario'() {
+ def 'Getting module definitions by module with exception.'() {
+ given: 'alternate id matcher always finds a match'
+ mockAlternateIdMatcher.getCmHandleId(_) >> 'some id'
+ and: 'ncmp inventory persistence service throws a not found exception'
+ mockInventoryPersistence.getModuleDefinitionsByCmHandleAndModule(*_) >> { throw new CmHandleNotFoundException ('') }
+ when: 'attempt to get the module definitions'
+ def result = objectUnderTest.getModuleDefinitionsByCmHandleAndModule('some reference', 'some module', 'some revision')
+ then: 'the result is an empty collection'
+ assert result == []
+ }
+
+ def 'Getting module definitions by cm handle for a given #scenario'() {
when: 'get module definitions is performed with cm handle reference'
objectUnderTest.getModuleDefinitionsByCmHandleReference(cmHandleRef)
then: 'alternate id matcher returns some cm handle id for a given cm handle reference'
'Cm Handle Reference as alternate-id' | 'some-alternate-id'
}
+ def 'Getting module definitions by cm handle with exception.'() {
+ given: 'alternate id matcher always finds a match'
+ mockAlternateIdMatcher.getCmHandleId(_) >> 'some id'
+ and: 'ncmp inventory persistence service throws a not found exception'
+ mockInventoryPersistence.getModuleDefinitionsByCmHandleId(_) >> { throw new CmHandleNotFoundException ('') }
+ when: 'attempt to get the module definitions'
+ def result = objectUnderTest.getModuleDefinitionsByCmHandleReference('some reference')
+ then: 'the result is an empty collection'
+ assert result == []
+ }
+
def 'Execute northbound cm handle search'() {
given: 'valid CmHandleQueryApiParameters input'
def cmHandleQueryApiParameters = new CmHandleQueryApiParameters()
/*
* ============LICENSE_START=======================================================
- * Copyright (C) 2025 Nordix Foundation.
+ * Copyright (C) 2025 OpenInfra Foundation Europe. All rights reserved.
* ================================================================================
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
package org.onap.cps.ncmp.impl.inventory.sync.lcm
-import static org.onap.cps.ncmp.api.inventory.models.CmHandleState.ADVISED
-import static org.onap.cps.ncmp.api.inventory.models.CmHandleState.READY
-
import com.hazelcast.core.Hazelcast
import com.hazelcast.map.IMap
import org.onap.cps.ncmp.api.inventory.models.CompositeState
import org.onap.cps.ncmp.impl.inventory.sync.lcm.CmHandleStateMonitor.IncreasingEntryProcessor
import org.onap.cps.ncmp.utils.events.NcmpInventoryModelOnboardingFinishedEvent
import spock.lang.Shared
-import spock.lang.Specification;
+import spock.lang.Specification
+
+import static org.onap.cps.ncmp.api.inventory.models.CmHandleState.ADVISED
+import static org.onap.cps.ncmp.api.inventory.models.CmHandleState.READY
class CmHandleStateMonitorSpec extends Specification {
def 'Update cm handle state metric'() {
given: 'a collection of cm handle state pair'
- def cmHandleTransitionPair = new LcmEventsCmHandleStateHandlerImpl.CmHandleTransitionPair()
- cmHandleTransitionPair.currentYangModelCmHandle = new YangModelCmHandle(compositeState: new CompositeState(cmHandleState: ADVISED))
- cmHandleTransitionPair.targetYangModelCmHandle = new YangModelCmHandle(compositeState: new CompositeState(cmHandleState: READY))
+ def cmHandleTransitionPair = new CmHandleTransitionPair(new YangModelCmHandle(compositeState: new CompositeState(cmHandleState: ADVISED)),
+ new YangModelCmHandle(compositeState: new CompositeState(cmHandleState: READY))
+ )
when: 'method to update cm handle state metrics is called'
objectUnderTest.updateCmHandleStateMetrics([cmHandleTransitionPair])
then: 'cm handle by state cache map is called once for current and target state for entry processing'
def 'Update cm handle state metric with no previous state'() {
given: 'a collection of cm handle state pair wherein current state is null'
- def cmHandleTransitionPair = new LcmEventsCmHandleStateHandlerImpl.CmHandleTransitionPair()
- cmHandleTransitionPair.currentYangModelCmHandle = new YangModelCmHandle(compositeState: null)
- cmHandleTransitionPair.targetYangModelCmHandle = new YangModelCmHandle(compositeState: new CompositeState(cmHandleState: ADVISED))
+ def cmHandleTransitionPair = new CmHandleTransitionPair(new YangModelCmHandle(compositeState: null),
+ new YangModelCmHandle(compositeState: new CompositeState(cmHandleState: ADVISED)))
when: 'updating cm handle state metrics'
objectUnderTest.updateCmHandleStateMetrics([cmHandleTransitionPair])
then: 'cm handle by state cache map is called only once'
def objectUnderTest = new LcmEventsCmHandleStateHandlerImpl(mockInventoryPersistence, lcmEventsHelper, mockCmHandleStateMonitor)
def cmHandleId = 'cmhandle-id-1'
- def compositeState
+ def currentCompositeState
def yangModelCmHandle
def 'Update and Send Events on State Change #stateChange'() {
given: 'Cm Handle represented as YangModelCmHandle'
- compositeState = new CompositeState(cmHandleState: fromCmHandleState)
- yangModelCmHandle = new YangModelCmHandle(id: cmHandleId, additionalProperties: [], publicProperties: [], compositeState: compositeState)
+ currentCompositeState = new CompositeState(cmHandleState: fromCmHandleState)
+ yangModelCmHandle = new YangModelCmHandle(id: cmHandleId, additionalProperties: [], publicProperties: [], compositeState: currentCompositeState)
when: 'update state is invoked'
- objectUnderTest.updateCmHandleStateBatch(Map.of(yangModelCmHandle, toCmHandleState))
+ objectUnderTest.updateCmHandleStateBatch([(yangModelCmHandle): toCmHandleState])
then: 'state is saved using inventory persistence'
- 1 * mockInventoryPersistence.saveCmHandleStateBatch(_) >> {
- args -> {
- def cmHandleStatePerCmHandleId = args[0] as Map<String, CompositeState>
+ 1 * mockInventoryPersistence.saveCmHandleStateBatch(cmHandleStatePerCmHandleId -> {
assert cmHandleStatePerCmHandleId.get(cmHandleId).cmHandleState == toCmHandleState
- }
- }
+ })
and: 'log message shows state change at DEBUG level'
def loggingEvent = logAppender.list[0]
assert loggingEvent.level == Level.DEBUG
def 'Update and Send Events on State Change from LOCKED to ADVISED'() {
given: 'Cm Handle represented as YangModelCmHandle in LOCKED state'
- compositeState = new CompositeState(cmHandleState: LOCKED,
- lockReason: CompositeState.LockReason.builder().lockReasonCategory(MODULE_SYNC_FAILED).details('some lock details').build())
- yangModelCmHandle = new YangModelCmHandle(id: cmHandleId, additionalProperties: [], publicProperties: [], compositeState: compositeState)
+ currentCompositeState = new CompositeState(cmHandleState: LOCKED, lockReason: CompositeState.LockReason.builder().lockReasonCategory(MODULE_SYNC_FAILED).details('some lock details').build())
+ yangModelCmHandle = new YangModelCmHandle(id: cmHandleId, additionalProperties: [], publicProperties: [], compositeState: currentCompositeState)
when: 'update state is invoked'
- objectUnderTest.updateCmHandleStateBatch(Map.of(yangModelCmHandle, ADVISED))
+ objectUnderTest.updateCmHandleStateBatch([(yangModelCmHandle): ADVISED])
then: 'state is saved using inventory persistence and old lock reason details are retained'
- 1 * mockInventoryPersistence.saveCmHandleStateBatch(_) >> {
- args -> {
- def cmHandleStatePerCmHandleId = args[0] as Map<String, CompositeState>
+ 1 * mockInventoryPersistence.saveCmHandleStateBatch(cmHandleStatePerCmHandleId -> {
assert cmHandleStatePerCmHandleId.get(cmHandleId).lockReason.details == 'some lock details'
- }
- }
+ })
and: 'event service is called to send event'
1 * mockLcmEventsProducer.sendLcmEvent(cmHandleId, _, _)
and: 'a log entry is written'
def 'Update and Send Events on State Change to from ADVISED to READY'() {
given: 'Cm Handle represented as YangModelCmHandle'
- compositeState = new CompositeState(cmHandleState: ADVISED)
- yangModelCmHandle = new YangModelCmHandle(id: cmHandleId, additionalProperties: [], publicProperties: [], compositeState: compositeState)
+ currentCompositeState = new CompositeState(cmHandleState: ADVISED)
+ yangModelCmHandle = new YangModelCmHandle(id: cmHandleId, additionalProperties: [], publicProperties: [], compositeState: currentCompositeState)
and: 'global sync flag is set'
- compositeState.setDataSyncEnabled(false)
+ currentCompositeState.setDataSyncEnabled(false)
when: 'update cmhandle state is invoked'
objectUnderTest.updateCmHandleStateBatch(Map.of(yangModelCmHandle, READY))
then: 'state is saved using inventory persistence with expected dataSyncState'
- 1 * mockInventoryPersistence.saveCmHandleStateBatch(_) >> {
- args-> {
- def cmHandleStatePerCmHandleId = args[0] as Map<String, CompositeState>
+ 1 * mockInventoryPersistence.saveCmHandleStateBatch(cmHandleStatePerCmHandleId -> {
assert cmHandleStatePerCmHandleId.get(cmHandleId).dataSyncEnabled == false
assert cmHandleStatePerCmHandleId.get(cmHandleId).dataStores.operationalDataStore.dataStoreSyncState == DataStoreSyncState.NONE_REQUESTED
- }
- }
+ })
and: 'event service is called to send event'
1 * mockLcmEventsProducer.sendLcmEvent(cmHandleId, _, _)
and: 'a log entry is written'
def 'Update cmHandle state from READY to DELETING' (){
given: 'cm Handle as Yang model'
- compositeState = new CompositeState(cmHandleState: READY)
- yangModelCmHandle = new YangModelCmHandle(id: cmHandleId, additionalProperties: [], publicProperties: [], compositeState: compositeState)
+ currentCompositeState = new CompositeState(cmHandleState: READY)
+ yangModelCmHandle = new YangModelCmHandle(id: cmHandleId, additionalProperties: [], publicProperties: [], compositeState: currentCompositeState)
when: 'updating cm handle state to "DELETING"'
- objectUnderTest.updateCmHandleStateBatch(Map.of(yangModelCmHandle, DELETING))
+ objectUnderTest.updateCmHandleStateBatch([(yangModelCmHandle): DELETING])
then: 'the cm handle state is as expected'
yangModelCmHandle.getCompositeState().getCmHandleState() == DELETING
and: 'method to persist cm handle state is called once'
- 1 * mockInventoryPersistence.saveCmHandleStateBatch(Map.of(yangModelCmHandle.getId(), yangModelCmHandle.getCompositeState()))
+ 1 * mockInventoryPersistence.saveCmHandleStateBatch([(cmHandleId): yangModelCmHandle.compositeState])
and: 'the method to send Lcm event is called once'
1 * mockLcmEventsProducer.sendLcmEvent(cmHandleId, _, _)
}
def 'Update cmHandle state to DELETING to DELETED' (){
given: 'cm Handle with state "DELETING" as Yang model '
- compositeState = new CompositeState(cmHandleState: DELETING)
- yangModelCmHandle = new YangModelCmHandle(id: cmHandleId, additionalProperties: [], publicProperties: [], compositeState: compositeState)
+ currentCompositeState = new CompositeState(cmHandleState: DELETING)
+ yangModelCmHandle = new YangModelCmHandle(id: cmHandleId, additionalProperties: [], publicProperties: [], compositeState: currentCompositeState)
when: 'updating cm handle state to "DELETED"'
- objectUnderTest.updateCmHandleStateBatch(Map.of(yangModelCmHandle, DELETED))
+ objectUnderTest.updateCmHandleStateBatch([(yangModelCmHandle): DELETED])
then: 'the cm handle state is as expected'
yangModelCmHandle.getCompositeState().getCmHandleState() == DELETED
and: 'the method to send Lcm event is called once'
when: 'instantiating a batch of new cm handles'
objectUnderTest.initiateStateAdvised(yangModelCmHandlesToBeCreated)
then: 'new cm handles are saved using inventory persistence'
- 1 * mockInventoryPersistence.saveCmHandleBatch(_) >> {
- args -> {
- assert (args[0] as Collection<YangModelCmHandle>).id.containsAll('cmhandle1', 'cmhandle2')
- }
- }
+ 1 * mockInventoryPersistence.saveCmHandleBatch(yangModelCmHandles -> {
+ assert yangModelCmHandles.id.containsAll('cmhandle1', 'cmhandle2')
+ })
and: 'no state updates are persisted'
1 * mockInventoryPersistence.saveCmHandleStateBatch(EMPTY_MAP)
and: 'event service is called to send events'
when: 'updating a batch of changes'
objectUnderTest.updateCmHandleStateBatch(cmHandleStateMap)
then: 'existing cm handles composite states are persisted'
- 1 * mockInventoryPersistence.saveCmHandleStateBatch(_) >> {
- args -> {
- assert (args[0] as Map<String, CompositeState>).keySet().containsAll(['cmhandle1', 'cmhandle2'])
- }
- }
+ 1 * mockInventoryPersistence.saveCmHandleStateBatch(cmHandleStatePerCmHandleId -> {
+ assert cmHandleStatePerCmHandleId.keySet().containsAll(['cmhandle1', 'cmhandle2'])
+ })
and: 'no new handles are persisted'
1 * mockInventoryPersistence.saveCmHandleBatch(EMPTY_LIST)
and: 'event service is called to send events'
package org.onap.cps.ncmp.impl.inventory.sync.lcm
import org.mapstruct.factory.Mappers
+import org.onap.cps.ncmp.api.inventory.models.CmHandleState
import org.onap.cps.ncmp.api.inventory.models.CompositeState
import org.onap.cps.ncmp.api.inventory.models.NcmpServiceCmHandle
import org.onap.cps.ncmp.events.lcm.v1.Values
-import org.onap.cps.ncmp.api.inventory.models.CmHandleState
import spock.lang.Specification
import static org.onap.cps.ncmp.api.inventory.models.CmHandleState.ADVISED
def 'Map the LcmEvent for #operation'() {
given: 'NCMP cm handle details with current and old properties'
- def existingNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(dataSyncEnabled: true, cmHandleState: existingCmHandleState),
- publicProperties: existingPublicProperties)
+ def currentNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(dataSyncEnabled: true, cmHandleState: currentCmHandleState),
+ publicProperties: currentPublicProperties)
def targetNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(dataSyncEnabled: true, cmHandleState: targetCmHandleState),
publicProperties: targetPublicProperties)
- when: 'the event is populated'
- def result = objectUnderTest.populateLcmEvent(cmHandleId, targetNcmpServiceCmHandle, existingNcmpServiceCmHandle)
+ when: 'the lcm event is created'
+ def result = objectUnderTest.createLcmEvent(cmHandleId, currentNcmpServiceCmHandle, targetNcmpServiceCmHandle)
then: 'event header is mapped correctly'
assert result.eventSource == 'org.onap.ncmp'
assert result.eventCorrelationId == cmHandleId
and: 'event payload is mapped correctly with correct cmhandle id'
assert result.event.cmHandleId == cmHandleId
and: 'it should have correct old state and properties'
- assert result.event.oldValues.cmHandleState == expectedExistingCmHandleState
- assert result.event.oldValues.cmHandleProperties == [expectedExistingPublicProperties]
+ assert result.event.oldValues.cmHandleState == expectedCurrentCmHandleState
+ assert result.event.oldValues.cmHandleProperties == [expectedCurrentPublicProperties]
and: 'the correct new state and properties'
assert result.event.newValues.cmHandleProperties == [expectedTargetPublicProperties]
assert result.event.newValues.cmHandleState == expectedTargetCmHandleState
where: 'following parameters are provided'
- operation | existingCmHandleState | targetCmHandleState | existingPublicProperties | targetPublicProperties || expectedExistingPublicProperties | expectedTargetPublicProperties | expectedExistingCmHandleState | expectedTargetCmHandleState
- 'UPDATE' | ADVISED | READY | ['publicProperty1': 'value1', 'publicProperty2': 'value2'] | ['publicProperty1': 'value11'] || ['publicProperty1': 'value1', 'publicProperty2': 'value2'] | ['publicProperty1': 'value11'] | Values.CmHandleState.ADVISED | Values.CmHandleState.READY
- 'DELETING' | READY | DELETING | ['publicProperty1': 'value3', 'publicProperty2': 'value4'] | ['publicProperty1': 'value33'] || ['publicProperty1': 'value3', 'publicProperty2': 'value4'] | ['publicProperty1': 'value33'] | Values.CmHandleState.READY | Values.CmHandleState.DELETING
- 'CHANGE' | READY | READY | ['publicProperty1': 'value3', 'publicProperty2': 'value4'] | ['publicProperty1': 'value33'] || ['publicProperty1': 'value3', 'publicProperty2': 'value4'] | ['publicProperty1': 'value33'] | null | null
+ operation | currentCmHandleState | targetCmHandleState | currentPublicProperties | targetPublicProperties || expectedCurrentPublicProperties | expectedTargetPublicProperties | expectedCurrentCmHandleState | expectedTargetCmHandleState
+ 'UPDATE' | ADVISED | READY | ['publicProperty1': 'value1', 'publicProperty2': 'value2'] | ['publicProperty1': 'value11'] || ['publicProperty1': 'value1', 'publicProperty2': 'value2'] | ['publicProperty1': 'value11'] | Values.CmHandleState.ADVISED | Values.CmHandleState.READY
+ 'DELETING' | READY | DELETING | ['publicProperty1': 'value3', 'publicProperty2': 'value4'] | ['publicProperty1': 'value33'] || ['publicProperty1': 'value3', 'publicProperty2': 'value4'] | ['publicProperty1': 'value33'] | Values.CmHandleState.READY | Values.CmHandleState.DELETING
+ 'CHANGE' | READY | READY | ['publicProperty1': 'value3', 'publicProperty2': 'value4'] | ['publicProperty1': 'value33'] || ['publicProperty1': 'value3', 'publicProperty2': 'value4'] | ['publicProperty1': 'value33'] | null | null
}
def 'Map the LcmEvent for all properties NO CHANGE'() {
given: 'NCMP cm handle details without any changes'
def publicProperties = ['publicProperty1': 'value3', 'publicProperty2': 'value4']
- def existingNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(dataSyncEnabled: true, cmHandleState: READY),
+ def currentNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(dataSyncEnabled: true, cmHandleState: READY),
publicProperties: publicProperties)
def targetNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(dataSyncEnabled: true, cmHandleState: READY),
publicProperties: publicProperties)
- when: 'the event is populated'
- def result = objectUnderTest.populateLcmEvent(cmHandleId, targetNcmpServiceCmHandle, existingNcmpServiceCmHandle)
+ when: 'the lcm event is created'
+ def result = objectUnderTest.createLcmEvent(cmHandleId, currentNcmpServiceCmHandle, targetNcmpServiceCmHandle)
then: 'Properties are just the one which are same'
assert result.event.oldValues == null
assert result.event.newValues == null
given: 'NCMP cm handle details'
def targetNcmpServiceCmhandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(dataSyncEnabled: false, cmHandleState: READY),
publicProperties: ['publicProperty1': 'value11', 'publicProperty2': 'value22'])
- def existingNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, publicProperties: ['publicProperty1': 'value1', 'publicProperty2': 'value2'])
- when: 'the event is populated'
- def result = objectUnderTest.populateLcmEvent(cmHandleId, targetNcmpServiceCmhandle, existingNcmpServiceCmHandle)
+ def currentNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, publicProperties: ['publicProperty1': 'value1', 'publicProperty2': 'value2'])
+ when: 'the lcm event is created'
+ def result = objectUnderTest.createLcmEvent(cmHandleId, currentNcmpServiceCmHandle, targetNcmpServiceCmhandle)
then: 'event header is mapped correctly'
assert result.eventSource == 'org.onap.ncmp'
assert result.eventCorrelationId == cmHandleId
given: 'NCMP cm handle details'
def targetNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(dataSyncEnabled: false, cmHandleState: CmHandleState.DELETED),
publicProperties: ['publicProperty1': 'value11', 'publicProperty2': 'value22'])
- def existingNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(dataSyncEnabled: true, cmHandleState: DELETING),
+ def currentNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(dataSyncEnabled: true, cmHandleState: DELETING),
publicProperties: ['publicProperty1': 'value1'])
- when: 'the event is populated'
- def result = objectUnderTest.populateLcmEvent(cmHandleId, targetNcmpServiceCmHandle, existingNcmpServiceCmHandle)
+ when: 'the lcm event is created'
+ def result = objectUnderTest.createLcmEvent(cmHandleId, currentNcmpServiceCmHandle, targetNcmpServiceCmHandle)
then: 'event header is mapped correctly'
assert result.eventSource == 'org.onap.ncmp'
assert result.eventCorrelationId == cmHandleId
def 'Map the LcmEvent for datasync flag transition from #operation'() {
given: 'NCMP cm handle details with current and old details'
- def existingNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(dataSyncEnabled: existingDataSyncEnableFlag, cmHandleState: ADVISED))
+ def currentNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(dataSyncEnabled: currentDataSyncEnableFlag, cmHandleState: ADVISED))
def targetNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(dataSyncEnabled: targetDataSyncEnableFlag, cmHandleState: READY))
- when: 'the event is populated'
- def result = objectUnderTest.populateLcmEvent(cmHandleId, targetNcmpServiceCmHandle, existingNcmpServiceCmHandle)
+ when: 'the lcm event is created'
+ def result = objectUnderTest.createLcmEvent(cmHandleId, currentNcmpServiceCmHandle, targetNcmpServiceCmHandle)
then: 'event header is mapped correctly'
assert result.eventSource == 'org.onap.ncmp'
assert result.eventCorrelationId == cmHandleId
assert result.event.cmHandleId == cmHandleId
and: 'it should have correct old values'
assert result.event.oldValues.cmHandleState == Values.CmHandleState.ADVISED
- assert result.event.oldValues.dataSyncEnabled == existingDataSyncEnableFlag
+ assert result.event.oldValues.dataSyncEnabled == currentDataSyncEnableFlag
and: 'the correct new values'
assert result.event.newValues.cmHandleState == Values.CmHandleState.READY
assert result.event.newValues.dataSyncEnabled == targetDataSyncEnableFlag
where: 'following parameters are provided'
- operation | existingDataSyncEnableFlag | targetDataSyncEnableFlag
- 'false to true' | false | true
- 'false to null' | false | null
- 'true to false' | true | false
- 'true to null' | true | null
- 'null to true' | null | true
- 'null to false' | null | false
-
+ operation | currentDataSyncEnableFlag | targetDataSyncEnableFlag
+ 'false to true' | false | true
+ 'false to null' | false | null
+ 'true to false' | true | false
+ 'true to null' | true | null
+ 'null to true' | null | true
+ 'null to false' | null | false
}
def 'Map the LcmEvent for datasync flag for same transition from #operation'() {
given: 'NCMP cm handle details with current and old details'
- def existingNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(dataSyncEnabled: existingDataSyncEnableFlag, cmHandleState: ADVISED))
+ def currentNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(dataSyncEnabled: currentDataSyncEnableFlag, cmHandleState: ADVISED))
def targetNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(dataSyncEnabled: targetDataSyncEnableFlag, cmHandleState: READY))
- when: 'the event is populated'
- def result = objectUnderTest.populateLcmEvent(cmHandleId, targetNcmpServiceCmHandle, existingNcmpServiceCmHandle)
+ when: 'the lcm event is created'
+ def result = objectUnderTest.createLcmEvent(cmHandleId, currentNcmpServiceCmHandle, targetNcmpServiceCmHandle)
then: 'the data sync flag is not present in the event'
assert result.event.oldValues.dataSyncEnabled == null
assert result.event.newValues.dataSyncEnabled == null
where: 'following parameters are provided'
- operation | existingDataSyncEnableFlag | targetDataSyncEnableFlag
- 'false to false' | false | false
- 'true to true' | true | true
- 'null to null' | null | null
-
+ operation | currentDataSyncEnableFlag | targetDataSyncEnableFlag
+ 'false to false' | false | false
+ 'true to true' | true | true
+ 'null to null' | null | null
}
def 'Map the LcmEventHeader'() {
given: 'NCMP cm handle details with current and old details'
- def existingNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(cmHandleState: ADVISED))
+ def currentNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(cmHandleState: ADVISED))
def targetNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(cmHandleState: READY))
- when: 'the event header is populated'
- def result = objectUnderTest.populateLcmEventHeader(cmHandleId, targetNcmpServiceCmHandle, existingNcmpServiceCmHandle)
- then: 'the header has fields populated'
+ when: 'the lcm event header is created'
+ def result = objectUnderTest.createLcmEventHeader(cmHandleId, currentNcmpServiceCmHandle, targetNcmpServiceCmHandle)
+ then: 'the header field are populated'
assert result.eventCorrelationId == cmHandleId
assert result.eventId != null
}
def 'Map the LcmEvent for alternate ID, data producer identifier, and module set tag when they contain #scenario'() {
given: 'NCMP cm handle details with current and old values for alternate ID, module set tag, and data producer identifier'
- def existingNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, alternateId: existingAlternateId, moduleSetTag: existingModuleSetTag, dataProducerIdentifier: existingDataProducerIdentifier, compositeState: new CompositeState(dataSyncEnabled: false))
+ def currentNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, alternateId: currentAlternateId, moduleSetTag: currentModuleSetTag, dataProducerIdentifier: currentDataProducerIdentifier, compositeState: new CompositeState(dataSyncEnabled: false))
def targetNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, alternateId: targetAlternateId, moduleSetTag: targetModuleSetTag, dataProducerIdentifier: targetDataProducerIdentifier, compositeState: new CompositeState(dataSyncEnabled: false))
- when: 'the event is populated'
- def result = objectUnderTest.populateLcmEvent(cmHandleId, targetNcmpServiceCmHandle, existingNcmpServiceCmHandle)
+ when: 'the lcm event is created'
+ def result = objectUnderTest.createLcmEvent(cmHandleId, currentNcmpServiceCmHandle, targetNcmpServiceCmHandle)
then: 'the alternate ID, module set tag, and data producer identifier are present or are an empty string in the payload'
assert result.event.alternateId == targetAlternateId
assert result.event.moduleSetTag == targetModuleSetTag
assert result.event.dataProducerIdentifier == targetDataProducerIdentifier
where: 'the following values are provided for the alternate ID, module set tag, and data producer identifier'
- scenario | existingAlternateId | targetAlternateId | existingModuleSetTag | targetModuleSetTag | existingDataProducerIdentifier | targetDataProducerIdentifier
- 'same target and existing values' | 'someAlternateId' | 'someAlternateId' | 'someModuleSetTag' | 'someModuleSetTag' | 'someDataProducerIdentifier' | 'someDataProducerIdentifier'
- 'blank target and existing values' | '' | '' | '' | '' | '' | ''
- 'new target value and blank existing values' | '' | 'someAlternateId' | '' | 'someAlternateId' | '' | 'someDataProducerIdentifier'
+ scenario | currentAlternateId | targetAlternateId | currentModuleSetTag | targetModuleSetTag | currentDataProducerIdentifier | targetDataProducerIdentifier
+ 'same target and current values' | 'myAlternateId' | 'myAlternateId' | 'myModuleSetTag' | 'myModuleSetTag' | 'myDataProducerIdentifier' | 'myDataProducerIdentifier'
+ 'blank target and current values' | '' | '' | '' | '' | '' | ''
+ 'new target value and blank current values' | '' | 'myAlternateId' | '' | 'myAlternateId' | '' | 'myDataProducerIdentifier'
}
}