[Module Sync] Log state changes after persistence 08/139308/5
authordanielhanrahan <daniel.hanrahan@est.tech>
Fri, 25 Oct 2024 15:28:17 +0000 (16:28 +0100)
committerdanielhanrahan <daniel.hanrahan@est.tech>
Fri, 1 Nov 2024 17:29:20 +0000 (17:29 +0000)
Currently, module sync will output messages such as:
  "cm-handle-1 is now in READY state"
before the state change is persisted to the DB. If an exception
occurs, the new state may not be persisted, which makes debugging
hard due to misleading log entries. This commit moves the logging to
the LCM state handler immediately after the change is persisted.

- Move logging of state changes to LcmEventsCmHandleStateHandler.
- Remove unused code in LcmEventsCmHandleStateHandler.
- Add tests of state change logging.
- Update the main test showing the bug to instead show expected
  behaviour, but disable test for now using @Ignore.

Issue-ID: CPS-2474
Signed-off-by: danielhanrahan <daniel.hanrahan@est.tech>
Change-Id: I58646af6df95e07e69ca525744713a7c01c25e41

cps-ncmp-service/src/main/java/org/onap/cps/ncmp/impl/inventory/sync/AsyncTaskExecutor.java
cps-ncmp-service/src/main/java/org/onap/cps/ncmp/impl/inventory/sync/ModuleSyncTasks.java
cps-ncmp-service/src/main/java/org/onap/cps/ncmp/impl/inventory/sync/lcm/LcmEventsCmHandleStateHandler.java
cps-ncmp-service/src/main/java/org/onap/cps/ncmp/impl/inventory/sync/lcm/LcmEventsCmHandleStateHandlerAsyncHelper.java
cps-ncmp-service/src/main/java/org/onap/cps/ncmp/impl/inventory/sync/lcm/LcmEventsCmHandleStateHandlerImpl.java
cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/impl/inventory/sync/ModuleSyncTasksSpec.groovy
cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/impl/inventory/sync/lcm/LcmEventsCmHandleStateHandlerImplSpec.groovy
docs/api/swagger/ncmp/openapi-inventory.yaml
docs/api/swagger/ncmp/openapi.yaml

index e8ee600..80bc4ab 100644 (file)
@@ -68,7 +68,7 @@ public class AsyncTaskExecutor {
     private void handleTaskCompletion(final Object response, final Throwable throwable) {
         if (throwable != null) {
             if (throwable instanceof TimeoutException) {
-                log.error("Async task didn't completed within the required time.", throwable);
+                log.error("Async task didn't complete within the required time.", throwable);
             } else {
                 log.error("Watchdog async batch failed.", throwable);
             }
index e627f8f..80cdea8 100644 (file)
@@ -84,7 +84,6 @@ public class ModuleSyncTasks {
                     setCmHandleStateLocked(yangModelCmHandle, compositeState.getLockReason());
                     cmHandelStatePerCmHandle.put(yangModelCmHandle, CmHandleState.LOCKED);
                 }
-                log.info("{} is now in {} state", cmHandleId, cmHandelStatePerCmHandle.get(yangModelCmHandle).name());
             }
             lcmEventsCmHandleStateHandler.updateCmHandleStateBatch(cmHandelStatePerCmHandle);
         } finally {
index 6cce153..de3df6b 100644 (file)
@@ -1,6 +1,6 @@
 /*
  * ============LICENSE_START=======================================================
- * Copyright (C) 2022-2023 Nordix Foundation
+ * Copyright (C) 2022-2024 Nordix Foundation
  * ================================================================================
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
@@ -31,14 +31,6 @@ import org.onap.cps.ncmp.impl.inventory.models.YangModelCmHandle;
  */
 public interface LcmEventsCmHandleStateHandler {
 
-    /**
-     * Updates the composite state of cmHandle based on cmHandleState.
-     *
-     * @param yangModelCmHandle   cm handle represented as yang model
-     * @param targetCmHandleState target cm handle state
-     */
-    void updateCmHandleState(final YangModelCmHandle yangModelCmHandle, final CmHandleState targetCmHandleState);
-
     /**
      * Updates the composite state of cmHandle based on cmHandleState in batch.
      *
index cf7921c..a53c902 100644 (file)
@@ -1,6 +1,6 @@
 /*
  * ============LICENSE_START=======================================================
- * Copyright (C) 2023 Nordix Foundation
+ * Copyright (C) 2023-2024 Nordix Foundation
  * ================================================================================
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
@@ -26,6 +26,7 @@ import org.onap.cps.ncmp.api.inventory.models.NcmpServiceCmHandle;
 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;
@@ -37,26 +38,13 @@ public class LcmEventsCmHandleStateHandlerAsyncHelper {
     private final LcmEventsCreator lcmEventsCreator;
     private final LcmEventsService lcmEventsService;
 
-    /**
-     * Publish LCM Event in asynchronous manner.
-     *
-     * @param targetNcmpServiceCmHandle  target NcmpServiceCmHandle
-     * @param currentNcmpServiceCmHandle current NcmpServiceCmHandle
-     */
-    @Async("notificationExecutor")
-    public void publishLcmEventAsynchronously(final NcmpServiceCmHandle targetNcmpServiceCmHandle,
-                                              final NcmpServiceCmHandle currentNcmpServiceCmHandle) {
-        publishLcmEvent(targetNcmpServiceCmHandle, currentNcmpServiceCmHandle);
-    }
-
     /**
      * Publish LcmEvent in batches and in asynchronous manner.
      *
      * @param cmHandleTransitionPairs Pair of existing and modified cm handle represented as YangModelCmHandle
      */
     @Async("notificationExecutor")
-    public void publishLcmEventBatchAsynchronously(
-            final Collection<LcmEventsCmHandleStateHandlerImpl.CmHandleTransitionPair> cmHandleTransitionPairs) {
+    public void publishLcmEventBatchAsynchronously(final Collection<CmHandleTransitionPair> cmHandleTransitionPairs) {
         cmHandleTransitionPairs.forEach(cmHandleTransitionPair -> publishLcmEvent(
                 toNcmpServiceCmHandle(cmHandleTransitionPair.getTargetYangModelCmHandle()),
                 toNcmpServiceCmHandle(cmHandleTransitionPair.getCurrentYangModelCmHandle())));
index b1b7b95..e9bd372 100644 (file)
@@ -38,12 +38,10 @@ import lombok.RequiredArgsConstructor;
 import lombok.Setter;
 import lombok.extern.slf4j.Slf4j;
 import org.onap.cps.ncmp.api.inventory.models.CompositeState;
-import org.onap.cps.ncmp.api.inventory.models.NcmpServiceCmHandle;
 import org.onap.cps.ncmp.impl.inventory.CompositeStateUtils;
 import org.onap.cps.ncmp.impl.inventory.InventoryPersistence;
 import org.onap.cps.ncmp.impl.inventory.models.CmHandleState;
 import org.onap.cps.ncmp.impl.inventory.models.YangModelCmHandle;
-import org.onap.cps.ncmp.impl.utils.YangDataConverter;
 import org.springframework.stereotype.Service;
 
 @Slf4j
@@ -54,25 +52,6 @@ public class LcmEventsCmHandleStateHandlerImpl implements LcmEventsCmHandleState
     private final InventoryPersistence inventoryPersistence;
     private final LcmEventsCmHandleStateHandlerAsyncHelper lcmEventsCmHandleStateHandlerAsyncHelper;
 
-    @Override
-    public void updateCmHandleState(final YangModelCmHandle updatedYangModelCmHandle,
-            final CmHandleState targetCmHandleState) {
-
-        final CompositeState compositeState = updatedYangModelCmHandle.getCompositeState();
-
-        if (isCompositeStateSame(compositeState, targetCmHandleState)) {
-            log.debug("CmHandle with id : {} already in state : {}", updatedYangModelCmHandle.getId(),
-                    targetCmHandleState);
-        } else {
-            final YangModelCmHandle currentYangModelCmHandle = YangModelCmHandle.deepCopyOf(updatedYangModelCmHandle);
-            updateToSpecifiedCmHandleState(updatedYangModelCmHandle, targetCmHandleState);
-            persistCmHandle(updatedYangModelCmHandle, currentYangModelCmHandle);
-            lcmEventsCmHandleStateHandlerAsyncHelper.publishLcmEventAsynchronously(
-                    toNcmpServiceCmHandle(updatedYangModelCmHandle),
-                    toNcmpServiceCmHandle(currentYangModelCmHandle));
-        }
-    }
-
     @Override
     @Timed(value = "cps.ncmp.cmhandle.state.update.batch",
             description = "Time taken to update a batch of cm handle states")
@@ -113,28 +92,13 @@ public class LcmEventsCmHandleStateHandlerImpl implements LcmEventsCmHandleState
         return cmHandleTransitionPairs;
     }
 
-
-    private void persistCmHandle(final YangModelCmHandle targetYangModelCmHandle,
-            final YangModelCmHandle currentYangModelCmHandle) {
-        if (isNew(currentYangModelCmHandle.getCompositeState())) {
-            log.debug("Registering a new cm handle {}", targetYangModelCmHandle.getId());
-            inventoryPersistence.saveCmHandle(targetYangModelCmHandle);
-        } else if (isDeleted(targetYangModelCmHandle.getCompositeState())) {
-            log.info("CmHandle with Id : {} is DELETED", targetYangModelCmHandle.getId());
-        } else {
-            inventoryPersistence.saveCmHandleState(targetYangModelCmHandle.getId(),
-                    targetYangModelCmHandle.getCompositeState());
-        }
-    }
-
     private void persistCmHandleBatch(final Collection<CmHandleTransitionPair> cmHandleTransitionPairs) {
 
         final List<YangModelCmHandle> newCmHandles = new ArrayList<>();
         final Map<String, CompositeState> compositeStatePerCmHandleId = new LinkedHashMap<>();
 
         cmHandleTransitionPairs.forEach(cmHandleTransitionPair -> {
-            if (isNew(cmHandleTransitionPair.getCurrentYangModelCmHandle().getCompositeState()
-            )) {
+            if (isNew(cmHandleTransitionPair.getCurrentYangModelCmHandle().getCompositeState())) {
                 newCmHandles.add(cmHandleTransitionPair.getTargetYangModelCmHandle());
             } else if (!isDeleted(cmHandleTransitionPair.getTargetYangModelCmHandle().getCompositeState())) {
                 compositeStatePerCmHandleId.put(cmHandleTransitionPair.getTargetYangModelCmHandle().getId(),
@@ -145,10 +109,11 @@ public class LcmEventsCmHandleStateHandlerImpl implements LcmEventsCmHandleState
         inventoryPersistence.saveCmHandleBatch(newCmHandles);
         inventoryPersistence.saveCmHandleStateBatch(compositeStatePerCmHandleId);
 
+        logCmHandleStateChanges(cmHandleTransitionPairs);
     }
 
     private void updateToSpecifiedCmHandleState(final YangModelCmHandle yangModelCmHandle,
-            final CmHandleState targetCmHandleState) {
+                                                final CmHandleState targetCmHandleState) {
 
         if (READY == targetCmHandleState) {
             setInitialStates(yangModelCmHandle);
@@ -193,8 +158,11 @@ public class LcmEventsCmHandleStateHandlerImpl implements LcmEventsCmHandleState
         return (compositeState != null && compositeState.getCmHandleState() == targetCmHandleState);
     }
 
-    private NcmpServiceCmHandle toNcmpServiceCmHandle(final YangModelCmHandle yangModelCmHandle) {
-        return YangDataConverter.toNcmpServiceCmHandle(yangModelCmHandle);
+    private static void logCmHandleStateChanges(final Collection<CmHandleTransitionPair> cmHandleTransitionPairs) {
+        cmHandleTransitionPairs.stream()
+                .map(CmHandleTransitionPair::getTargetYangModelCmHandle)
+                .forEach(yangModelCmHandle -> log.info("{} is now in {} state", yangModelCmHandle.getId(),
+                        yangModelCmHandle.getCompositeState().getCmHandleState().name()));
     }
 
     @Getter
index 794bbc9..8ce1e93 100644 (file)
@@ -37,6 +37,7 @@ import org.onap.cps.ncmp.impl.inventory.sync.lcm.LcmEventsCmHandleStateHandler
 import org.onap.cps.spi.exceptions.DataNodeNotFoundException
 import org.onap.cps.spi.model.DataNode
 import org.slf4j.LoggerFactory
+import spock.lang.Ignore
 import spock.lang.Specification
 import java.util.concurrent.atomic.AtomicInteger
 
@@ -122,25 +123,25 @@ class ModuleSyncTasksSpec extends Specification {
             'module upgrade' | MODULE_UPGRADE        | 'Upgrade in progress'                          || MODULE_UPGRADE_FAILED
     }
 
-    // TODO Update this test once the bug CPS-2474 is fixed
-    def 'Module sync fails if a handle gets deleted during module sync.'() {
+    @Ignore  // TODO Enable this test once the bug CPS-2474 is fixed
+    def 'Module sync succeeds even if a handle gets deleted during module sync.'() {
         given: 'cm handles in an ADVISED state'
             def cmHandle1 = cmHandleAsDataNodeByIdAndState('cm-handle-1', CmHandleState.ADVISED)
             def cmHandle2 = cmHandleAsDataNodeByIdAndState('cm-handle-2', CmHandleState.ADVISED)
-        and: 'inventory persistence returns the first handle with ADVISED state'
-            mockInventoryPersistence.getCmHandleState('cm-handle-1') >> new CompositeState(cmHandleState: CmHandleState.ADVISED)
-        and: 'inventory persistence cannot find the second handle'
-            mockInventoryPersistence.getCmHandleState('cm-handle-2') >> { throw new DataNodeNotFoundException('dataspace', 'anchor', 'xpath') }
+        and: 'inventory persistence cannot find the first handle'
+            mockInventoryPersistence.getCmHandleState('cm-handle-1') >> { throw new DataNodeNotFoundException('dataspace', 'anchor', 'xpath') }
+        and: 'inventory persistence returns the second handle with ADVISED state'
+            mockInventoryPersistence.getCmHandleState('cm-handle-2') >> new CompositeState(cmHandleState: CmHandleState.ADVISED)
         when: 'module sync poll is executed'
             objectUnderTest.performModuleSync([cmHandle1, cmHandle2], batchCount)
-        then: 'an exception is thrown'
-            thrown(DataNodeNotFoundException)
-        and: 'even though the existing cm-handle did sync'
-            1 * mockModuleSyncService.syncAndCreateSchemaSetAndAnchor(_) >> { args -> assert args[0].id == 'cm-handle-1' }
-        and: 'logs report the cm-handle is in READY state'
-            assert getLoggingEvent().formattedMessage == 'cm-handle-1 is now in READY state'
-        and: 'this is impossible as the state handler was not called at all'
-            0 * mockLcmEventsCmHandleStateHandler.updateCmHandleStateBatch(_)
+        then: 'no exception is thrown'
+            noExceptionThrown()
+        and: 'the deleted cm-handle did not sync'
+            0 * mockModuleSyncService.syncAndCreateSchemaSetAndAnchor(_) >> { args -> assert args[0].id == 'cm-handle-1' }
+        and: 'the existing cm-handle synced'
+            1 * mockModuleSyncService.syncAndCreateSchemaSetAndAnchor(_) >> { args -> assert args[0].id == 'cm-handle-2' }
+        and: 'the state handler called'
+            1 * mockLcmEventsCmHandleStateHandler.updateCmHandleStateBatch(_)
     }
 
     def 'Reset failed CM Handles #scenario.'() {
@@ -174,7 +175,7 @@ class ModuleSyncTasksSpec extends Specification {
         when: 'module sync poll is executed'
             objectUnderTest.performModuleSync([cmHandle1], batchCount)
         then: 'module sync service is invoked for cm handle'
-            1 * mockModuleSyncService.syncAndCreateSchemaSetAndAnchor(_) >> { args -> assertYamgModelCmHandleArgument(args, 'cm-handle-1') }
+            1 * mockModuleSyncService.syncAndCreateSchemaSetAndAnchor(_)
         and: 'the entry for other cm handle is still in the progress map'
             assert moduleSyncStartedOnCmHandles.get('other-cm-handle') != null
     }
index bd7c321..4b676e1 100644 (file)
@@ -1,6 +1,6 @@
 /*
  * ============LICENSE_START=======================================================
- * Copyright (C) 2022-2023 Nordix Foundation
+ * Copyright (C) 2022-2024 Nordix Foundation
  * ================================================================================
  * 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 ch.qos.logback.classic.Level
+import ch.qos.logback.classic.Logger
+import ch.qos.logback.classic.spi.ILoggingEvent
+import ch.qos.logback.core.read.ListAppender
 import org.onap.cps.ncmp.api.inventory.models.CompositeState
 import org.onap.cps.ncmp.impl.inventory.DataStoreSyncState
 import org.onap.cps.ncmp.impl.inventory.InventoryPersistence
 import org.onap.cps.ncmp.impl.inventory.models.YangModelCmHandle
+import org.slf4j.LoggerFactory
 import spock.lang.Specification
 
+import static java.util.Collections.EMPTY_LIST
+import static java.util.Collections.EMPTY_MAP
 import static org.onap.cps.ncmp.impl.inventory.models.CmHandleState.ADVISED
 import static org.onap.cps.ncmp.impl.inventory.models.CmHandleState.DELETED
 import static org.onap.cps.ncmp.impl.inventory.models.CmHandleState.DELETING
@@ -35,6 +42,17 @@ import static org.onap.cps.ncmp.impl.inventory.models.LockReasonCategory.MODULE_
 
 class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
 
+    def logger = Spy(ListAppender<ILoggingEvent>)
+
+    void setup() {
+        ((Logger) LoggerFactory.getLogger(LcmEventsCmHandleStateHandlerImpl.class)).addAppender(logger)
+        logger.start()
+    }
+
+    void cleanup() {
+        ((Logger) LoggerFactory.getLogger(LcmEventsCmHandleStateHandlerImpl.class)).detachAndStopAllAppenders()
+    }
+
     def mockInventoryPersistence = Mock(InventoryPersistence)
     def mockLcmEventsCreator = Mock(LcmEventsCreator)
     def mockLcmEventsService = Mock(LcmEventsService)
@@ -51,30 +69,39 @@ class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
             compositeState = new CompositeState(cmHandleState: fromCmHandleState)
             yangModelCmHandle = new YangModelCmHandle(id: cmHandleId, dmiProperties: [], publicProperties: [], compositeState: compositeState)
         when: 'update state is invoked'
-            objectUnderTest.updateCmHandleState(yangModelCmHandle, toCmHandleState)
+            objectUnderTest.updateCmHandleStateBatch(Map.of(yangModelCmHandle, toCmHandleState))
         then: 'state is saved using inventory persistence'
-            expectedCallsToInventoryPersistence * mockInventoryPersistence.saveCmHandleState(cmHandleId, _)
+            1 * mockInventoryPersistence.saveCmHandleStateBatch(_) >> {
+                args -> {
+                    def cmHandleStatePerCmHandleId = args[0] as Map<String, CompositeState>
+                    assert cmHandleStatePerCmHandleId.get(cmHandleId).cmHandleState == toCmHandleState
+                }
+            }
+        and: 'log message shows state change at INFO level'
+            def loggingEvent = (ILoggingEvent) logger.list[0]
+            assert loggingEvent.level == Level.INFO
+            assert loggingEvent.formattedMessage == "${cmHandleId} is now in ${toCmHandleState} state"
         and: 'event service is called to publish event'
-            expectedCallsToEventService * mockLcmEventsService.publishLcmEvent(cmHandleId, _, _)
+            1 * mockLcmEventsService.publishLcmEvent(cmHandleId, _, _)
         where: 'state change parameters are provided'
-            stateChange          | fromCmHandleState | toCmHandleState || expectedCallsToInventoryPersistence | expectedCallsToEventService
-            'ADVISED to READY'   | ADVISED           | READY           || 1                                   | 1
-            'READY to LOCKED'    | READY             | LOCKED          || 1                                   | 1
-            'ADVISED to ADVISED' | ADVISED           | ADVISED         || 0                                   | 0
-            'READY to READY'     | READY             | READY           || 0                                   | 0
-            'LOCKED to LOCKED'   | LOCKED            | LOCKED          || 0                                   | 0
-            'DELETED to ADVISED' | DELETED           | ADVISED         || 0                                   | 1
+            stateChange           | fromCmHandleState | toCmHandleState
+            'ADVISED to READY'    | ADVISED           | READY
+            'READY to LOCKED'     | READY             | LOCKED
+            'ADVISED to LOCKED'   | ADVISED           | LOCKED
+            'ADVISED to DELETING' | ADVISED           | DELETING
     }
 
-    def 'Update and Publish Events on State Change from NO_EXISTING state to ADVISED'() {
+    def 'Update and Publish Events on State Change from non-existing to ADVISED'() {
         given: 'Cm Handle represented as YangModelCmHandle'
             yangModelCmHandle = new YangModelCmHandle(id: cmHandleId, dmiProperties: [], publicProperties: [])
         when: 'update state is invoked'
-            objectUnderTest.updateCmHandleState(yangModelCmHandle, ADVISED)
-        then: 'state is saved using inventory persistence'
-            1 * mockInventoryPersistence.saveCmHandle(yangModelCmHandle)
+            objectUnderTest.updateCmHandleStateBatch(Map.of(yangModelCmHandle, ADVISED))
+        then: 'CM-handle is saved using inventory persistence'
+            1 * mockInventoryPersistence.saveCmHandleBatch(List.of(yangModelCmHandle))
         and: 'event service is called to publish event'
             1 * mockLcmEventsService.publishLcmEvent(cmHandleId, _, _)
+        and: 'a log entry is written'
+            assert getLogMessage(0) == "${cmHandleId} is now in ADVISED state"
     }
 
     def 'Update and Publish Events on State Change from LOCKED to ADVISED'() {
@@ -83,69 +110,62 @@ class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
                 lockReason: CompositeState.LockReason.builder().lockReasonCategory(MODULE_SYNC_FAILED).details('some lock details').build())
             yangModelCmHandle = new YangModelCmHandle(id: cmHandleId, dmiProperties: [], publicProperties: [], compositeState: compositeState)
         when: 'update state is invoked'
-            objectUnderTest.updateCmHandleState(yangModelCmHandle, ADVISED)
+            objectUnderTest.updateCmHandleStateBatch(Map.of(yangModelCmHandle, ADVISED))
         then: 'state is saved using inventory persistence and old lock reason details are retained'
-            1 * mockInventoryPersistence.saveCmHandleState(cmHandleId, _) >> {
+            1 * mockInventoryPersistence.saveCmHandleStateBatch(_) >> {
                 args -> {
-                    assert (args[1] as CompositeState).lockReason.details == 'some lock details'
+                    def cmHandleStatePerCmHandleId = args[0] as Map<String, CompositeState>
+                    assert cmHandleStatePerCmHandleId.get(cmHandleId).lockReason.details == 'some lock details'
                 }
             }
         and: 'event service is called to publish event'
             1 * mockLcmEventsService.publishLcmEvent(cmHandleId, _, _)
+        and: 'a log entry is written'
+            assert getLogMessage(0) == "${cmHandleId} is now in ADVISED state"
     }
 
-    def 'Update and Publish Events on State Change from DELETING to ADVISED'() {
-        given: 'Cm Handle represented as YangModelCmHandle in DELETING state'
-            yangModelCmHandle = new YangModelCmHandle(id: cmHandleId, dmiProperties: [], publicProperties: [], compositeState: compositeState)
-        when: 'update state is invoked'
-            objectUnderTest.updateCmHandleState(yangModelCmHandle, ADVISED)
-        then: 'the cm handle is saved using inventory persistence'
-            1 * mockInventoryPersistence.saveCmHandle(yangModelCmHandle)
-        and: 'event service is called to publish event'
-            1 * mockLcmEventsService.publishLcmEvent(cmHandleId, _, _)
-    }
-
-    def 'Update and Publish Events on State Change to READY'() {
+    def 'Update and Publish 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, dmiProperties: [], publicProperties: [], compositeState: compositeState)
         and: 'global sync flag is set'
             compositeState.setDataSyncEnabled(false)
         when: 'update cmhandle state is invoked'
-            objectUnderTest.updateCmHandleState(yangModelCmHandle, READY)
+            objectUnderTest.updateCmHandleStateBatch(Map.of(yangModelCmHandle, READY))
         then: 'state is saved using inventory persistence with expected dataSyncState'
-            1 * mockInventoryPersistence.saveCmHandleState(cmHandleId, _) >> {
+            1 * mockInventoryPersistence.saveCmHandleStateBatch(_) >> {
                 args-> {
-                    def result = (args[1] as CompositeState)
-                    assert result.dataSyncEnabled == false
-                    assert result.dataStores.operationalDataStore.dataStoreSyncState == DataStoreSyncState.NONE_REQUESTED
-
+                    def cmHandleStatePerCmHandleId = args[0] as Map<String, CompositeState>
+                    assert cmHandleStatePerCmHandleId.get(cmHandleId).dataSyncEnabled == false
+                    assert cmHandleStatePerCmHandleId.get(cmHandleId).dataStores.operationalDataStore.dataStoreSyncState == DataStoreSyncState.NONE_REQUESTED
                 }
             }
         and: 'event service is called to publish event'
             1 * mockLcmEventsService.publishLcmEvent(cmHandleId, _, _)
+        and: 'a log entry is written'
+            assert getLogMessage(0) == "${cmHandleId} is now in READY state"
     }
 
-    def 'Update cmHandle state to "DELETING"' (){
+    def 'Update cmHandle state from READY to DELETING' (){
         given: 'cm Handle as Yang model'
             compositeState = new CompositeState(cmHandleState: READY)
             yangModelCmHandle = new YangModelCmHandle(id: cmHandleId, dmiProperties: [], publicProperties: [], compositeState: compositeState)
         when: 'updating cm handle state to "DELETING"'
-            objectUnderTest.updateCmHandleState(yangModelCmHandle, DELETING)
+            objectUnderTest.updateCmHandleStateBatch(Map.of(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.saveCmHandleState(yangModelCmHandle.getId(), yangModelCmHandle.getCompositeState())
+            1 * mockInventoryPersistence.saveCmHandleStateBatch(Map.of(yangModelCmHandle.getId(), yangModelCmHandle.getCompositeState()))
         and: 'the method to publish Lcm event is called once'
             1 * mockLcmEventsService.publishLcmEvent(cmHandleId, _, _)
     }
 
-    def 'Update cmHandle state to "DELETED"' (){
+    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, dmiProperties: [], publicProperties: [], compositeState: compositeState)
         when: 'updating cm handle state to "DELETED"'
-            objectUnderTest.updateCmHandleState(yangModelCmHandle, DELETED)
+            objectUnderTest.updateCmHandleStateBatch(Map.of(yangModelCmHandle, DELETED))
         then: 'the cm handle state is as expected'
             yangModelCmHandle.getCompositeState().getCmHandleState() == DELETED
         and: 'the method to publish Lcm event is called once'
@@ -157,14 +177,13 @@ class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
             def cmHandleStateMap = setupBatch('NO_CHANGE')
         when: 'updating a batch of changes'
             objectUnderTest.updateCmHandleStateBatch(cmHandleStateMap)
-        then: 'batch is empty and nothing to update'
-            1 * mockInventoryPersistence.saveCmHandleBatch(_) >> {
-                args -> {
-                    assert (args[0] as Collection<YangModelCmHandle>).size() == 0
-                }
-            }
+        then: 'no changes are persisted'
+            1 * mockInventoryPersistence.saveCmHandleBatch(EMPTY_LIST)
+            1 * mockInventoryPersistence.saveCmHandleStateBatch(EMPTY_MAP)
         and: 'no event will be published'
             0 * mockLcmEventsService.publishLcmEvent(*_)
+        and: 'no log entries are written'
+            assert logger.list.empty
     }
 
     def 'Batch of new cm handles provided'() {
@@ -178,8 +197,13 @@ class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
                     assert (args[0] as Collection<YangModelCmHandle>).id.containsAll('cmhandle1', 'cmhandle2')
                 }
             }
+        and: 'no state updates are persisted'
+            1 * mockInventoryPersistence.saveCmHandleStateBatch(EMPTY_MAP)
         and: 'event service is called to publish events'
             2 * mockLcmEventsService.publishLcmEvent(_, _, _)
+        and: 'two log entries are written'
+            assert getLogMessage(0) == 'cmhandle1 is now in ADVISED state'
+            assert getLogMessage(1) == 'cmhandle2 is now in ADVISED state'
     }
 
     def 'Batch of existing cm handles is updated'() {
@@ -187,14 +211,19 @@ class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
             def cmHandleStateMap = setupBatch('UPDATE')
         when: 'updating a batch of changes'
             objectUnderTest.updateCmHandleStateBatch(cmHandleStateMap)
-        then : 'existing cm handles composite state is persisted'
+        then: 'existing cm handles composite states are persisted'
             1 * mockInventoryPersistence.saveCmHandleStateBatch(_) >> {
                 args -> {
-                    assert (args[0] as Map<String, CompositeState>).keySet().containsAll(['cmhandle1','cmhandle2'])
+                    assert (args[0] as Map<String, CompositeState>).keySet().containsAll(['cmhandle1', 'cmhandle2'])
                 }
             }
+        and: 'no new handles are persisted'
+            1 * mockInventoryPersistence.saveCmHandleBatch(EMPTY_LIST)
         and: 'event service is called to publish events'
             2 * mockLcmEventsService.publishLcmEvent(_, _, _)
+        and: 'two log entries are written'
+            assert getLogMessage(0) == 'cmhandle1 is now in READY state'
+            assert getLogMessage(1) == 'cmhandle2 is now in DELETING state'
     }
 
     def 'Batch of existing cm handles is deleted'() {
@@ -202,14 +231,30 @@ class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
             def cmHandleStateMap = setupBatch('DELETED')
         when: 'updating a batch of changes'
             objectUnderTest.updateCmHandleStateBatch(cmHandleStateMap)
-        then : 'existing cm handles composite state is persisted'
-            1 * mockInventoryPersistence.saveCmHandleStateBatch(_) >> {
-                args -> {
-                    assert (args[0] as Map<String, CompositeState>).isEmpty()
-                }
-            }
+        then: 'state of deleted handles is not persisted'
+            1 * mockInventoryPersistence.saveCmHandleStateBatch(EMPTY_MAP)
+        and: 'no new handles are persisted'
+            1 * mockInventoryPersistence.saveCmHandleBatch(EMPTY_LIST)
         and: 'event service is called to publish events'
             2 * mockLcmEventsService.publishLcmEvent(_, _, _)
+        and: 'two log entries are written'
+            assert getLogMessage(0) == 'cmhandle1 is now in DELETED state'
+            assert getLogMessage(1) == 'cmhandle2 is now in DELETED state'
+    }
+
+    def 'Log entries and events are not sent when an error occurs during persistence'() {
+        given: 'A batch of updated cm handles'
+            def cmHandleStateMap = setupBatch('UPDATE')
+        and: 'an error will be thrown when trying to persist'
+            mockInventoryPersistence.saveCmHandleStateBatch(_) >> { throw new RuntimeException() }
+        when: 'updating a batch of changes'
+            objectUnderTest.updateCmHandleStateBatch(cmHandleStateMap)
+        then: 'the exception is not handled'
+            thrown(RuntimeException)
+        and: 'no events are published'
+            0 * mockLcmEventsService.publishLcmEvent(_, _, _)
+        and: 'no log entries are written'
+            assert logger.list.empty
     }
 
     def setupBatch(type) {
@@ -217,26 +262,31 @@ class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
         def yangModelCmHandle1 = new YangModelCmHandle(id: 'cmhandle1', dmiProperties: [], publicProperties: [])
         def yangModelCmHandle2 = new YangModelCmHandle(id: 'cmhandle2', dmiProperties: [], publicProperties: [])
 
-        if ('NEW' == type) {
-            return [yangModelCmHandle1, yangModelCmHandle2]
-        }
+        switch (type) {
+            case 'NEW':
+                return [yangModelCmHandle1, yangModelCmHandle2]
 
-        if ('DELETED' == type) {
-            yangModelCmHandle1.compositeState = new CompositeState(cmHandleState: READY)
-            yangModelCmHandle2.compositeState = new CompositeState(cmHandleState: READY)
-            return [(yangModelCmHandle1): DELETED, (yangModelCmHandle2): DELETED]
-        }
+            case 'DELETED':
+                yangModelCmHandle1.compositeState = new CompositeState(cmHandleState: READY)
+                yangModelCmHandle2.compositeState = new CompositeState(cmHandleState: READY)
+                return [(yangModelCmHandle1): DELETED, (yangModelCmHandle2): DELETED]
 
-        if ('UPDATE' == type) {
-            yangModelCmHandle1.compositeState = new CompositeState(cmHandleState: ADVISED)
-            yangModelCmHandle2.compositeState = new CompositeState(cmHandleState: READY)
-            return [(yangModelCmHandle1): READY, (yangModelCmHandle2): DELETING]
-        }
+            case 'UPDATE':
+                yangModelCmHandle1.compositeState = new CompositeState(cmHandleState: ADVISED)
+                yangModelCmHandle2.compositeState = new CompositeState(cmHandleState: READY)
+                return [(yangModelCmHandle1): READY, (yangModelCmHandle2): DELETING]
+
+            case 'NO_CHANGE':
+                yangModelCmHandle1.compositeState = new CompositeState(cmHandleState: ADVISED)
+                yangModelCmHandle2.compositeState = new CompositeState(cmHandleState: READY)
+                return [(yangModelCmHandle1): ADVISED, (yangModelCmHandle2): READY]
 
-        if ('NO_CHANGE' == type) {
-            yangModelCmHandle1.compositeState = new CompositeState(cmHandleState: ADVISED)
-            yangModelCmHandle2.compositeState = new CompositeState(cmHandleState: READY)
-            return [(yangModelCmHandle1): ADVISED, (yangModelCmHandle2): READY]
+            default:
+                throw new IllegalArgumentException("batch type '${type}' not recognized")
         }
     }
+
+    def getLogMessage(index) {
+        return logger.list[index].formattedMessage
+    }
 }
index a2c7af6..8552ad5 100644 (file)
@@ -131,9 +131,19 @@ paths:
       - network-cm-proxy-inventory
   /v1/ch/searches:
     post:
-      description: "Query and get CMHandleIds for additional properties, public properties\
-        \ and registered DMI plugin (DMI plugin, DMI data plugin, DMI model plugin)."
+      description: "Query and get CMHandle references for additional properties, public\
+        \ properties and registered DMI plugin (DMI plugin, DMI data plugin, DMI model\
+        \ plugin)."
       operationId: searchCmHandleIds
+      parameters:
+      - description: Boolean parameter to determine if returned value(s) will be cmHandle
+          Ids or Alternate Ids for a given query
+        in: query
+        name: outputAlternateId
+        required: false
+        schema:
+          default: false
+          type: boolean
       requestBody:
         content:
           application/json:
@@ -182,6 +192,15 @@ components:
       schema:
         example: my-dmi-plugin
         type: string
+    outputAlternateIdOptionInQuery:
+      description: Boolean parameter to determine if returned value(s) will be cmHandle
+        Ids or Alternate Ids for a given query
+      in: query
+      name: outputAlternateId
+      required: false
+      schema:
+        default: false
+        type: boolean
   responses:
     NoContent:
       content: {}
index f93395a..aa732c8 100644 (file)
@@ -1129,7 +1129,7 @@ paths:
       - network-cm-proxy
   /v1/ch/id-searches:
     post:
-      description: Execute cm handle query search and return a list of cm handle ids.
+      description: Execute cm handle query search and return a list of cm handle references.
         Any number of conditions can be applied. To be included in the result a cm-handle
         must fulfill ALL the conditions. An empty collection will be returned in the
         case that the cm handle does not match a condition. For more on cm handle
@@ -1140,6 +1140,15 @@ paths:
         Path Read the Docs</a>. The cm handle ancestor is automatically returned for
         this query.
       operationId: searchCmHandleIds
+      parameters:
+      - description: Boolean parameter to determine if returned value(s) will be cmHandle
+          Ids or Alternate Ids for a given query
+        in: query
+        name: outputAlternateId
+        required: false
+        schema:
+          default: false
+          type: boolean
       requestBody:
         content:
           application/json:
@@ -1608,6 +1617,15 @@ components:
       schema:
         example: 2024-01-22
         type: string
+    outputAlternateIdOptionInQuery:
+      description: Boolean parameter to determine if returned value(s) will be cmHandle
+        Ids or Alternate Ids for a given query
+      in: query
+      name: outputAlternateId
+      required: false
+      schema:
+        default: false
+        type: boolean
     dataSyncEnabled:
       description: Is used to enable or disable the data synchronization flag
       in: query