summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
-rw-r--r--cps-application/src/main/resources/application.yml38
-rwxr-xr-xcps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImpl.java21
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/config/NcmpConfiguration.java6
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/config/embeddedcache/SynchronizationCacheConfig.java6
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/DataSyncWatchdog.java2
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncTasks.java27
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncWatchdog.java11
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/executor/AsyncTaskExecutor.java2
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplRegistrationSpec.groovy16
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplSpec.groovy17
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/config/embeddedcache/SynchronizationCacheConfigSpec.groovy5
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncTasksSpec.groovy47
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncWatchdogSpec.groovy6
-rw-r--r--cps-ncmp-service/src/test/resources/application.yml19
-rw-r--r--cps-ri/src/main/java/org/onap/cps/spi/repository/FragmentRepositoryCpsPathQueryImpl.java14
-rw-r--r--cps-ri/src/test/groovy/org/onap/cps/spi/impl/CpsDataPersistenceQueryDataNodeSpec.groovy25
-rw-r--r--cps-ri/src/test/resources/data/cps-path-query.sql4
-rw-r--r--cps-service/src/main/java/org/onap/cps/utils/YangUtils.java12
-rw-r--r--cps-service/src/test/groovy/org/onap/cps/utils/YangUtilsSpec.groovy12
-rw-r--r--docker-compose/README.md7
-rw-r--r--docker-compose/docker-compose.yml4
-rw-r--r--docs/admin-guide.rst17
-rw-r--r--docs/cps-path.rst2
-rw-r--r--docs/deployment.rst22
-rwxr-xr-xdocs/release-notes.rst75
-rw-r--r--releases/3.1.0-container.yaml6
-rw-r--r--releases/3.1.0.yaml2
27 files changed, 283 insertions, 142 deletions
diff --git a/cps-application/src/main/resources/application.yml b/cps-application/src/main/resources/application.yml
index f7a06c53c..8b932f61d 100644
--- a/cps-application/src/main/resources/application.yml
+++ b/cps-application/src/main/resources/application.yml
@@ -159,22 +159,22 @@ logging:
springframework: INFO
onap:
cps: INFO
-
-dmi:
- auth:
- username: ${DMI_USERNAME}
- password: ${DMI_PASSWORD}
- api:
- base-path: dmi
-
-timers:
- advised-modules-sync:
- sleep-time-ms: 5000
- locked-modules-sync:
- sleep-time-ms: 300000
- cm-handle-data-sync:
- sleep-time-ms: 30000
-
-modules-sync-watchdog:
- async-executor:
- parallelism-level: 10 \ No newline at end of file
+ncmp:
+ dmi:
+ auth:
+ username: ${DMI_USERNAME}
+ password: ${DMI_PASSWORD}
+ api:
+ base-path: dmi
+
+ timers:
+ advised-modules-sync:
+ sleep-time-ms: 5000
+ locked-modules-sync:
+ sleep-time-ms: 300000
+ cm-handle-data-sync:
+ sleep-time-ms: 30000
+
+ modules-sync-watchdog:
+ async-executor:
+ parallelism-level: 10 \ No newline at end of file
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImpl.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImpl.java
index 3f440d65b..098388980 100755
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImpl.java
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImpl.java
@@ -27,6 +27,7 @@ import static org.onap.cps.ncmp.api.impl.constants.DmiRegistryConstants.NFP_OPER
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;
@@ -77,20 +78,14 @@ import org.springframework.stereotype.Service;
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(
@@ -329,7 +324,7 @@ public class NetworkCmProxyDataServiceImpl implements NetworkCmProxyDataService
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);
@@ -353,9 +348,17 @@ public class NetworkCmProxyDataServiceImpl implements NetworkCmProxyDataService
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>
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/config/NcmpConfiguration.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/config/NcmpConfiguration.java
index f250ddb73..6decaf844 100644
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/config/NcmpConfiguration.java
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/config/NcmpConfiguration.java
@@ -46,11 +46,11 @@ public class NcmpConfiguration {
@Getter
@Component
public static class DmiProperties {
- @Value("${dmi.auth.username}")
+ @Value("${ncmp.dmi.auth.username}")
private String authUsername;
- @Value("${dmi.auth.password}")
+ @Value("${ncmp.dmi.auth.password}")
private String authPassword;
- @Value("${dmi.api.base-path}")
+ @Value("${ncmp.dmi.api.base-path}")
private String dmiBasePath;
}
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/config/embeddedcache/SynchronizationCacheConfig.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/config/embeddedcache/SynchronizationCacheConfig.java
index abde4c2d5..c89388b29 100644
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/config/embeddedcache/SynchronizationCacheConfig.java
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/config/embeddedcache/SynchronizationCacheConfig.java
@@ -26,7 +26,7 @@ import com.hazelcast.config.NamedConfig;
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;
@@ -62,7 +62,7 @@ public class SynchronizationCacheConfig {
* @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");
}
@@ -73,7 +73,7 @@ public class SynchronizationCacheConfig {
* @return configured map of data sync semaphores
*/
@Bean
- public Map<String, Boolean> dataSyncSemaphores() {
+ public IMap<String, Boolean> dataSyncSemaphores() {
return createHazelcastInstance("dataSyncSemaphores", dataSyncSemaphoresConfig)
.getMap("dataSyncSemaphores");
}
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/DataSyncWatchdog.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/DataSyncWatchdog.java
index 107f8a04b..9336c3b21 100644
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/DataSyncWatchdog.java
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/DataSyncWatchdog.java
@@ -52,7 +52,7 @@ public class DataSyncWatchdog {
* Execute Cm Handle poll which queries the cm handle state in 'READY' and Operational Datastore Sync State in
* 'UNSYNCHRONIZED'.
*/
- @Scheduled(fixedDelayString = "${timers.cm-handle-data-sync.sleep-time-ms:30000}")
+ @Scheduled(fixedDelayString = "${ncmp.timers.cm-handle-data-sync.sleep-time-ms:30000}")
public void executeUnSynchronizedReadyCmHandlePoll() {
syncUtils.getUnsynchronizedReadyCmHandles().forEach(unSynchronizedReadyCmHandle -> {
final String cmHandleId = unSynchronizedReadyCmHandle.getId();
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncTasks.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncTasks.java
index f914547a5..004ef289a 100644
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncTasks.java
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncTasks.java
@@ -20,6 +20,7 @@
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;
@@ -46,15 +47,14 @@ public class ModuleSyncTasks {
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,
@@ -71,7 +71,7 @@ public class ModuleSyncTasks {
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());
@@ -84,28 +84,28 @@ public class ModuleSyncTasks {
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,
@@ -113,4 +113,9 @@ public class ModuleSyncTasks {
advisedCmHandle.getCompositeState().setLockReason(lockReason);
}
+ private void removeResetCmHandleFromModuleSyncMap(final String resetCmHandleId) {
+ if (moduleSyncStartedOnCmHandles.remove(resetCmHandleId) != null) {
+ log.debug("{} removed from in progress map", resetCmHandleId);
+ }
+ }
}
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncWatchdog.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncWatchdog.java
index 64d111f99..b96889fc5 100644
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncWatchdog.java
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncWatchdog.java
@@ -21,10 +21,10 @@
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;
@@ -44,7 +44,7 @@ public class ModuleSyncWatchdog {
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;
@@ -60,7 +60,7 @@ public class ModuleSyncWatchdog {
* This method will only finish when there are no more 'ADVISED' cm handles in the DB.
* This method wil be triggered on a configurable interval
*/
- @Scheduled(fixedDelayString = "${timers.advised-modules-sync.sleep-time-ms:5000}")
+ @Scheduled(fixedDelayString = "${ncmp.timers.advised-modules-sync.sleep-time-ms:5000}")
public void moduleSyncAdvisedCmHandles() {
log.info("Processing module sync watchdog waking up.");
populateWorkQueueIfNeeded();
@@ -72,8 +72,7 @@ public class ModuleSyncWatchdog {
nextBatch.size(), batchCounter.get());
asyncTaskExecutor.executeTask(() ->
moduleSyncTasks.performModuleSync(nextBatch, batchCounter),
- ASYNC_TASK_TIMEOUT_IN_MILLISECONDS
- );
+ ASYNC_TASK_TIMEOUT_IN_MILLISECONDS);
batchCounter.getAndIncrement();
} else {
preventBusyWait();
@@ -84,7 +83,7 @@ public class ModuleSyncWatchdog {
/**
* Find any failed (locked) cm handles and change state back to 'ADVISED'.
*/
- @Scheduled(fixedDelayString = "${timers.locked-modules-sync.sleep-time-ms:300000}")
+ @Scheduled(fixedDelayString = "${ncmp.timers.locked-modules-sync.sleep-time-ms:300000}")
public void resetPreviouslyFailedCmHandles() {
log.info("Processing module sync retry-watchdog waking up.");
final List<YangModelCmHandle> failedCmHandles = syncUtils.getModuleSyncFailedCmHandles();
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/executor/AsyncTaskExecutor.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/executor/AsyncTaskExecutor.java
index 7b4d2cfaa..cac78af05 100644
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/executor/AsyncTaskExecutor.java
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/executor/AsyncTaskExecutor.java
@@ -37,7 +37,7 @@ import org.springframework.stereotype.Service;
@Service
public class AsyncTaskExecutor {
- @Value("${modules-sync-watchdog.async-executor.parallelism-level:10}")
+ @Value("${ncmp.modules-sync-watchdog.async-executor.parallelism-level:10}")
@Getter
private int asyncTaskParallelismLevel;
private ExecutorService executorService;
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplRegistrationSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplRegistrationSpec.groovy
index 0b58d4419..3a0f3c73b 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplRegistrationSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplRegistrationSpec.groovy
@@ -22,6 +22,7 @@
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
@@ -63,6 +64,7 @@ class NetworkCmProxyDataServiceImplRegistrationSpec extends Specification {
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'() {
@@ -76,6 +78,8 @@ class NetworkCmProxyDataServiceImplRegistrationSpec extends Specification {
// 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'
@@ -268,7 +272,7 @@ class NetworkCmProxyDataServiceImplRegistrationSpec extends Specification {
'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'])
@@ -278,6 +282,11 @@ class NetworkCmProxyDataServiceImplRegistrationSpec extends Specification {
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
@@ -347,7 +356,8 @@ class NetworkCmProxyDataServiceImplRegistrationSpec extends Specification {
def getObjectUnderTest() {
return Spy(new NetworkCmProxyDataServiceImpl(spiedJsonObjectMapper, mockDmiDataOperations,
- mockNetworkCmProxyDataServicePropertyHandler, mockInventoryPersistence, mockCmhandleQueries,
- stubbedNetworkCmProxyCmHandlerQueryService, mockLcmEventsCmHandleStateHandler, mockCpsDataService))
+ mockNetworkCmProxyDataServicePropertyHandler, mockInventoryPersistence, mockCmhandleQueries,
+ stubbedNetworkCmProxyCmHandlerQueryService, mockLcmEventsCmHandleStateHandler, mockCpsDataService,
+ mockModuleSyncStartedOnCmHandles))
}
}
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplSpec.groovy
index def0db32d..91eb218ea 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplSpec.groovy
@@ -22,6 +22,7 @@
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
@@ -39,14 +40,7 @@ import org.onap.cps.spi.exceptions.CpsException
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
@@ -57,6 +51,11 @@ import org.springframework.http.HttpStatus
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)
@@ -68,6 +67,7 @@ class NetworkCmProxyDataServiceImplSpec extends Specification {
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
@@ -84,7 +84,8 @@ class NetworkCmProxyDataServiceImplSpec extends Specification {
mockCmHandleQueries,
mockCpsCmHandlerQueryService,
mockLcmEventsCmHandleStateHandler,
- mockCpsDataService)
+ mockCpsDataService,
+ stubModuleSyncStartedOnCmHandles)
def cmHandleXPath = "/dmi-registry/cm-handles[@id='testCmHandle']"
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/config/embeddedcache/SynchronizationCacheConfigSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/config/embeddedcache/SynchronizationCacheConfigSpec.groovy
index 80aa81b04..4cfc02b9e 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/config/embeddedcache/SynchronizationCacheConfigSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/config/embeddedcache/SynchronizationCacheConfigSpec.groovy
@@ -17,8 +17,11 @@
* 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
@@ -34,7 +37,7 @@ class SynchronizationCacheConfigSpec extends Specification {
private BlockingQueue<DataNode> moduleSyncWorkQueue
@Autowired
- private Map<String, Object> moduleSyncStartedOnCmHandles
+ private IMap<String, Object> moduleSyncStartedOnCmHandles
@Autowired
private Map<String, Boolean> dataSyncSemaphores
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncTasksSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncTasksSpec.groovy
index 67fb89dbb..3deab112a 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncTasksSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncTasksSpec.groovy
@@ -21,8 +21,10 @@
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
@@ -43,9 +45,14 @@ class ModuleSyncTasksSpec extends Specification {
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'
@@ -95,17 +102,39 @@ class ModuleSyncTasksSpec extends Specification {
.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) {
@@ -131,4 +160,8 @@ class ModuleSyncTasksSpec extends Specification {
}
return true
}
+
+ def resetModuleSyncStartedOnCmHandles(moduleSyncStartedOnCmHandles) {
+ moduleSyncStartedOnCmHandles.clear();
+ }
}
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncWatchdogSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncWatchdogSpec.groovy
index dd989bf67..a8bbf7c48 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncWatchdogSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncWatchdogSpec.groovy
@@ -21,10 +21,10 @@
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
@@ -36,13 +36,13 @@ class ModuleSyncWatchdogSpec extends 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() {
diff --git a/cps-ncmp-service/src/test/resources/application.yml b/cps-ncmp-service/src/test/resources/application.yml
index 03d70c26c..8d8bfaf9b 100644
--- a/cps-ncmp-service/src/test/resources/application.yml
+++ b/cps-ncmp-service/src/test/resources/application.yml
@@ -16,13 +16,14 @@
# SPDX-License-Identifier: Apache-2.0
# ============LICENSE_END=========================================================
-dmi:
- auth:
- username: some-user
- password: some-password
- api:
- base-path: dmi
+ncmp:
+ dmi:
+ auth:
+ username: some-user
+ password: some-password
+ api:
+ base-path: dmi
-modules-sync-watchdog:
- async-executor:
- parallelism-level: 3 \ No newline at end of file
+ modules-sync-watchdog:
+ async-executor:
+ parallelism-level: 3 \ No newline at end of file
diff --git a/cps-ri/src/main/java/org/onap/cps/spi/repository/FragmentRepositoryCpsPathQueryImpl.java b/cps-ri/src/main/java/org/onap/cps/spi/repository/FragmentRepositoryCpsPathQueryImpl.java
index 47a3e8f31..654c1c085 100644
--- a/cps-ri/src/main/java/org/onap/cps/spi/repository/FragmentRepositoryCpsPathQueryImpl.java
+++ b/cps-ri/src/main/java/org/onap/cps/spi/repository/FragmentRepositoryCpsPathQueryImpl.java
@@ -38,8 +38,8 @@ import org.onap.cps.utils.JsonObjectMapper;
@Slf4j
public class FragmentRepositoryCpsPathQueryImpl implements FragmentRepositoryCpsPathQuery {
- public static final String SIMILAR_TO_ABSOLUTE_PATH_PREFIX = "%/";
- public static final String SIMILAR_TO_OPTIONAL_LIST_INDEX_POSTFIX = "(\\[[^/]*])?";
+ public static final String REGEX_ABSOLUTE_PATH_PREFIX = ".*\\/";
+ public static final String REGEX_OPTIONAL_LIST_INDEX_POSTFIX = "(\\[@(?!.*\\[).*?])?$";
@PersistenceContext
private EntityManager entityManager;
@@ -51,8 +51,8 @@ public class FragmentRepositoryCpsPathQueryImpl implements FragmentRepositoryCps
final StringBuilder sqlStringBuilder = new StringBuilder("SELECT * FROM FRAGMENT WHERE anchor_id = :anchorId");
final Map<String, Object> queryParameters = new HashMap<>();
queryParameters.put("anchorId", anchorId);
- sqlStringBuilder.append(" AND xpath SIMILAR TO :xpathRegex");
- final String xpathRegex = getSimilarToXpathSqlRegex(cpsPathQuery);
+ sqlStringBuilder.append(" AND xpath ~ :xpathRegex");
+ final String xpathRegex = getXpathSqlRegex(cpsPathQuery);
queryParameters.put("xpathRegex", xpathRegex);
if (cpsPathQuery.hasLeafConditions()) {
sqlStringBuilder.append(" AND attributes @> :leafDataAsJson\\:\\:jsonb");
@@ -68,15 +68,15 @@ public class FragmentRepositoryCpsPathQueryImpl implements FragmentRepositoryCps
return fragmentEntities;
}
- private static String getSimilarToXpathSqlRegex(final CpsPathQuery cpsPathQuery) {
+ private static String getXpathSqlRegex(final CpsPathQuery cpsPathQuery) {
final StringBuilder xpathRegexBuilder = new StringBuilder();
if (CpsPathPrefixType.ABSOLUTE.equals(cpsPathQuery.getCpsPathPrefixType())) {
xpathRegexBuilder.append(escapeXpath(cpsPathQuery.getXpathPrefix()));
} else {
- xpathRegexBuilder.append(SIMILAR_TO_ABSOLUTE_PATH_PREFIX);
+ xpathRegexBuilder.append(REGEX_ABSOLUTE_PATH_PREFIX);
xpathRegexBuilder.append(escapeXpath(cpsPathQuery.getDescendantName()));
}
- xpathRegexBuilder.append(SIMILAR_TO_OPTIONAL_LIST_INDEX_POSTFIX);
+ xpathRegexBuilder.append(REGEX_OPTIONAL_LIST_INDEX_POSTFIX);
return xpathRegexBuilder.toString();
}
diff --git a/cps-ri/src/test/groovy/org/onap/cps/spi/impl/CpsDataPersistenceQueryDataNodeSpec.groovy b/cps-ri/src/test/groovy/org/onap/cps/spi/impl/CpsDataPersistenceQueryDataNodeSpec.groovy
index 36b378a77..be2f8febf 100644
--- a/cps-ri/src/test/groovy/org/onap/cps/spi/impl/CpsDataPersistenceQueryDataNodeSpec.groovy
+++ b/cps-ri/src/test/groovy/org/onap/cps/spi/impl/CpsDataPersistenceQueryDataNodeSpec.groovy
@@ -50,7 +50,7 @@ class CpsDataPersistenceQueryDataNodeSpec extends CpsPersistenceSpecBase {
scenario | cpsPath | includeDescendantsOption || expectedNumberOfParentNodes | expectedNumberOfChildNodes
'String and no descendants' | '/shops/shop[@id=1]/categories[@code=1]/book[@title="Dune"]' | OMIT_DESCENDANTS || 1 | 0
'Integer and descendants' | '/shops/shop[@id=1]/categories[@code=1]/book[@price=5]' | INCLUDE_ALL_DESCENDANTS || 1 | 1
- 'No condition no descendants' | '/shops/shop[@id=1]/categories' | OMIT_DESCENDANTS || 2 | 0
+ 'No condition no descendants' | '/shops/shop[@id=1]/categories' | OMIT_DESCENDANTS || 3 | 0
}
@Sql([CLEAR_DATA, SET_DATA])
@@ -91,16 +91,19 @@ class CpsDataPersistenceQueryDataNodeSpec extends CpsPersistenceSpecBase {
assert result[i].getXpath() == expectedXPaths[i]
}
where: 'the following data is used'
- scenario | cpsPath || expectedXPaths
- 'fully unique descendant name' | '//categories[@code=2]' || ["/shops/shop[@id='1']/categories[@code='2']", "/shops/shop[@id='2']/categories[@code='1']", "/shops/shop[@id='2']/categories[@code='2']"]
- 'descendant name match end of other node' | '//book' || ["/shops/shop[@id='1']/categories[@code='1']/book", "/shops/shop[@id='1']/categories[@code='2']/book"]
- 'descendant with text condition on leaf' | '//book/title[text()="Chapters"]' || ["/shops/shop[@id='1']/categories[@code='2']/book"]
- 'descendant with text condition case mismatch' | '//book/title[text()="chapters"]' || []
- 'descendant with text condition on int leaf' | '//book/price[text()="5"]' || ["/shops/shop[@id='1']/categories[@code='1']/book"]
- 'descendant with text condition on leaf-list' | '//book/labels[text()="special offer"]' || ["/shops/shop[@id='1']/categories[@code='1']/book"]
- 'descendant with text condition partial match' | '//book/labels[text()="special"]' || []
- 'descendant with text condition (existing) empty string' | '//book/labels[text()=""]' || ["/shops/shop[@id='1']/categories[@code='1']/book"]
- 'descendant with text condition on int leaf-list' | '//book/editions[text()="2000"]' || ["/shops/shop[@id='1']/categories[@code='2']/book"]
+ scenario | cpsPath || expectedXPaths
+ 'fully unique descendant name' | '//categories[@code=2]' || ["/shops/shop[@id='1']/categories[@code='2']", "/shops/shop[@id='2']/categories[@code='1']", "/shops/shop[@id='2']/categories[@code='2']"]
+ 'descendant name match end of other node' | '//book' || ["/shops/shop[@id='1']/categories[@code='1']/book", "/shops/shop[@id='1']/categories[@code='2']/book"]
+ 'descendant with text condition on leaf' | '//book/title[text()="Chapters"]' || ["/shops/shop[@id='1']/categories[@code='2']/book"]
+ 'descendant with text condition case mismatch' | '//book/title[text()="chapters"]' || []
+ 'descendant with text condition on int leaf' | '//book/price[text()="5"]' || ["/shops/shop[@id='1']/categories[@code='1']/book"]
+ 'descendant with text condition on leaf-list' | '//book/labels[text()="special offer"]' || ["/shops/shop[@id='1']/categories[@code='1']/book"]
+ 'descendant with text condition partial match' | '//book/labels[text()="special"]' || []
+ 'descendant with text condition (existing) empty string' | '//book/labels[text()=""]' || ["/shops/shop[@id='1']/categories[@code='1']/book"]
+ 'descendant with text condition on int leaf-list' | '//book/editions[text()="2000"]' || ["/shops/shop[@id='1']/categories[@code='2']/book"]
+ 'descendant name match of leaf containing /' | '//categories/type[text()="text/with/slash"]' || ["/shops/shop[@id='1']/categories[@code='string/with/slash/']"]
+ 'descendant with text condition on leaf containing /' | '//categories[@code=\'string/with/slash\']' || ["/shops/shop[@id='1']/categories[@code='string/with/slash/']"]
+ 'descendant with text condition on leaf containing [' | '//book/author[@Address="String[with]square[bracket]"]'|| []
}
@Sql([CLEAR_DATA, SET_DATA])
diff --git a/cps-ri/src/test/resources/data/cps-path-query.sql b/cps-ri/src/test/resources/data/cps-path-query.sql
index b6000cffb..fa711cbbf 100644
--- a/cps-ri/src/test/resources/data/cps-path-query.sql
+++ b/cps-ri/src/test/resources/data/cps-path-query.sql
@@ -55,10 +55,12 @@ INSERT INTO FRAGMENT (ID, DATASPACE_ID, ANCHOR_ID, PARENT_ID, XPATH, ATTRIBUTES)
(2, 1001, 1003, 1, '/shops/shop[@id=''1'']', '{"id" : 1, "type" : "bookstore"}'),
(3, 1001, 1003, 2, '/shops/shop[@id=''1'']/categories[@code=''1'']', '{"code" : 1, "type" : "bookstore", "name": "SciFi"}'),
(4, 1001, 1003, 2, '/shops/shop[@id=''1'']/categories[@code=''2'']', '{"code" : 2, "type" : "bookstore", "name": "Fiction"}'),
+ (31, 1001, 1003, 2, '/shops/shop[@id=''1'']/categories[@code=''string/with/slash/'']', '{"code" : "string/with/slash", "type" : "text/with/slash", "name": "Fiction"}'),
(5, 1001, 1003, 3, '/shops/shop[@id=''1'']/categories[@code=''1'']/book', '{"price" : 5, "title" : "Dune", "labels" : ["special offer","classics",""]}'),
(6, 1001, 1003, 4, '/shops/shop[@id=''1'']/categories[@code=''2'']/book', '{"price" : 15, "title" : "Chapters", "editions" : [2000,2010,2020]}'),
(7, 1001, 1003, 5, '/shops/shop[@id=''1'']/categories[@code=''1'']/book/author[@FirstName=''Joe'' and @Surname=''Bloggs'']', '{"FirstName" : "Joe", "Surname": "Bloggs","title": "Dune"}'),
- (8, 1001, 1003, 6, '/shops/shop[@id=''1'']/categories[@code=''2'']/book/author[@FirstName=''Joe'' and @Surname=''Smith'']', '{"FirstName" : "Joe", "Surname": "Smith","title": "Chapters"}');
+ (8, 1001, 1003, 6, '/shops/shop[@id=''1'']/categories[@code=''2'']/book/author[@FirstName=''Joe'' and @Surname=''Smith'']', '{"FirstName" : "Joe", "Surname": "Smith","title": "Chapters"}'),
+ (32, 1001, 1003, 6, '/shops/shop[@id=''1'']/categories[@code=''2'']/book/author[@FirstName=''Joe'' and @Address=''string[with]square[brackets]'']', '{"FirstName" : "Joe", "Address": "string[with]square[brackets]","title": "Chapters"}');
INSERT INTO FRAGMENT (ID, DATASPACE_ID, ANCHOR_ID, PARENT_ID, XPATH, ATTRIBUTES) VALUES
(9, 1001, 1003, 1, '/shops/shop[@id=''2'']', '{"type" : "bookstore"}'),
diff --git a/cps-service/src/main/java/org/onap/cps/utils/YangUtils.java b/cps-service/src/main/java/org/onap/cps/utils/YangUtils.java
index 8c32010d7..5e1b486aa 100644
--- a/cps-service/src/main/java/org/onap/cps/utils/YangUtils.java
+++ b/cps-service/src/main/java/org/onap/cps/utils/YangUtils.java
@@ -52,7 +52,8 @@ import org.opendaylight.yangtools.yang.model.api.SchemaContext;
public class YangUtils {
private static final String XPATH_DELIMITER_REGEX = "\\/";
- private static final String XPATH_NODE_KEY_ATTRIBUTES_REGEX = "\\[.+";
+ private static final String XPATH_NODE_KEY_ATTRIBUTES_REGEX = "\\[.*?\\]";
+ //Might cause an issue with [] inside [] in key-values
/**
* Parses jsonData into NormalizedNode according to given schema context.
@@ -149,10 +150,11 @@ public class YangUtils {
}
private static String[] xpathToNodeIdSequence(final String xpath) {
- final String[] xpathNodeIdSequence = Arrays.stream(xpath.split(XPATH_DELIMITER_REGEX))
- .map(identifier -> identifier.replaceFirst(XPATH_NODE_KEY_ATTRIBUTES_REGEX, ""))
- .filter(identifier -> !identifier.isEmpty())
- .toArray(String[]::new);
+ final String[] xpathNodeIdSequence = Arrays.stream(xpath
+ .replaceAll(XPATH_NODE_KEY_ATTRIBUTES_REGEX, "")
+ .split(XPATH_DELIMITER_REGEX))
+ .filter(identifier -> !identifier.isEmpty())
+ .toArray(String[]::new);
if (xpathNodeIdSequence.length < 1) {
throw new DataValidationException("Invalid xpath.", "Xpath contains no node identifiers.");
}
diff --git a/cps-service/src/test/groovy/org/onap/cps/utils/YangUtilsSpec.groovy b/cps-service/src/test/groovy/org/onap/cps/utils/YangUtilsSpec.groovy
index 25b90d702..3f190910b 100644
--- a/cps-service/src/test/groovy/org/onap/cps/utils/YangUtilsSpec.groovy
+++ b/cps-service/src/test/groovy/org/onap/cps/utils/YangUtilsSpec.groovy
@@ -115,4 +115,16 @@ class YangUtilsSpec extends Specification {
noExceptionThrown()
}
+ def 'Parsing xPath to nodeId for #scenario.'() {
+ when: 'xPath is parsed'
+ def result = YangUtils.xpathToNodeIdSequence(xPath)
+ then: 'result represents an array of expected identifiers'
+ assert result == expectedNodeIdentifier
+ where: 'the following parameters are used'
+ scenario | xPath || expectedNodeIdentifier
+ 'container xpath' | '/test-tree' || ['test-tree']
+ 'xpath contains list attribute' | '/test-tree/branch[@name=\'Branch\']' || ['test-tree','branch']
+ 'xpath contains list attributes with /' | '/test-tree/branch[@name=\'/Branch\']/categories[@id=\'/broken\']' || ['test-tree','branch','categories']
+ }
+
}
diff --git a/docker-compose/README.md b/docker-compose/README.md
index 623968f33..8cc6e57d5 100644
--- a/docker-compose/README.md
+++ b/docker-compose/README.md
@@ -3,6 +3,7 @@
Copyright (C) 2020 Pantheon.tech
Modifications Copyright (C) 2020-2021 Nordix Foundation.
Modifications Copyright (C) 2021 Bell Canada.
+ Modifications Copyright (C) 2022 TechMahindra Ltd.
================================================================================
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
@@ -86,7 +87,8 @@ Then CPS can be started either using a Java Archive previously built or directly
Following command starts the application using JAR file:
```bash
-DB_HOST=localhost DB_USERNAME=cps DB_PASSWORD=cps CPS_CORE_USERNAME=cpsuser CPS_CORE_PASSWORD=cpsr0cks! \
+DB_HOST=localhost DB_USERNAME=cps DB_PASSWORD=cps CPS_USERNAME=cpsuser CPS_PASSWORD=cpsr0cks! \
+ DMI_USERNAME=cpsuser DMI_PASSWORD=cpsr0cks! \
java -jar cps-application/target/cps-application-x.y.z-SNAPSHOT.jar
```
@@ -97,7 +99,8 @@ Here are the steps to run or debug the application from Intellij:
1. Enable the desired maven profile form Maven Tool Window
2. Run a configuration from `Run -> Edit configurations` with following settings:
* `Environment variables`: `DB_HOST=localhost;DB_USERNAME=cps;DB_PASSWORD=cps
- CPS_CORE_USERNAME=cpsuser CPS_CORE_PASSWORD=cpsr0cks!`
+ CPS_USERNAME=cpsuser CPS_PASSWORD=cpsr0cks!
+ DMI_USERNAME=cpsuser DMI_PASSWORD=cpsr0cks!`
## Accessing services
diff --git a/docker-compose/docker-compose.yml b/docker-compose/docker-compose.yml
index ca7795f2f..94ffe6182 100644
--- a/docker-compose/docker-compose.yml
+++ b/docker-compose/docker-compose.yml
@@ -49,8 +49,8 @@ services:
notification.enabled: 'true'
notification.async.executor.time-out-value-in-ms: 2000
NOTIFICATION_DATASPACE_FILTER_PATTERNS: '.*'
- TIMERS_ADVISED-MODULES-SYNC_SLEEP-TIME-MS: ${ADVISED_MODULES_SYNC_SLEEP_TIME_MS:-30000}
- TIMERS_CM-HANDLE-DATA-SYNC_SLEEP-TIME-MS: ${CMHANDLE_DATA_SYNC_SLEEP_TIME_MS:-30000}
+ NCMP_TIMERS_ADVISED-MODULES-SYNC_SLEEP-TIME-MS: ${ADVISED_MODULES_SYNC_SLEEP_TIME_MS:-30000}
+ NCMP_TIMERS_CM-HANDLE-DATA-SYNC_SLEEP-TIME-MS: ${CMHANDLE_DATA_SYNC_SLEEP_TIME_MS:-30000}
restart: unless-stopped
depends_on:
- dbpostgresql
diff --git a/docs/admin-guide.rst b/docs/admin-guide.rst
index efa23b503..eeac01377 100644
--- a/docs/admin-guide.rst
+++ b/docs/admin-guide.rst
@@ -34,15 +34,14 @@ CPS Log pattern
<pattern>
{
- "timestamp" : "%timestamp", // 2022-01-28 18:39:17.768
- "severity": "%level", // DEBUG
- "service": "${springAppName}", // cps-application
- "trace": "${TraceId}", // e17da1571e518c59
- "span": "${SpanId}", // e17da1571e518c59
- "pid": "${PID}", //11128
- "thread": "%thread", //tp1901272535-29
+ "logTimeStamp" : "%timestamp", // 2022-01-28 18:39:17.768
+ "logLevel": "%level", // DEBUG
+ "principalId": "%userId", // cpsuser
+ "serviceName": "${springAppName}", // cps-application
+ "message":"%message", // Execution time ...
+ "processId": "${PID}", //11128
+ "threadName": "%thread", //tp1901272535-29
"class": "%logger{40}", .// o.onap.cps.aop.CpsLoggingAspectService
- "rest": "%message" // Execution time ...
}
</pattern>
@@ -198,7 +197,7 @@ Prometheus Metrics can be checked at the following endpoint
Naming Validation
-----------------
-As part of the Jakarta 3.1.0 release, CPS has added validation to the names of the following components:
+As part of the Kohn 3.1.0 release, CPS has added validation to the names of the following components:
- Dataspace names
- Schema Set names
diff --git a/docs/cps-path.rst b/docs/cps-path.rst
index 44662d893..f321adfa9 100644
--- a/docs/cps-path.rst
+++ b/docs/cps-path.rst
@@ -241,6 +241,7 @@ leaf-conditions
- The key should be supplied with correct data type for it to be queried from DB. In the last example above the attribute code is of type
Integer so the cps query will not work if the value is passed as string.
eg: ``//categories[@code="1"]`` or ``//categories[@code='1']`` will not work because the key attribute code is treated a string.
+ - Having '[' token in any index in any list will have a negative impact on this function.
**Notes**
- For performance reasons it does not make sense to query using key leaf as attribute. If the key value is known it is better to execute a get request with the complete xpath.
@@ -265,6 +266,7 @@ The text()-condition can be added to any CPS path query.
- Only string and integer values are supported, boolean and float values are not supported.
- Since CPS cannot return individual leaves it will always return the container with all its leaves. Ancestor-axis can be used to specify a parent higher up the tree.
- When querying a leaf value (instead of leaf-list) it is better, more performant to use a text value condition use @<leaf-name> as described above.
+ - Having '[' token in any index in any list will have a negative impact on this function.
ancestor-axis
-------------
diff --git a/docs/deployment.rst b/docs/deployment.rst
index 9c9a9ff7c..82c54f12c 100644
--- a/docs/deployment.rst
+++ b/docs/deployment.rst
@@ -197,8 +197,8 @@ Any spring supported property can be configured by providing in ``config.additio
| logging.level | Logging level set in cps-core | info |
| | | |
+---------------------------------------+---------------------------------------------------------------------------------------------------------+-------------------------------+
-| config.useStrimziKafka | If targeting a custom kafka cluster, ie useStrimziKakfa: false, the config.eventPublisher.spring.kafka | true |
-| | values must be set. | |
+| config.useStrimziKafka | If targeting a custom kafka cluster, ie useStrimziKafka: false, the config.eventPublisher.spring.kafka | true |
+| | values below must be set. | |
+---------------------------------------+---------------------------------------------------------------------------------------------------------+-------------------------------+
| config.eventPublisher. | Kafka hostname and port | ``<kafka-bootstrap>:9092`` |
| spring.kafka.bootstrap-servers | | |
@@ -251,7 +251,7 @@ Any spring supported property can be configured by providing in ``config.additio
| enabled-dataspaces | | |
+---------------------------------------+---------------------------------------------------------------------------------------------------------+-------------------------------+
| config.additional. | If asynchronous messaging, user notifications, and updated event persistence should be enabled | ``true`` |
-| notification.enabled | | |
+| notification.data-updated.enabled | | |
+---------------------------------------+---------------------------------------------------------------------------------------------------------+-------------------------------+
| config.additional. | Core pool size in asynchronous execution of notification. | ``2`` |
| notification.async.executor. | | |
@@ -269,7 +269,7 @@ Any spring supported property can be configured by providing in ``config.additio
| notification.async.executor. | | |
| wait-for-tasks-to-complete-on-shutdown| | |
+---------------------------------------+---------------------------------------------------------------------------------------------------------+-------------------------------+
-| config.additional. | Prefix to be added to the thread name in asynchronous execution of notifications. | ``async_`` |
+| config.additional. | Prefix to be added to the thread name in asynchronous execution of notifications. | ``Async-`` |
| notification.async.executor. | | |
| thread-name-prefix | | |
+---------------------------------------+---------------------------------------------------------------------------------------------------------+-------------------------------+
@@ -291,17 +291,17 @@ Additional CPS-NCMP Customizations
| | | |
| | See also :ref:`cps_common_credentials_retrieval`. | |
+---------------------------------------+---------------------------------------------------------------------------------------------------------+-------------------------------+
-| config.timers.advised-modules-sync | Specifies the delay in milliseconds in which the module sync watch dog will wake again after finishing. | ``30000`` |
-| .sleep-time-ms | | |
+| config.ncmp.timers | Specifies the delay in milliseconds in which the module sync watch dog will wake again after finishing. | ``30000`` |
+| .advised-modules-sync.sleep-time-ms | | |
| | | |
+---------------------------------------+---------------------------------------------------------------------------------------------------------+-------------------------------+
-| config.timers.locked-modules-sync | Specifies the delay in milliseconds in which the retry mechanism watch dog | |
-| | will wake again after finishing. | ``300000`` |
-| .sleep-time-ms | | |
+| config.ncmp.timers | Specifies the delay in milliseconds in which the retry mechanism watch dog | |
+| .locked-modules-sync.sleep-time-ms | will wake again after finishing. | ``300000`` |
+| | | |
| | | |
+---------------------------------------+---------------------------------------------------------------------------------------------------------+-------------------------------+
-| config.timers.cm-handle-data-sync | Specifies the delay in milliseconds in which the data sync watch dog will wake again after finishing. | ``30000`` |
-| .sleep-time-ms | | |
+| config.ncmp.timers | Specifies the delay in milliseconds in which the data sync watch dog will wake again after finishing. | ``30000`` |
+| .cm-handle-data-sync.sleep-time-ms | | |
| | | |
+---------------------------------------+---------------------------------------------------------------------------------------------------------+-------------------------------+
diff --git a/docs/release-notes.rst b/docs/release-notes.rst
index d4f984361..6752a4023 100755
--- a/docs/release-notes.rst
+++ b/docs/release-notes.rst
@@ -16,8 +16,8 @@ CPS Release Notes
.. * * * KOHN * * *
.. ====================
-Version: 3.1.0 (not released yet)
-=================================
+Version: 3.1.0
+==============
Release Data
------------
@@ -32,18 +32,81 @@ Release Data
| **Release designation** | 3.1.0 Kohn |
| | |
+--------------------------------------+--------------------------------------------------------+
-| **Release date** | N/A |
+| **Release date** | 2022 September 14 |
| | |
+--------------------------------------+--------------------------------------------------------+
Features
--------
- - `CPS-322 <https://jira.onap.org/browse/CPS-322>`_ Implement additional validation for names and identifiers
- - `CPS-1136 <https://jira.onap.org/browse/CPS-1136>`_ Get all cm handles by DMI plugin Identifier
- - `CPS-1001 <https://jira.onap.org/browse/CPS-1001>`_ Add CPS-E-05 endpoint for Read data, NCMP-Operational Datastore
+ - `CPS-340 <https://jira.onap.org/browse/CPS-340>`_ Patch and update the root data node
+ - `CPS-575 <https://jira.onap.org/browse/CPS-575>`_ Write data for cmHandle using ncmp-datastores:passthrough-running (NCMP.)
+ - `CPS-731 <https://jira.onap.org/browse/CPS-731>`_ Query based on Public CM Properties
+ - `CPS-828 <https://jira.onap.org/browse/CPS-828>`_ Async: NCMP Rest impl. including Request ID generation
+ - `CPS-829 <https://jira.onap.org/browse/CPS-829>`_ Async: Internal message topic incl. basic producer & Consumer
+ - `CPS-830 <https://jira.onap.org/browse/CPS-830>`_ DMI-NCMP Asynchronously Publish Response Event to Client Topic
+ - `CPS-869 <https://jira.onap.org/browse/CPS-869>`_ Apply Standardized logging fields to adhere to ONAP Best practice REQ-1072
+ - `CPS-870 <https://jira.onap.org/browse/CPS-870>`_ Align CPS-Core output with SDN-C output (add module name)
+ - `CPS-875 <https://jira.onap.org/browse/CPS-875>`_ CM Handle State: Watchdog-process that syncs 'ADVISED' CM Handles
+ - `CPS-877 <https://jira.onap.org/browse/CPS-877>`_ CM Handle State: Exclude any CM-Handles from queries/operations that are not in state 'READY'
+ - `CPS-899 <https://jira.onap.org/browse/CPS-899>`_ Start and stop sessions on Java API
+ - `CPS-909 <https://jira.onap.org/browse/CPS-909>`_ Separate NCMP endpoint for ch/{cm-handle}/properties and ch/{cm-handle}/state
+ - `CPS-917 <https://jira.onap.org/browse/CPS-917>`_ Structured Errors response for passthrough use-cases in NCMP
+ - `CPS-953 <https://jira.onap.org/browse/CPS-953>`_ Update maven deploy plugin version
+ - `CPS-977 <https://jira.onap.org/browse/CPS-977>`_ Query CM Handles using CpsPath
+ - `CPS-1000 <https://jira.onap.org/browse/CPS-1000>`_ Create Data Synchronization watchdog
+ - `CPS-1016 <https://jira.onap.org/browse/CPS-1016>`_ Merge 2 'query' end points in NCMP
+ - `CPS-1034 <https://jira.onap.org/browse/CPS-1034>`_ Publish lifecycle events for ADVISED , READY and LOCKED state transition"
+ - `CPS-1064 <https://jira.onap.org/browse/CPS-1064>`_ Support retrieval of YANG module sources for CM handle on the NCMP interface
+ - `CPS-1099 <https://jira.onap.org/browse/CPS-1099>`_ Expose simplified 'external' lock reason enum state over REST interface
+ - `CPS-1101 <https://jira.onap.org/browse/CPS-1101>`_ Introducing the DELETING and DELETED Cmhandle State
+ - `CPS-1102 <https://jira.onap.org/browse/CPS-1102>`_ Register the Cmhandle Sends Advised State notification.
+ - `CPS-1133 <https://jira.onap.org/browse/CPS-1133>`_ Enable/Disable Data Sync for Cm Handle
+ - `CPS-1136 <https://jira.onap.org/browse/CPS-1136>`_ DMI Audit Support (get all CM Handles for a registered DMI)
+
Bug Fixes
---------
+ - `CPS-896 <https://jira.onap.org/browse/CPS-896>`_ CM Handle Registration Process only partially completes when exception is thrown
+ - `CPS-957 <https://jira.onap.org/browse/CPS-957>`_ NCMP: fix getResourceDataForPassthroughOperational endpoint
+ - `CPS-1020 <https://jira.onap.org/browse/CPS-1020>`_ DuplicatedYangResourceException error at parallel cmHandle registration
+ - `CPS-1056 <https://jira.onap.org/browse/CPS-1056>`_ Wrong error response format in case of Dmi plugin error
+ - `CPS-1067 <https://jira.onap.org/browse/CPS-1067>`_ NCMP returns 500 error on searches endpoint when No DMi Handles registered
+ - `CPS-1085 <https://jira.onap.org/browse/CPS-1085>`_ Performance degradation on ncmp/v1/ch/searches endpoint
+ - `CPS-1088 <https://jira.onap.org/browse/CPS-1088>`_ Kafka consumer can not be turned off
+ - `CPS-1097 <https://jira.onap.org/browse/CPS-1097>`_ Unable to change state from LOCKED to ADVISED
+ - `CPS-1126 <https://jira.onap.org/browse/CPS-1126>`_ CmHandle creation performance degradation
+ - `CPS-1175 <https://jira.onap.org/browse/CPS-1175>`_ Incorrect response when empty body executed for cmhandle id-searches
+ - `CPS-1179 <https://jira.onap.org/browse/CPS-1179>`_ Node API - GET method returns invalid response when identifier contains '/'
+ - `CPS-1212 <https://jira.onap.org/browse/CPS-1212>`_ Additional Properties for CM Handles not included when send to DMI Plugin
+ - `CPS-1217 <https://jira.onap.org/browse/CPS-1217>`_ Searches endpoint gives back empty list however there are already available cmhandles
+ - `CPS-1218 <https://jira.onap.org/browse/CPS-1218>`_ NCMP logs are flooded with SyncUtils logs
+
+Known Limitations, Issues and Workarounds
+-----------------------------------------
+
+*System Limitations*
+
+Having '[' token in any index in any list will have a negative impact on the search functions leaf-conditions and text()-condition.
+Example of an xpath that would cause problems while using cps-path queries : /parent/child[@id='id[with]braces']
+
+*Known Vulnerabilities*
+
+None
+
+*Workarounds*
+
+None
+
+Security Notes
+--------------
+
+*Fixed Security Issues*
+
+ - `CPS-963 <https://jira.onap.org/browse/CPS-963>`_ Liquibase has got serious vulnerability, upgrade required
+
+*Known Security Issues*
+
+None
.. ========================
.. * * * JAKARTA * * *
diff --git a/releases/3.1.0-container.yaml b/releases/3.1.0-container.yaml
index cc11ed26d..10f34e6ac 100644
--- a/releases/3.1.0-container.yaml
+++ b/releases/3.1.0-container.yaml
@@ -1,8 +1,8 @@
distribution_type: container
container_release_tag: 3.1.0
project: cps
-log_dir: cps-maven-docker-stage-master/690/
-ref: 9697e76c319e4cf59fc494216a720393545503a9
+log_dir: cps-maven-docker-stage-master/699/
+ref: 0e7970a1ec12eac0f45fa016f98148da7a2266a8
containers:
- name: 'cps-and-ncmp'
- version: '3.1.0-20220914T140025Z'
+ version: '3.1.0-20220922T123733Z'
diff --git a/releases/3.1.0.yaml b/releases/3.1.0.yaml
index 1258b26e6..a3b6d5370 100644
--- a/releases/3.1.0.yaml
+++ b/releases/3.1.0.yaml
@@ -1,4 +1,4 @@
distribution_type: maven
-log_dir: cps-maven-stage-master/696/
+log_dir: cps-maven-stage-master/705/
project: cps
version: 3.1.0