import static org.onap.cps.ncmp.api.impl.operations.DmiRequestBody.OperationEnum;
 import static org.onap.cps.utils.CmHandleQueryRestParametersValidator.validateCmHandleQueryParameters;
 
+import com.hazelcast.map.IMap;
 import java.time.OffsetDateTime;
 import java.util.ArrayList;
 import java.util.Collection;
 public class NetworkCmProxyDataServiceImpl implements NetworkCmProxyDataService {
 
     private final JsonObjectMapper jsonObjectMapper;
-
     private final DmiDataOperations dmiDataOperations;
-
     private final NetworkCmProxyDataServicePropertyHandler networkCmProxyDataServicePropertyHandler;
-
     private final InventoryPersistence inventoryPersistence;
-
     private final CmHandleQueries cmHandleQueries;
-
     private final NetworkCmProxyCmHandlerQueryService networkCmProxyCmHandlerQueryService;
-
     private final LcmEventsCmHandleStateHandler lcmEventsCmHandleStateHandler;
-
     private final CpsDataService cpsDataService;
+    private final IMap<String, Object> moduleSyncStartedOnCmHandles;
 
     @Override
     public DmiPluginRegistrationResponse updateDmiRegistrationAndSyncModule(
                 final YangModelCmHandle yangModelCmHandle = inventoryPersistence.getYangModelCmHandle(cmHandleId);
                 lcmEventsCmHandleStateHandler.updateCmHandleState(yangModelCmHandle,
                         CmHandleState.DELETING);
-                deleteCmHandleByCmHandleId(cmHandleId);
+                deleteCmHandleFromDbAndModuleSyncMap(cmHandleId);
                 cmHandleRegistrationResponses.add(CmHandleRegistrationResponse.createSuccessResponse(cmHandleId));
                 lcmEventsCmHandleStateHandler.updateCmHandleState(yangModelCmHandle,
                         CmHandleState.DELETED);
         return cmHandleRegistrationResponses;
     }
 
-    private void deleteCmHandleByCmHandleId(final String cmHandleId) {
+    private void deleteCmHandleFromDbAndModuleSyncMap(final String cmHandleId) {
         inventoryPersistence.deleteSchemaSetWithCascade(cmHandleId);
         inventoryPersistence.deleteListOrListElement("/dmi-registry/cm-handles[@id='" + cmHandleId + "']");
+        removeDeletedCmHandleFromModuleSyncMap(cmHandleId);
+    }
+
+    // CPS-1239 Robustness cleaning of in progress cache
+    private void removeDeletedCmHandleFromModuleSyncMap(final String deletedCmHandleId) {
+        if (moduleSyncStartedOnCmHandles.remove(deletedCmHandleId) != null) {
+            log.debug("{} removed from in progress map", deletedCmHandleId);
+        }
     }
 
     private List<CmHandleRegistrationResponse> registerNewCmHandles(final Map<YangModelCmHandle, CmHandleState>
 
 import com.hazelcast.config.QueueConfig;
 import com.hazelcast.core.Hazelcast;
 import com.hazelcast.core.HazelcastInstance;
-import java.util.Map;
+import com.hazelcast.map.IMap;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.TimeUnit;
 import org.onap.cps.spi.model.DataNode;
      * @return Map of cm handles (ids) and objects (not used really) for which module sync has started or been completed
      */
     @Bean
-    public Map<String, Object> moduleSyncStartedOnCmHandles() {
+    public IMap<String, Object> moduleSyncStartedOnCmHandles() {
         return createHazelcastInstance("moduleSyncStartedOnCmHandles", moduleSyncStartedConfig)
             .getMap("moduleSyncStartedOnCmHandles");
     }
      * @return configured map of data sync semaphores
      */
     @Bean
-    public Map<String, Boolean> dataSyncSemaphores() {
+    public IMap<String, Boolean> dataSyncSemaphores() {
         return createHazelcastInstance("dataSyncSemaphores", dataSyncSemaphoresConfig)
             .getMap("dataSyncSemaphores");
     }
 
 
 package org.onap.cps.ncmp.api.inventory.sync;
 
+import com.hazelcast.map.IMap;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
     private final SyncUtils syncUtils;
     private final ModuleSyncService moduleSyncService;
     private final LcmEventsCmHandleStateHandler lcmEventsCmHandleStateHandler;
-
-    private static final CompletableFuture<Void> COMPLETED_FUTURE = CompletableFuture.completedFuture(null);
+    private final IMap<String, Object> moduleSyncStartedOnCmHandles;
 
     /**
      * Perform module sync on a batch of cm handles.
      *
-     * @param cmHandlesAsDataNodes a batch of Data nodes representing cm handles to perform module sync on
-     * @param batchCounter the number of batches currently being processed, will be decreased when task is finished
-     *                     or fails
+     * @param cmHandlesAsDataNodes         a batch of Data nodes representing cm handles to perform module sync on
+     * @param batchCounter                 the number of batches currently being processed, will be decreased when
+     *                                     task is finished or fails
      * @return completed future to handle post-processing
      */
     public CompletableFuture<Void> performModuleSync(final Collection<DataNode> cmHandlesAsDataNodes,
                     moduleSyncService.syncAndCreateSchemaSetAndAnchor(yangModelCmHandle);
                     cmHandelStatePerCmHandle.put(yangModelCmHandle, CmHandleState.READY);
                 } catch (final Exception e) {
-                    log.warn("Processing module sync batch failed.");
+                    log.warn("Processing of {} module sync failed.", cmHandleId);
                     syncUtils.updateLockReasonDetailsAndAttempts(compositeState,
                             LockReasonCategory.LOCKED_MODULE_SYNC_FAILED, e.getMessage());
                     setCmHandleStateLocked(yangModelCmHandle, compositeState.getLockReason());
             batchCounter.getAndDecrement();
             log.info("Processing module sync batch finished. {} batch(es) active.", batchCounter.get());
         }
-        return COMPLETED_FUTURE;
+        return CompletableFuture.completedFuture(null);
     }
 
     /**
      * Reset state to "ADVISED" for any previously failed cm handles.
      *
      * @param failedCmHandles previously failed (locked) cm handles
-     * @return completed future to handle post-processing
      */
-    public CompletableFuture<Void> resetFailedCmHandles(final List<YangModelCmHandle> failedCmHandles) {
+    public void resetFailedCmHandles(final List<YangModelCmHandle> failedCmHandles) {
         final Map<YangModelCmHandle, CmHandleState> cmHandleStatePerCmHandle = new HashMap<>(failedCmHandles.size());
         for (final YangModelCmHandle failedCmHandle : failedCmHandles) {
             final CompositeState compositeState = failedCmHandle.getCompositeState();
             final boolean isReadyForRetry = syncUtils.isReadyForRetry(compositeState);
             if (isReadyForRetry) {
+                final String resetCmHandleId = failedCmHandle.getId();
                 log.debug("Reset cm handle {} state to ADVISED to be re-attempted by module-sync watchdog",
-                        failedCmHandle.getId());
+                        resetCmHandleId);
                 cmHandleStatePerCmHandle.put(failedCmHandle, CmHandleState.ADVISED);
+                removeResetCmHandleFromModuleSyncMap(resetCmHandleId);
             }
         }
         lcmEventsCmHandleStateHandler.updateCmHandleStateBatch(cmHandleStatePerCmHandle);
-        return COMPLETED_FUTURE;
     }
 
     private void setCmHandleStateLocked(final YangModelCmHandle advisedCmHandle,
         advisedCmHandle.getCompositeState().setLockReason(lockReason);
     }
 
+    private void removeResetCmHandleFromModuleSyncMap(final String resetCmHandleId) {
+        if (moduleSyncStartedOnCmHandles.remove(resetCmHandleId) != null) {
+            log.debug("{} removed from in progress map", resetCmHandleId);
+        }
+    }
 }
 
 
 package org.onap.cps.ncmp.api.inventory.sync;
 
+import com.hazelcast.map.IMap;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
     private final SyncUtils syncUtils;
     private final BlockingQueue<DataNode> moduleSyncWorkQueue;
-    private final Map<String, Object> moduleSyncStartedOnCmHandles;
+    private final IMap<String, Object> moduleSyncStartedOnCmHandles;
     private final ModuleSyncTasks moduleSyncTasks;
     private final AsyncTaskExecutor asyncTaskExecutor;
     private static final int MODULE_SYNC_BATCH_SIZE = 100;
                         nextBatch.size(), batchCounter.get());
                 asyncTaskExecutor.executeTask(() ->
                                 moduleSyncTasks.performModuleSync(nextBatch, batchCounter),
-                        ASYNC_TASK_TIMEOUT_IN_MILLISECONDS
-                );
+                        ASYNC_TASK_TIMEOUT_IN_MILLISECONDS);
                 batchCounter.getAndIncrement();
             } else {
                 preventBusyWait();
 
 package org.onap.cps.ncmp.api.impl
 
 import com.fasterxml.jackson.databind.ObjectMapper
+import com.hazelcast.map.IMap
 import org.onap.cps.api.CpsDataService
 import org.onap.cps.api.CpsModuleService
 import org.onap.cps.ncmp.api.NetworkCmProxyCmHandlerQueryService
     def stubbedNetworkCmProxyCmHandlerQueryService = Stub(NetworkCmProxyCmHandlerQueryService)
     def mockLcmEventsCmHandleStateHandler = Mock(LcmEventsCmHandleStateHandler)
     def mockCpsDataService = Mock(CpsDataService)
+    def mockModuleSyncStartedOnCmHandles = Mock(IMap<String, Object>)
     def objectUnderTest = getObjectUnderTest()
 
     def 'DMI Registration: Create, Update & Delete operations are processed in the right order'() {
             // Spock validated invocation order between multiple then blocks
         then: 'cm-handles are removed first'
             1 * objectUnderTest.parseAndRemoveCmHandlesInDmiRegistration(*_)
+        and: 'de-registered cm handle entry is removed from in progress map'
+            1 * mockModuleSyncStartedOnCmHandles.remove('cmhandle-2')
         then: 'cm-handles are created'
             1 * objectUnderTest.parseAndCreateCmHandlesInDmiRegistrationAndSyncModules(*_)
         then: 'cm-handles are updated'
             'schema-set does not exist'                         | false
     }
 
-    def 'Remove CmHandle: All cm-handles delete requests are processed'() {
+    def 'Remove CmHandle: Partial Success'() {
         given: 'a registration with three cm-handles to be deleted'
             def dmiPluginRegistration = new DmiPluginRegistration(dmiPlugin: 'my-server',
                 removedCmHandles: ['cmhandle1', 'cmhandle2', 'cmhandle3'])
             def response = objectUnderTest.updateDmiRegistrationAndSyncModule(dmiPluginRegistration)
         then: 'a response is received for all cm-handles'
             response.getRemovedCmHandles().size() == 3
+        and: 'successfully de-registered cm handle entries are removed from in progress map'
+            1 * mockModuleSyncStartedOnCmHandles.remove('cmhandle1')
+            1 * mockModuleSyncStartedOnCmHandles.remove('cmhandle3')
+        and: 'failed de-registered cm handle entries should not be removed from in progress map'
+            0 * mockModuleSyncStartedOnCmHandles.remove('cmhandle2')
         and: '1st and 3rd cm-handle deletes successfully'
             with(response.getRemovedCmHandles().get(0)) {
                 assert it.status == Status.SUCCESS
 
     def getObjectUnderTest() {
         return Spy(new NetworkCmProxyDataServiceImpl(spiedJsonObjectMapper, mockDmiDataOperations,
-            mockNetworkCmProxyDataServicePropertyHandler, mockInventoryPersistence, mockCmhandleQueries,
-                stubbedNetworkCmProxyCmHandlerQueryService, mockLcmEventsCmHandleStateHandler, mockCpsDataService))
+                mockNetworkCmProxyDataServicePropertyHandler, mockInventoryPersistence, mockCmhandleQueries,
+                stubbedNetworkCmProxyCmHandlerQueryService, mockLcmEventsCmHandleStateHandler, mockCpsDataService,
+                mockModuleSyncStartedOnCmHandles))
     }
 }
 
 
 package org.onap.cps.ncmp.api.impl
 
+import com.hazelcast.map.IMap
 import org.onap.cps.ncmp.api.NetworkCmProxyCmHandlerQueryService
 import org.onap.cps.ncmp.api.impl.event.lcm.LcmEventsCmHandleStateHandler
 import org.onap.cps.ncmp.api.impl.yangmodels.YangModelCmHandle
 import org.onap.cps.spi.exceptions.DataValidationException
 import org.onap.cps.spi.model.CmHandleQueryServiceParameters
 import spock.lang.Shared
-
 import java.util.stream.Collectors
-
-import static org.onap.cps.ncmp.api.impl.operations.DmiOperations.DataStoreEnum.PASSTHROUGH_OPERATIONAL
-import static org.onap.cps.ncmp.api.impl.operations.DmiOperations.DataStoreEnum.PASSTHROUGH_RUNNING
-import static org.onap.cps.ncmp.api.impl.operations.DmiRequestBody.OperationEnum.CREATE
-import static org.onap.cps.ncmp.api.impl.operations.DmiRequestBody.OperationEnum.UPDATE
-
 import org.onap.cps.utils.JsonObjectMapper
 import com.fasterxml.jackson.databind.ObjectMapper
 import org.onap.cps.api.CpsDataService
 import org.springframework.http.ResponseEntity
 import spock.lang.Specification
 
+import static org.onap.cps.ncmp.api.impl.operations.DmiOperations.DataStoreEnum.PASSTHROUGH_OPERATIONAL
+import static org.onap.cps.ncmp.api.impl.operations.DmiOperations.DataStoreEnum.PASSTHROUGH_RUNNING
+import static org.onap.cps.ncmp.api.impl.operations.DmiRequestBody.OperationEnum.CREATE
+import static org.onap.cps.ncmp.api.impl.operations.DmiRequestBody.OperationEnum.UPDATE
+
 class NetworkCmProxyDataServiceImplSpec extends Specification {
 
     def mockCpsDataService = Mock(CpsDataService)
     def mockDmiPluginRegistration = Mock(DmiPluginRegistration)
     def mockCpsCmHandlerQueryService = Mock(NetworkCmProxyCmHandlerQueryService)
     def mockLcmEventsCmHandleStateHandler = Mock(LcmEventsCmHandleStateHandler)
+    def stubModuleSyncStartedOnCmHandles = Stub(IMap<String, Object>)
 
     def NO_TOPIC = null
     def NO_REQUEST_ID = null
             mockCmHandleQueries,
             mockCpsCmHandlerQueryService,
             mockLcmEventsCmHandleStateHandler,
-            mockCpsDataService)
+            mockCpsDataService,
+            stubModuleSyncStartedOnCmHandles)
 
     def cmHandleXPath = "/dmi-registry/cm-handles[@id='testCmHandle']"
 
 
  *  SPDX-License-Identifier: Apache-2.0
  *  ============LICENSE_END=========================================================
  */
+
 package org.onap.cps.ncmp.api.impl.config.embeddedcache
+
 import com.hazelcast.core.Hazelcast
+import com.hazelcast.map.IMap
 import org.onap.cps.spi.model.DataNode
 import org.springframework.beans.factory.annotation.Autowired
 import org.springframework.boot.test.context.SpringBootTest
     private BlockingQueue<DataNode> moduleSyncWorkQueue
 
     @Autowired
-    private Map<String, Object> moduleSyncStartedOnCmHandles
+    private IMap<String, Object> moduleSyncStartedOnCmHandles
 
     @Autowired
     private Map<String, Boolean> dataSyncSemaphores
 
 
 package org.onap.cps.ncmp.api.inventory.sync
 
+import com.hazelcast.config.Config
+import com.hazelcast.instance.impl.HazelcastInstanceFactory
+import com.hazelcast.map.IMap
 import org.onap.cps.ncmp.api.impl.event.lcm.LcmEventsCmHandleStateHandler
-import org.onap.cps.ncmp.api.impl.utils.YangDataConverter
 import org.onap.cps.ncmp.api.impl.yangmodels.YangModelCmHandle
 import org.onap.cps.ncmp.api.inventory.CmHandleState
 import org.onap.cps.ncmp.api.inventory.CompositeState
 
     def mockLcmEventsCmHandleStateHandler = Mock(LcmEventsCmHandleStateHandler)
 
+    IMap<String, Object> moduleSyncStartedOnCmHandles = HazelcastInstanceFactory
+            .getOrCreateHazelcastInstance(new Config('hazelcastInstanceName'))
+            .getMap('mapInstanceName')
+
     def batchCount = new AtomicInteger(5)
 
-    def objectUnderTest = new ModuleSyncTasks(mockInventoryPersistence, mockSyncUtils, mockModuleSyncService, mockLcmEventsCmHandleStateHandler)
+    def objectUnderTest = new ModuleSyncTasks(mockInventoryPersistence, mockSyncUtils, mockModuleSyncService,
+            mockLcmEventsCmHandleStateHandler, moduleSyncStartedOnCmHandles)
 
     def 'Module Sync ADVISED cm handles.'() {
         given: 'cm handles in an ADVISED state'
                     .withLockReason(LockReasonCategory.LOCKED_MODULE_SYNC_FAILED, '').withLastUpdatedTimeNow().build()
             def yangModelCmHandle1 = new YangModelCmHandle(id: 'cm-handle-1', compositeState: lockedState)
             def yangModelCmHandle2 = new YangModelCmHandle(id: 'cm-handle-2', compositeState: lockedState)
+            def expectedCmHandleStatePerCmHandle = [(yangModelCmHandle1): CmHandleState.ADVISED]
+        and: 'clear in progress map'
+            resetModuleSyncStartedOnCmHandles(moduleSyncStartedOnCmHandles)
+        and: 'add cm handle entry into progress map'
+            moduleSyncStartedOnCmHandles.put('cm-handle-1', 'started')
+            moduleSyncStartedOnCmHandles.put('cm-handle-2', 'started')
         and: 'sync utils retry locked cm handle returns #isReadyForRetry'
             mockSyncUtils.isReadyForRetry(lockedState) >>> isReadyForRetry
         when: 'resetting failed cm handles'
             objectUnderTest.resetFailedCmHandles([yangModelCmHandle1, yangModelCmHandle2])
         then: 'updated to state "ADVISED" from "READY" is called as often as there are cm handles ready for retry'
-//            expectedNumberOfInvocationsToSaveCmHandleState * mockLcmEventsCmHandleStateHandler.updateCmHandleState(_, CmHandleState.ADVISED)
+            expectedNumberOfInvocationsToUpdateCmHandleState * mockLcmEventsCmHandleStateHandler.updateCmHandleStateBatch(expectedCmHandleStatePerCmHandle)
+        and: 'after reset performed size of in progress map'
+            assert moduleSyncStartedOnCmHandles.size() == inProgressMapSize
         where:
-            scenario                        | isReadyForRetry || expectedNumberOfInvocationsToSaveCmHandleState
-            'retry locked cm handle once'   | [true, false]   || 1
-            'retry locked cm handle twice'  | [true, true]    || 2
-            'do not retry locked cm handle' | [false, false]  || 0
+            scenario                        | isReadyForRetry | inProgressMapSize || expectedNumberOfInvocationsToUpdateCmHandleState
+            'retry locked cm handle'        | [true, false]   | 1                 || 1
+            'do not retry locked cm handle' | [false, false]  | 2                 || 0
+    }
+
+    def 'Module Sync ADVISED cm handle without entry in progress map.'() {
+        given: 'cm handles in an ADVISED state'
+            def cmHandle1 = advisedCmHandleAsDataNode('cm-handle-1')
+        and: 'the inventory persistence cm handle returns a ADVISED state for the any handle'
+            mockInventoryPersistence.getCmHandleState(_) >> new CompositeState(cmHandleState: CmHandleState.ADVISED)
+        and: 'entry in progress map for other cm handle'
+            moduleSyncStartedOnCmHandles.put('other-cm-handle', 'started')
+        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') }
+        and: 'the entry for other cm handle is still in the progress map'
+            assert moduleSyncStartedOnCmHandles.get('other-cm-handle') != null
     }
 
     def advisedCmHandleAsDataNode(cmHandleId) {
         }
         return true
     }
+
+    def resetModuleSyncStartedOnCmHandles(moduleSyncStartedOnCmHandles) {
+        moduleSyncStartedOnCmHandles.clear();
+    }
 }
 
 
 package org.onap.cps.ncmp.api.inventory.sync
 
+import com.hazelcast.map.IMap
 import org.onap.cps.ncmp.api.impl.yangmodels.YangModelCmHandle
 import org.onap.cps.ncmp.api.inventory.sync.executor.AsyncTaskExecutor
 import java.util.concurrent.ArrayBlockingQueue
-import java.util.concurrent.BlockingQueue
 import org.onap.cps.spi.model.DataNode
 import spock.lang.Specification
 
 
     def moduleSyncWorkQueue = new ArrayBlockingQueue(testQueueCapacity)
 
-    def moduleSyncStartedOnCmHandles = [:]
+    def stubModuleSyncStartedOnCmHandles = Stub(IMap<String, Object>)
 
     def mockModuleSyncTasks = Mock(ModuleSyncTasks)
 
     def spiedAsyncTaskExecutor = Spy(AsyncTaskExecutor)
 
-    def objectUnderTest = new ModuleSyncWatchdog(mockSyncUtils, moduleSyncWorkQueue , moduleSyncStartedOnCmHandles,
+    def objectUnderTest = new ModuleSyncWatchdog(mockSyncUtils, moduleSyncWorkQueue , stubModuleSyncStartedOnCmHandles,
             mockModuleSyncTasks, spiedAsyncTaskExecutor)
 
     void setup() {