summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
-rw-r--r--cps-application/src/main/resources/application.yml14
-rw-r--r--cps-ncmp-events/src/main/resources/schemas/avc-event-schema-v1.json57
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/NcmpAsyncRequestResponseEventConsumer.java6
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/config/embeddedcache/SynchronizationCacheConfig.java2
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/notifications/avc/AvcEventConsumer.java53
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/notifications/avc/AvcEventMapper.java50
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/notifications/avc/AvcEventProducer.java52
-rw-r--r--cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/inventory/sync/ModuleSyncTasks.java4
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/notifications/avc/AvcEventProducerIntegrationSpec.groovy83
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/kafka/MessagingBaseSpec.groovy10
-rw-r--r--cps-ncmp-service/src/test/resources/sampleAvcInputEvent.json12
-rw-r--r--cps-path-parser/src/test/groovy/org/onap/cps/cpspath/parser/performance/CpsPathUtilPerfTest.groovy7
-rw-r--r--cps-rest/docs/openapi/components.yml11
-rw-r--r--cps-rest/docs/openapi/cpsData.yml35
-rw-r--r--cps-rest/docs/openapi/cpsDataV1Deprecated.yml35
-rw-r--r--cps-rest/docs/openapi/cpsDataV2.yml49
-rw-r--r--cps-rest/docs/openapi/cpsQueryV1Deprecated.yml (renamed from cps-rest/docs/openapi/cpsQuery.yml)4
-rw-r--r--cps-rest/docs/openapi/cpsQueryV2.yml49
-rw-r--r--cps-rest/docs/openapi/openapi.yml16
-rwxr-xr-xcps-rest/src/main/java/org/onap/cps/rest/controller/DataRestController.java18
-rw-r--r--cps-rest/src/main/java/org/onap/cps/rest/controller/QueryRestController.java28
-rwxr-xr-xcps-rest/src/test/groovy/org/onap/cps/rest/controller/DataRestControllerSpec.groovy25
-rw-r--r--cps-rest/src/test/groovy/org/onap/cps/rest/controller/QueryRestControllerSpec.groovy32
-rwxr-xr-xcps-ri/src/main/java/org/onap/cps/spi/entities/FragmentEntity.java5
-rw-r--r--cps-ri/src/main/java/org/onap/cps/spi/impl/CpsDataPersistenceServiceImpl.java21
-rw-r--r--cps-ri/src/main/java/org/onap/cps/spi/repository/FragmentNativeRepository.java28
-rw-r--r--cps-ri/src/main/java/org/onap/cps/spi/repository/FragmentNativeRepositoryImpl.java61
-rw-r--r--cps-ri/src/test/groovy/org/onap/cps/spi/impl/CpsDataPersistenceQueryDataNodeSpec.groovy14
-rwxr-xr-xcps-ri/src/test/groovy/org/onap/cps/spi/impl/CpsDataPersistenceServiceIntegrationSpec.groovy6
-rw-r--r--cps-ri/src/test/groovy/org/onap/cps/spi/impl/CpsDataPersistenceServiceSpec.groovy5
-rw-r--r--cps-ri/src/test/groovy/org/onap/cps/spi/performance/CpsDataPersistenceServiceDeletePerfTest.groovy20
-rw-r--r--cps-ri/src/test/groovy/org/onap/cps/spi/performance/CpsDataPersistenceServicePerfTest.groovy2
-rw-r--r--cps-service/src/main/java/org/onap/cps/spi/FetchDescendantsOption.java38
-rw-r--r--cps-service/src/test/groovy/org/onap/cps/api/impl/CpsQueryServiceImplSpec.groovy4
-rw-r--r--cps-service/src/test/groovy/org/onap/cps/spi/FetchDescendantsOptionSpec.groovy17
-rw-r--r--docs/api/swagger/cps/openapi.yaml194
36 files changed, 962 insertions, 105 deletions
diff --git a/cps-application/src/main/resources/application.yml b/cps-application/src/main/resources/application.yml
index b5b10b0f7..f7fe46e4e 100644
--- a/cps-application/src/main/resources/application.yml
+++ b/cps-application/src/main/resources/application.yml
@@ -1,7 +1,7 @@
# ============LICENSE_START=======================================================
# Copyright (C) 2021 Pantheon.tech
# Modifications Copyright (C) 2021-2022 Bell Canada
-# Modifications Copyright (C) 2021-2022 Nordix Foundation
+# Modifications Copyright (C) 2021-2023 Nordix Foundation
# ================================================================================
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
@@ -84,16 +84,15 @@ spring:
properties:
spring.deserializer.key.delegate.class: org.apache.kafka.common.serialization.StringDeserializer
spring.deserializer.value.delegate.class: org.springframework.kafka.support.serializer.JsonDeserializer
- spring.json.value.default.type: org.onap.cps.ncmp.event.model.DmiAsyncRequestResponseEvent
spring.json.use.type.headers: false
jackson:
- default-property-inclusion: NON_NULL
- serialization:
- FAIL_ON_EMPTY_BEANS: false
+ default-property-inclusion: NON_NULL
+ serialization:
+ FAIL_ON_EMPTY_BEANS: false
sql:
- init:
- mode: ALWAYS
+ init:
+ mode: ALWAYS
app:
ncmp:
async-m2m:
@@ -104,6 +103,7 @@ app:
events:
topic: ${LCM_EVENTS_TOPIC:ncmp-events}
+
notification:
enabled: true
data-updated:
diff --git a/cps-ncmp-events/src/main/resources/schemas/avc-event-schema-v1.json b/cps-ncmp-events/src/main/resources/schemas/avc-event-schema-v1.json
new file mode 100644
index 000000000..6db03f6eb
--- /dev/null
+++ b/cps-ncmp-events/src/main/resources/schemas/avc-event-schema-v1.json
@@ -0,0 +1,57 @@
+{
+ "$schema": "https://json-schema.org/draft/2019-09/schema",
+ "$id": "urn:cps:org.onap.cps.ncmp.events:avc-event-schema:v1",
+ "$ref": "#/definitions/AvcEvent",
+ "definitions": {
+ "AvcEvent": {
+ "description": "The payload for AVC event.",
+ "type": "object",
+ "properties": {
+ "eventId": {
+ "description": "The unique id identifying the event generated by DMI for this AVC event.",
+ "type": "string"
+ },
+ "eventCorrelationId": {
+ "description": "The request id passed by NCMP for this AVC event.",
+ "type": "string"
+ },
+ "eventTime": {
+ "description": "The time of the AVC event. The expected format is 'yyyy-MM-dd'T'HH:mm:ss.SSSZ'.",
+ "type": "string"
+ },
+ "eventTarget": {
+ "description": "The target of the AVC event.",
+ "type": "string"
+ },
+ "eventType": {
+ "description": "The type of the AVC event.",
+ "type": "string"
+ },
+ "eventSchema": {
+ "description": "The event schema for AVC events.",
+ "type": "string"
+ },
+ "eventSchemaVersion": {
+ "description": "The event schema version for AVC events.",
+ "type": "string"
+ },
+ "event": {
+ "$ref": "#/definitions/Event"
+ }
+ },
+ "required": [
+ "eventId",
+ "eventCorrelationId",
+ "eventTime",
+ "eventTarget",
+ "eventType",
+ "eventSchema",
+ "eventSchemaVersion"
+ ]
+ },
+ "Event": {
+ "description": "The AVC event content.",
+ "type": "object"
+ }
+ }
+} \ No newline at end of file
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/NcmpAsyncRequestResponseEventConsumer.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/NcmpAsyncRequestResponseEventConsumer.java
index a9e7164fd..bc6624dee 100644
--- a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/NcmpAsyncRequestResponseEventConsumer.java
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/async/NcmpAsyncRequestResponseEventConsumer.java
@@ -1,6 +1,6 @@
/*
* ============LICENSE_START=======================================================
- * Copyright (c) 2022 Nordix Foundation.
+ * Copyright (c) 2023 Nordix Foundation.
* ================================================================================
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -45,7 +45,9 @@ public class NcmpAsyncRequestResponseEventConsumer {
*
* @param dmiAsyncRequestResponseEvent the event to be consumed and produced.
*/
- @KafkaListener(topics = "${app.ncmp.async-m2m.topic}")
+ @KafkaListener(
+ topics = "${app.ncmp.async-m2m.topic}",
+ properties = {"spring.json.value.default.type=org.onap.cps.ncmp.event.model.DmiAsyncRequestResponseEvent"})
public void consumeAndForward(final DmiAsyncRequestResponseEvent dmiAsyncRequestResponseEvent) {
log.debug("Consuming event {} ...", dmiAsyncRequestResponseEvent);
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 5c3cb60c2..ac2bd4596 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
@@ -38,7 +38,7 @@ import org.springframework.context.annotation.Configuration;
@Configuration
public class SynchronizationCacheConfig {
- public static final int MODULE_SYNC_STARTED_TTL_SECS = 120;
+ public static final int MODULE_SYNC_STARTED_TTL_SECS = 600;
public static final int DATA_SYNC_SEMAPHORE_TTL_SECS = 1800;
private static final QueueConfig commonQueueConfig = createQueueConfig();
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/notifications/avc/AvcEventConsumer.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/notifications/avc/AvcEventConsumer.java
new file mode 100644
index 000000000..79a36bf50
--- /dev/null
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/notifications/avc/AvcEventConsumer.java
@@ -0,0 +1,53 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (c) 2023 Nordix Foundation.
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ * SPDX-License-Identifier: Apache-2.0
+ * ============LICENSE_END=========================================================
+ */
+
+package org.onap.cps.ncmp.api.impl.notifications.avc;
+
+import lombok.RequiredArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+import org.onap.cps.ncmp.event.model.AvcEvent;
+import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty;
+import org.springframework.kafka.annotation.KafkaListener;
+import org.springframework.stereotype.Component;
+
+/**
+ * Listener for AVC events.
+ */
+@Component
+@Slf4j
+@RequiredArgsConstructor
+@ConditionalOnProperty(name = "notification.enabled", havingValue = "true", matchIfMissing = true)
+public class AvcEventConsumer {
+
+ private final AvcEventProducer avcEventProducer;
+
+ /**
+ * Consume the specified event.
+ *
+ * @param avcEvent the event to be consumed and produced.
+ */
+ @KafkaListener(
+ topics = "dmi-cm-events",
+ properties = {"spring.json.value.default.type=org.onap.cps.ncmp.event.model.AvcEvent"})
+ public void consumeAndForward(final AvcEvent avcEvent) {
+ log.debug("Consuming AVC event {} ...", avcEvent);
+ avcEventProducer.sendMessage(avcEvent);
+ }
+}
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/notifications/avc/AvcEventMapper.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/notifications/avc/AvcEventMapper.java
new file mode 100644
index 000000000..531de4641
--- /dev/null
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/notifications/avc/AvcEventMapper.java
@@ -0,0 +1,50 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2023 Nordix Foundation
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ * SPDX-License-Identifier: Apache-2.0
+ * ============LICENSE_END=========================================================
+ */
+
+package org.onap.cps.ncmp.api.impl.notifications.avc;
+
+import java.util.UUID;
+import org.mapstruct.Mapper;
+import org.mapstruct.Mapping;
+import org.mapstruct.Named;
+import org.onap.cps.ncmp.event.model.AvcEvent;
+
+
+/**
+ * Mapper for converting incoming {@link AvcEvent} to outgoing {@link AvcEvent}.
+ */
+@Mapper(componentModel = "spring")
+public interface AvcEventMapper {
+
+ @Mapping(source = "eventTime", target = "eventTime")
+ @Mapping(source = "eventId", target = "eventId", qualifiedByName = "avcEventId")
+ @Mapping(source = "eventCorrelationId", target = "eventCorrelationId")
+ @Mapping(source = "eventSchema", target = "eventSchema")
+ @Mapping(source = "eventSchemaVersion", target = "eventSchemaVersion")
+ @Mapping(source = "eventTarget", target = "eventTarget")
+ @Mapping(source = "eventType", target = "eventType")
+ AvcEvent toOutgoingAvcEvent(AvcEvent incomingAvcEvent);
+
+ @Named("avcEventId")
+ static String getAvcEventId(String eventId) {
+ return UUID.randomUUID().toString();
+ }
+
+}
diff --git a/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/notifications/avc/AvcEventProducer.java b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/notifications/avc/AvcEventProducer.java
new file mode 100644
index 000000000..049f66100
--- /dev/null
+++ b/cps-ncmp-service/src/main/java/org/onap/cps/ncmp/api/impl/notifications/avc/AvcEventProducer.java
@@ -0,0 +1,52 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2023 Nordix Foundation
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ * SPDX-License-Identifier: Apache-2.0
+ * ============LICENSE_END=========================================================
+ */
+
+package org.onap.cps.ncmp.api.impl.notifications.avc;
+
+import lombok.RequiredArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+import org.onap.cps.ncmp.event.model.AvcEvent;
+import org.springframework.kafka.core.KafkaTemplate;
+import org.springframework.stereotype.Service;
+
+/**
+ * Producer for AVC events.
+ */
+@Slf4j
+@Service
+@RequiredArgsConstructor
+public class AvcEventProducer {
+
+ private final KafkaTemplate<String, AvcEvent> kafkaTemplate;
+
+ private final AvcEventMapper avcEventMapper;
+
+ /**
+ * Sends message to the configured topic with a message key.
+ *
+ * @param incomingAvcEvent message payload
+ */
+ public void sendMessage(final AvcEvent incomingAvcEvent) {
+ // generate new event id while keeping other data
+ final AvcEvent outgoingAvcEvent = avcEventMapper.toOutgoingAvcEvent(incomingAvcEvent);
+ log.debug("Forwarding AVC event {} to topic {} ", outgoingAvcEvent.getEventId(), "cm-events");
+ kafkaTemplate.send("cm-events", outgoingAvcEvent.getEventId(), outgoingAvcEvent);
+ }
+}
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 3fbebe077..d778afc3e 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
@@ -71,13 +71,13 @@ public class ModuleSyncTasks {
moduleSyncService.syncAndCreateSchemaSetAndAnchor(yangModelCmHandle);
cmHandelStatePerCmHandle.put(yangModelCmHandle, CmHandleState.READY);
} catch (final Exception e) {
- log.warn("Processing of {} module sync failed.", cmHandleId);
+ log.warn("Processing of {} module sync failed due to reason {}.", cmHandleId, e.getMessage());
syncUtils.updateLockReasonDetailsAndAttempts(compositeState,
LockReasonCategory.LOCKED_MODULE_SYNC_FAILED, e.getMessage());
setCmHandleStateLocked(yangModelCmHandle, compositeState.getLockReason());
cmHandelStatePerCmHandle.put(yangModelCmHandle, CmHandleState.LOCKED);
}
- log.debug("{} is now in {} state", cmHandleId, compositeState.getCmHandleState().name());
+ log.info("{} is now in {} state", cmHandleId, compositeState.getCmHandleState().name());
}
lcmEventsCmHandleStateHandler.updateCmHandleStateBatch(cmHandelStatePerCmHandle);
} finally {
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/notifications/avc/AvcEventProducerIntegrationSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/notifications/avc/AvcEventProducerIntegrationSpec.groovy
new file mode 100644
index 000000000..0089f777d
--- /dev/null
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/notifications/avc/AvcEventProducerIntegrationSpec.groovy
@@ -0,0 +1,83 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (c) 2023 Nordix Foundation.
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an 'AS IS' BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ * SPDX-License-Identifier: Apache-2.0
+ * ============LICENSE_END=========================================================
+ */
+
+package org.onap.cps.ncmp.api.impl.notifications.avc
+
+import com.fasterxml.jackson.databind.ObjectMapper
+import org.apache.kafka.clients.consumer.KafkaConsumer
+import org.mapstruct.factory.Mappers
+import org.onap.cps.ncmp.api.impl.async.NcmpAsyncRequestResponseEventMapper
+import org.onap.cps.ncmp.api.kafka.MessagingBaseSpec
+import org.onap.cps.ncmp.event.model.AvcEvent
+import org.onap.cps.ncmp.utils.TestUtils
+import org.onap.cps.utils.JsonObjectMapper
+import org.spockframework.spring.SpringBean
+import org.springframework.beans.factory.annotation.Autowired
+import org.springframework.boot.test.context.SpringBootTest
+import org.springframework.test.annotation.DirtiesContext
+import org.testcontainers.spock.Testcontainers
+
+import java.time.Duration
+
+@SpringBootTest(classes = [AvcEventProducer, AvcEventConsumer, ObjectMapper, JsonObjectMapper])
+@Testcontainers
+@DirtiesContext
+class AvcEventProducerIntegrationSpec extends MessagingBaseSpec {
+
+ @SpringBean
+ AvcEventMapper avcEventMapper = Mappers.getMapper(AvcEventMapper.class)
+
+ @SpringBean
+ AvcEventProducer avcEventProducer = new AvcEventProducer(kafkaTemplate, avcEventMapper)
+
+ @SpringBean
+ AvcEventConsumer acvEventConsumer = new AvcEventConsumer(avcEventProducer)
+
+ @Autowired
+ JsonObjectMapper jsonObjectMapper
+
+ def kafkaConsumer = new KafkaConsumer<>(consumerConfigProperties('ncmp-group'))
+
+ def 'Consume and forward valid message'() {
+ given: 'consumer has a subscription'
+ kafkaConsumer.subscribe(['cm-events'] as List<String>)
+ and: 'an event is sent'
+ def jsonData = TestUtils.getResourceFileContent('sampleAvcInputEvent.json')
+ def testEventSent = jsonObjectMapper.convertJsonString(jsonData, AvcEvent.class)
+ when: 'the event is consumed'
+ acvEventConsumer.consumeAndForward(testEventSent)
+ and: 'the topic is polled'
+ def records = kafkaConsumer.poll(Duration.ofMillis(1500))
+ then: 'poll returns one record'
+ assert records.size() == 1
+ and: 'record can be converted to AVC event'
+ def record = records.iterator().next()
+ def convertedAvcEvent = jsonObjectMapper.convertJsonString(record.value(), AvcEvent)
+ and: 'consumed forwarded NCMP event id differs from DMI event id'
+ assert testEventSent.eventId != convertedAvcEvent.getEventId()
+ and: 'correlation id matches'
+ assert testEventSent.eventCorrelationId == convertedAvcEvent.getEventCorrelationId()
+ and: 'timestamps match'
+ assert testEventSent.eventTime == convertedAvcEvent.getEventTime()
+ and: 'target matches'
+ assert testEventSent.eventTarget == convertedAvcEvent.getEventTarget()
+ }
+
+} \ No newline at end of file
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/kafka/MessagingBaseSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/kafka/MessagingBaseSpec.groovy
index f7c41ecdf..bb0ce8745 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/kafka/MessagingBaseSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/kafka/MessagingBaseSpec.groovy
@@ -1,6 +1,6 @@
/*
* ============LICENSE_START=======================================================
- * Copyright (c) 2022 Nordix Foundation.
+ * Copyright (c) 2023 Nordix Foundation.
* ================================================================================
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -33,14 +33,14 @@ import spock.lang.Specification
class MessagingBaseSpec extends Specification {
- static {
- Runtime.getRuntime().addShutdownHook(new Thread(kafkaTestContainer::stop))
- }
-
def setupSpec() {
kafkaTestContainer.start()
}
+ def cleanupSpec() {
+ kafkaTestContainer.stop()
+ }
+
static kafkaTestContainer = new KafkaContainer(DockerImageName.parse('registry.nordix.org/onaptest/confluentinc/cp-kafka:6.2.1').asCompatibleSubstituteFor('confluentinc/cp-kafka'))
def producerConfigProperties() {
diff --git a/cps-ncmp-service/src/test/resources/sampleAvcInputEvent.json b/cps-ncmp-service/src/test/resources/sampleAvcInputEvent.json
new file mode 100644
index 000000000..d7d252b9a
--- /dev/null
+++ b/cps-ncmp-service/src/test/resources/sampleAvcInputEvent.json
@@ -0,0 +1,12 @@
+{
+ "eventId": "4cb32729-85e3-44d1-aa6e-c923b9b059a5",
+ "eventCorrelationId": "68f15800-8ed4-4bae-9e53-27a9e03e1911",
+ "eventTime": "2022-12-12T14:29:23.876+0000",
+ "eventTarget": "NCMP",
+ "eventType": "org.onap.cps.ncmp.event.model.AvcEvent",
+ "eventSchema": "urn:cps:org.onap.cps.ncmp.event.model.AvcEvent",
+ "eventSchemaVersion": "v1",
+ "event": {
+ "payload": "Hello world!"
+ }
+} \ No newline at end of file
diff --git a/cps-path-parser/src/test/groovy/org/onap/cps/cpspath/parser/performance/CpsPathUtilPerfTest.groovy b/cps-path-parser/src/test/groovy/org/onap/cps/cpspath/parser/performance/CpsPathUtilPerfTest.groovy
index e5e304b47..1dfd194c8 100644
--- a/cps-path-parser/src/test/groovy/org/onap/cps/cpspath/parser/performance/CpsPathUtilPerfTest.groovy
+++ b/cps-path-parser/src/test/groovy/org/onap/cps/cpspath/parser/performance/CpsPathUtilPerfTest.groovy
@@ -1,6 +1,6 @@
/*
* ============LICENSE_START=======================================================
- * Copyright (C) 2022 Nordix Foundation
+ * Copyright (C) 2022-2023 Nordix Foundation
* ================================================================================
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -35,9 +35,8 @@ class CpsPathUtilPerfTest extends Specification {
CpsPathUtil.getNormalizedXpath('//child[@other-leaf=1]/leaf-name[text()="search"]/ancestor::parent')
}
stopWatch.stop()
- then: 'it takes less then 1,100 milliseconds'
- // In CI this actually takes about 0.3-0.5 sec which is approx. 50+ parser executions per millisecond!
- assert stopWatch.getTotalTimeMillis() < 1100
+ then: 'it takes less then 2100 milliseconds'
+ assert stopWatch.getTotalTimeMillis() < 2100
}
}
diff --git a/cps-rest/docs/openapi/components.yml b/cps-rest/docs/openapi/components.yml
index e700da6ea..60b4ca318 100644
--- a/cps-rest/docs/openapi/components.yml
+++ b/cps-rest/docs/openapi/components.yml
@@ -1,7 +1,7 @@
# ============LICENSE_START=======================================================
# Copyright (c) 2021-2022 Bell Canada.
# Modifications Copyright (C) 2021-2022 Nordix Foundation
-# Modifications Copyright (C) 2022 TechMahindra Ltd.
+# Modifications Copyright (C) 2022-2023 TechMahindra Ltd.
# Modifications Copyright (C) 2022 Deutsche Telekom AG
# ================================================================================
# Licensed under the Apache License, Version 2.0 (the "License");
@@ -240,6 +240,15 @@ components:
type: string
example: 'application/json'
required: true
+ descendantsInQuery:
+ name: descendants
+ in: query
+ description: descendents to query depth of children. allowed values are none, all, any number starting from -1
+ required: false
+ schema:
+ type: string
+ default: none
+ example: 3
responses:
NotFound:
diff --git a/cps-rest/docs/openapi/cpsData.yml b/cps-rest/docs/openapi/cpsData.yml
index 0dc388706..1d60e1f78 100644
--- a/cps-rest/docs/openapi/cpsData.yml
+++ b/cps-rest/docs/openapi/cpsData.yml
@@ -1,7 +1,7 @@
# ============LICENSE_START=======================================================
# Copyright (c) 2021-2022 Bell Canada.
# Modifications Copyright (C) 2021-2022 Nordix Foundation
-# Modifications Copyright (C) 2022 TechMahindra Ltd.
+# Modifications Copyright (C) 2022-2023 TechMahindra Ltd.
# Modifications Copyright (C) 2022 Deutsche Telekom AG
# ================================================================================
# Licensed under the Apache License, Version 2.0 (the "License");
@@ -19,39 +19,6 @@
# SPDX-License-Identifier: Apache-2.0
# ============LICENSE_END=========================================================
-nodeByDataspaceAndAnchor:
- get:
- description: Get a node with an option to retrieve all the children for a given anchor and dataspace
- tags:
- - cps-data
- summary: Get a node
- operationId: getNodeByDataspaceAndAnchor
- parameters:
- - $ref: 'components.yml#/components/parameters/apiVersionInPath'
- - $ref: 'components.yml#/components/parameters/dataspaceNameInPath'
- - $ref: 'components.yml#/components/parameters/anchorNameInPath'
- - $ref: 'components.yml#/components/parameters/xpathInQuery'
- - $ref: 'components.yml#/components/parameters/includeDescendantsOptionInQuery'
- responses:
- '200':
- description: OK
- content:
- application/json:
- schema:
- type: object
- examples:
- dataSample:
- $ref: 'components.yml#/components/examples/dataSample'
- '400':
- $ref: 'components.yml#/components/responses/BadRequest'
- '401':
- $ref: 'components.yml#/components/responses/Unauthorized'
- '403':
- $ref: 'components.yml#/components/responses/Forbidden'
- '500':
- $ref: 'components.yml#/components/responses/InternalServerError'
- x-codegen-request-body-name: xpath
-
listElementByDataspaceAndAnchor:
post:
description: Add list element(s) to a list for a given anchor and dataspace
diff --git a/cps-rest/docs/openapi/cpsDataV1Deprecated.yml b/cps-rest/docs/openapi/cpsDataV1Deprecated.yml
index 194ca3e07..67ddecd2a 100644
--- a/cps-rest/docs/openapi/cpsDataV1Deprecated.yml
+++ b/cps-rest/docs/openapi/cpsDataV1Deprecated.yml
@@ -1,5 +1,5 @@
# ============LICENSE_START=======================================================
-# Copyright (C) 2022 TechMahindra Ltd.
+# Copyright (C) 2022-2023 TechMahindra Ltd.
# ================================================================================
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
@@ -16,6 +16,39 @@
# SPDX-License-Identifier: Apache-2.0
# ============LICENSE_END=========================================================
+nodeByDataspaceAndAnchor:
+ get:
+ description: Get a node with an option to retrieve all the children for a given anchor and dataspace
+ deprecated: true
+ tags:
+ - cps-data
+ summary: Get a node
+ operationId: getNodeByDataspaceAndAnchor
+ parameters:
+ - $ref: 'components.yml#/components/parameters/dataspaceNameInPath'
+ - $ref: 'components.yml#/components/parameters/anchorNameInPath'
+ - $ref: 'components.yml#/components/parameters/xpathInQuery'
+ - $ref: 'components.yml#/components/parameters/includeDescendantsOptionInQuery'
+ responses:
+ '200':
+ description: OK
+ content:
+ application/json:
+ schema:
+ type: object
+ examples:
+ dataSample:
+ $ref: 'components.yml#/components/examples/dataSample'
+ '400':
+ $ref: 'components.yml#/components/responses/BadRequest'
+ '401':
+ $ref: 'components.yml#/components/responses/Unauthorized'
+ '403':
+ $ref: 'components.yml#/components/responses/Forbidden'
+ '500':
+ $ref: 'components.yml#/components/responses/InternalServerError'
+ x-codegen-request-body-name: xpath
+
listElementByDataspaceAndAnchor:
delete:
description: Delete one or all list element(s) for a given anchor and dataspace
diff --git a/cps-rest/docs/openapi/cpsDataV2.yml b/cps-rest/docs/openapi/cpsDataV2.yml
new file mode 100644
index 000000000..61663ab3a
--- /dev/null
+++ b/cps-rest/docs/openapi/cpsDataV2.yml
@@ -0,0 +1,49 @@
+# ============LICENSE_START=======================================================
+# Copyright (c) 2022-2023 TechMahindra Ltd.
+# ================================================================================
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+# SPDX-License-Identifier: Apache-2.0
+# ============LICENSE_END=========================================================
+
+nodeByDataspaceAndAnchor:
+ get:
+ description: Get a node with an option to retrieve all the children for a given anchor and dataspace
+ tags:
+ - cps-data
+ summary: Get a node
+ operationId: getNodeByDataspaceAndAnchorV2
+ parameters:
+ - $ref: 'components.yml#/components/parameters/dataspaceNameInPath'
+ - $ref: 'components.yml#/components/parameters/anchorNameInPath'
+ - $ref: 'components.yml#/components/parameters/xpathInQuery'
+ - $ref: 'components.yml#/components/parameters/descendantsInQuery'
+ responses:
+ '200':
+ description: OK
+ content:
+ application/json:
+ schema:
+ type: object
+ examples:
+ dataSample:
+ $ref: 'components.yml#/components/examples/dataSample'
+ '400':
+ $ref: 'components.yml#/components/responses/BadRequest'
+ '401':
+ $ref: 'components.yml#/components/responses/Unauthorized'
+ '403':
+ $ref: 'components.yml#/components/responses/Forbidden'
+ '500':
+ $ref: 'components.yml#/components/responses/InternalServerError'
+ x-codegen-request-body-name: xpath
diff --git a/cps-rest/docs/openapi/cpsQuery.yml b/cps-rest/docs/openapi/cpsQueryV1Deprecated.yml
index 45fc70c86..6ec117f31 100644
--- a/cps-rest/docs/openapi/cpsQuery.yml
+++ b/cps-rest/docs/openapi/cpsQueryV1Deprecated.yml
@@ -1,7 +1,7 @@
# ============LICENSE_START=======================================================
# Copyright (C) 2021 Nordix Foundation
# Modifications Copyright (c) 2022 Bell Canada.
-# Modifications Copyright (c) 2022 TechMahindra Ltd.
+# Modifications Copyright (c) 2023 TechMahindra Ltd.
# ================================================================================
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
@@ -24,9 +24,9 @@ nodesByDataspaceAndAnchorAndCpsPath:
tags:
- cps-query
summary: Query data nodes
+ deprecated: true
operationId: getNodesByDataspaceAndAnchorAndCpsPath
parameters:
- - $ref: 'components.yml#/components/parameters/apiVersionInPath'
- $ref: 'components.yml#/components/parameters/dataspaceNameInPath'
- $ref: 'components.yml#/components/parameters/anchorNameInPath'
- $ref: 'components.yml#/components/parameters/cpsPathInQuery'
diff --git a/cps-rest/docs/openapi/cpsQueryV2.yml b/cps-rest/docs/openapi/cpsQueryV2.yml
new file mode 100644
index 000000000..5bfd1bbd2
--- /dev/null
+++ b/cps-rest/docs/openapi/cpsQueryV2.yml
@@ -0,0 +1,49 @@
+# ============LICENSE_START=======================================================
+# Copyright (C) 2023 TechMahindra Ltd.
+# ================================================================================
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+# SPDX-License-Identifier: Apache-2.0
+# ============LICENSE_END=========================================================
+
+nodesByDataspaceAndAnchorAndCpsPath:
+ get:
+ description: Query data nodes for the given dataspace and anchor using CPS path
+ tags:
+ - cps-query
+ summary: Query data nodes
+ operationId: getNodesByDataspaceAndAnchorAndCpsPathV2
+ parameters:
+ - $ref: 'components.yml#/components/parameters/dataspaceNameInPath'
+ - $ref: 'components.yml#/components/parameters/anchorNameInPath'
+ - $ref: 'components.yml#/components/parameters/cpsPathInQuery'
+ - $ref: 'components.yml#/components/parameters/descendantsInQuery'
+ responses:
+ '200':
+ description: OK
+ content:
+ application/json:
+ schema:
+ type: object
+ examples:
+ dataSample:
+ $ref: 'components.yml#/components/examples/dataSample'
+ '400':
+ $ref: 'components.yml#/components/responses/BadRequest'
+ '401':
+ $ref: 'components.yml#/components/responses/Unauthorized'
+ '403':
+ $ref: 'components.yml#/components/responses/Forbidden'
+ '500':
+ $ref: 'components.yml#/components/responses/InternalServerError'
+ x-codegen-request-body-name: xpath
diff --git a/cps-rest/docs/openapi/openapi.yml b/cps-rest/docs/openapi/openapi.yml
index 0918b56c3..0ac825a6d 100644
--- a/cps-rest/docs/openapi/openapi.yml
+++ b/cps-rest/docs/openapi/openapi.yml
@@ -2,7 +2,7 @@
# Copyright (C) 2021 Nordix Foundation
# Modifications Copyright (C) 2021 Pantheon.tech
# Modifications Copyright (C) 2021 Bell Canada.
-# Modifications Copyright (C) 2022 TechMahindra Ltd.
+# Modifications Copyright (C) 2022-2023 TechMahindra Ltd.
# ================================================================================
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
@@ -89,8 +89,11 @@ paths:
/{apiVersion}/dataspaces/{dataspace-name}/schema-sets/{schema-set-name}:
$ref: 'cpsAdmin.yml#/schemaSetBySchemaSetName'
- /{apiVersion}/dataspaces/{dataspace-name}/anchors/{anchor-name}/node:
- $ref: 'cpsData.yml#/nodeByDataspaceAndAnchor'
+ /v1/dataspaces/{dataspace-name}/anchors/{anchor-name}/node:
+ $ref: 'cpsDataV1Deprecated.yml#/nodeByDataspaceAndAnchor'
+
+ /v2/dataspaces/{dataspace-name}/anchors/{anchor-name}/node:
+ $ref: 'cpsDataV2.yml#/nodeByDataspaceAndAnchor'
/{apiVersion}/dataspaces/{dataspace-name}/anchors/{anchor-name}/nodes:
$ref: 'cpsData.yml#/nodesByDataspaceAndAnchor'
@@ -101,8 +104,11 @@ paths:
/{apiVersion}/dataspaces/{dataspace-name}/anchors/{anchor-name}/list-nodes:
$ref: 'cpsData.yml#/listElementByDataspaceAndAnchor'
- /{apiVersion}/dataspaces/{dataspace-name}/anchors/{anchor-name}/nodes/query:
- $ref: 'cpsQuery.yml#/nodesByDataspaceAndAnchorAndCpsPath'
+ /v1/dataspaces/{dataspace-name}/anchors/{anchor-name}/nodes/query:
+ $ref: 'cpsQueryV1Deprecated.yml#/nodesByDataspaceAndAnchorAndCpsPath'
+
+ /v2/dataspaces/{dataspace-name}/anchors/{anchor-name}/nodes/query:
+ $ref: 'cpsQueryV2.yml#/nodesByDataspaceAndAnchorAndCpsPath'
security:
- basicAuth: []
diff --git a/cps-rest/src/main/java/org/onap/cps/rest/controller/DataRestController.java b/cps-rest/src/main/java/org/onap/cps/rest/controller/DataRestController.java
index 30bed1277..3a9c764bc 100755
--- a/cps-rest/src/main/java/org/onap/cps/rest/controller/DataRestController.java
+++ b/cps-rest/src/main/java/org/onap/cps/rest/controller/DataRestController.java
@@ -3,7 +3,7 @@
* Copyright (C) 2020-2022 Bell Canada.
* Modifications Copyright (C) 2021 Pantheon.tech
* Modifications Copyright (C) 2021-2022 Nordix Foundation
- * Modifications Copyright (C) 2022 TechMahindra Ltd.
+ * Modifications Copyright (C) 2023 TechMahindra Ltd.
* Modifications Copyright (C) 2022 Deutsche Telekom AG
* ================================================================================
* Licensed under the Apache License, Version 2.0 (the "License");
@@ -93,8 +93,8 @@ public class DataRestController implements CpsDataApi {
}
@Override
- public ResponseEntity<Object> getNodeByDataspaceAndAnchor(final String apiVersion,
- final String dataspaceName, final String anchorName, final String xpath, final Boolean includeDescendants) {
+ public ResponseEntity<Object> getNodeByDataspaceAndAnchor(final String dataspaceName,
+ final String anchorName, final String xpath, final Boolean includeDescendants) {
final FetchDescendantsOption fetchDescendantsOption = Boolean.TRUE.equals(includeDescendants)
? FetchDescendantsOption.INCLUDE_ALL_DESCENDANTS : FetchDescendantsOption.OMIT_DESCENDANTS;
final DataNode dataNode = cpsDataService.getDataNode(dataspaceName, anchorName, xpath,
@@ -104,6 +104,17 @@ public class DataRestController implements CpsDataApi {
}
@Override
+ public ResponseEntity<Object> getNodeByDataspaceAndAnchorV2(final String dataspaceName, final String anchorName,
+ final String xpath, final String fetchDescendantsOptionAsString) {
+ final FetchDescendantsOption fetchDescendantsOption =
+ FetchDescendantsOption.getFetchDescendantsOption(fetchDescendantsOptionAsString);
+ final DataNode dataNode = cpsDataService.getDataNode(dataspaceName, anchorName, xpath,
+ fetchDescendantsOption);
+ final String prefix = prefixResolver.getPrefix(dataspaceName, anchorName, xpath);
+ return new ResponseEntity<>(DataMapUtils.toDataMapWithIdentifier(dataNode, prefix), HttpStatus.OK);
+ }
+
+ @Override
public ResponseEntity<Object> updateNodeLeaves(final String apiVersion, final String dataspaceName,
final String anchorName, final Object jsonData, final String parentNodeXpath, final String observedTimestamp) {
cpsDataService.updateNodeLeaves(dataspaceName, anchorName, parentNodeXpath,
@@ -151,4 +162,5 @@ public class DataRestController implements CpsDataApi {
String.format("observed-timestamp must be in '%s' format", ISO_TIMESTAMP_FORMAT));
}
}
+
}
diff --git a/cps-rest/src/main/java/org/onap/cps/rest/controller/QueryRestController.java b/cps-rest/src/main/java/org/onap/cps/rest/controller/QueryRestController.java
index 3e162ae68..81938dcd9 100644
--- a/cps-rest/src/main/java/org/onap/cps/rest/controller/QueryRestController.java
+++ b/cps-rest/src/main/java/org/onap/cps/rest/controller/QueryRestController.java
@@ -2,7 +2,7 @@
* ============LICENSE_START=======================================================
* Copyright (C) 2021-2022 Nordix Foundation
* Modifications Copyright (C) 2022 Bell Canada.
- * Modifications Copyright (C) 2022 TechMahindra Ltd.
+ * Modifications Copyright (C) 2022-2023 TechMahindra Ltd.
* ================================================================================
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -49,22 +49,36 @@ public class QueryRestController implements CpsQueryApi {
private final PrefixResolver prefixResolver;
@Override
- public ResponseEntity<Object> getNodesByDataspaceAndAnchorAndCpsPath(final String apiVersion,
- final String dataspaceName, final String anchorName, final String cpsPath, final Boolean includeDescendants) {
+ public ResponseEntity<Object> getNodesByDataspaceAndAnchorAndCpsPath(final String dataspaceName,
+ final String anchorName, final String cpsPath, final Boolean includeDescendants) {
final FetchDescendantsOption fetchDescendantsOption = Boolean.TRUE.equals(includeDescendants)
? FetchDescendantsOption.INCLUDE_ALL_DESCENDANTS : FetchDescendantsOption.OMIT_DESCENDANTS;
+ return executeNodesByDataspaceQueryAndCreateResponse(dataspaceName, anchorName, cpsPath,
+ fetchDescendantsOption);
+ }
+
+ @Override
+ public ResponseEntity<Object> getNodesByDataspaceAndAnchorAndCpsPathV2(final String dataspaceName,
+ final String anchorName, final String cpsPath, final String fetchDescendantsOptionAsString) {
+ final FetchDescendantsOption fetchDescendantsOption =
+ FetchDescendantsOption.getFetchDescendantsOption(fetchDescendantsOptionAsString);
+ return executeNodesByDataspaceQueryAndCreateResponse(dataspaceName, anchorName, cpsPath,
+ fetchDescendantsOption);
+ }
+
+ private ResponseEntity<Object> executeNodesByDataspaceQueryAndCreateResponse(final String dataspaceName,
+ final String anchorName, final String cpsPath, final FetchDescendantsOption fetchDescendantsOption) {
final Collection<DataNode> dataNodes =
cpsQueryService.queryDataNodes(dataspaceName, anchorName, cpsPath, fetchDescendantsOption);
- final List<Map<String, Object>> dataMaps = new ArrayList<>(dataNodes.size());
+ final List<Map<String, Object>> dataNodesAsListOfMaps = new ArrayList<>(dataNodes.size());
String prefix = null;
for (final DataNode dataNode : dataNodes) {
if (prefix == null) {
prefix = prefixResolver.getPrefix(dataspaceName, anchorName, dataNode.getXpath());
}
final Map<String, Object> dataMap = DataMapUtils.toDataMapWithIdentifier(dataNode, prefix);
- dataMaps.add(dataMap);
+ dataNodesAsListOfMaps.add(dataMap);
}
-
- return new ResponseEntity<>(jsonObjectMapper.asJsonString(dataMaps), HttpStatus.OK);
+ return new ResponseEntity<>(jsonObjectMapper.asJsonString(dataNodesAsListOfMaps), HttpStatus.OK);
}
}
diff --git a/cps-rest/src/test/groovy/org/onap/cps/rest/controller/DataRestControllerSpec.groovy b/cps-rest/src/test/groovy/org/onap/cps/rest/controller/DataRestControllerSpec.groovy
index 94f62f8c2..16d106ba6 100755
--- a/cps-rest/src/test/groovy/org/onap/cps/rest/controller/DataRestControllerSpec.groovy
+++ b/cps-rest/src/test/groovy/org/onap/cps/rest/controller/DataRestControllerSpec.groovy
@@ -4,6 +4,7 @@
* Modifications Copyright (C) 2021 Pantheon.tech
* Modifications Copyright (C) 2021-2022 Bell Canada.
* Modifications Copyright (C) 2022 Deutsche Telekom AG
+ * Modifications Copyright (C) 2022-2023 TechMahindra Ltd.
* ================================================================================
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -68,6 +69,7 @@ class DataRestControllerSpec extends Specification {
def basePath
def dataNodeBaseEndpoint
+ def dataNodeBaseEndpointV2
def dataspaceName = 'my_dataspace'
def anchorName = 'my_anchor'
def noTimestamp = null
@@ -94,6 +96,7 @@ class DataRestControllerSpec extends Specification {
def setup() {
dataNodeBaseEndpoint = "$basePath/v1/dataspaces/$dataspaceName"
+ dataNodeBaseEndpointV2 = "$basePath/v2/dataspaces/$dataspaceName"
}
def 'Create a node: #scenario.'() {
@@ -237,6 +240,28 @@ class DataRestControllerSpec extends Specification {
'with descendants' | dataNodeWithChild | 'true' || INCLUDE_ALL_DESCENDANTS | true | 'parent'
}
+
+ def 'Get data node using v2 api'() {
+ given: 'the service returns data node'
+ def xpath = 'some xPath'
+ def endpoint = "$dataNodeBaseEndpointV2/anchors/$anchorName/node"
+ mockCpsDataService.getDataNode(dataspaceName, anchorName, xpath, {descendantsOption -> {
+ assert descendantsOption.depth == 2}}) >> dataNodeWithChild
+ when: 'get request is performed through REST API'
+ def response =
+ mvc.perform(
+ get(endpoint)
+ .param('xpath', xpath)
+ .param('descendants', '2'))
+ .andReturn().response
+ then: 'a success response is returned'
+ assert response.status == HttpStatus.OK.value()
+ and: 'the response contains the root node identifier'
+ assert response.contentAsString.contains('parent')
+ and: 'the response contains child is true'
+ assert response.contentAsString.contains('"child"') == true
+ }
+
def 'Update data node leaves: #scenario.'() {
given: 'endpoint to update a node '
def endpoint = "$dataNodeBaseEndpoint/anchors/$anchorName/nodes"
diff --git a/cps-rest/src/test/groovy/org/onap/cps/rest/controller/QueryRestControllerSpec.groovy b/cps-rest/src/test/groovy/org/onap/cps/rest/controller/QueryRestControllerSpec.groovy
index 27ca0cc09..b881c3832 100644
--- a/cps-rest/src/test/groovy/org/onap/cps/rest/controller/QueryRestControllerSpec.groovy
+++ b/cps-rest/src/test/groovy/org/onap/cps/rest/controller/QueryRestControllerSpec.groovy
@@ -3,6 +3,7 @@
* Copyright (C) 2021-2022 Nordix Foundation
* Modifications Copyright (C) 2021-2022 Bell Canada.
* Modifications Copyright (C) 2021 Pantheon.tech
+ * Modifications Copyright (C) 2022-2023 TechMahindra Ltd.
* ================================================================================
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -58,13 +59,19 @@ class QueryRestControllerSpec extends Specification {
@Value('${rest.api.cps-base-path}')
def basePath
+ def dataspaceName = 'my_dataspace'
+ def anchorName = 'my_anchor'
+ def cpsPath = 'some cps-path'
+ def dataNodeEndpointV2
+
+ def setup() {
+ dataNodeEndpointV2 = "$basePath/v2/dataspaces/$dataspaceName/anchors/$anchorName/nodes/query"
+ }
+
def 'Query data node by cps path for the given dataspace and anchor with #scenario.'() {
given: 'service method returns a list containing a data node'
def dataNode1 = new DataNodeBuilder().withXpath('/xpath')
.withLeaves([leaf: 'value', leafList: ['leaveListElement1', 'leaveListElement2']]).build()
- def dataspaceName = 'my_dataspace'
- def anchorName = 'my_anchor'
- def cpsPath = 'some cps-path'
mockCpsQueryService.queryDataNodes(dataspaceName, anchorName, cpsPath, expectedCpsDataServiceOption) >> [dataNode1, dataNode1]
and: 'the query endpoint'
def dataNodeEndpoint = "$basePath/v1/dataspaces/$dataspaceName/anchors/$anchorName/nodes/query"
@@ -84,4 +91,23 @@ class QueryRestControllerSpec extends Specification {
'no descendant explicitly' | 'false' || OMIT_DESCENDANTS
'descendants' | 'true' || INCLUDE_ALL_DESCENDANTS
}
+
+ def 'Query data node v2 api by cps path for the given dataspace and anchor with #scenario.'() {
+ given: 'service method returns a list containing a data node'
+ def dataNode1 = new DataNodeBuilder().withXpath('/xpath')
+ .withLeaves([leaf: 'value', leafList: ['leaveListElement1', 'leaveListElement2']]).build()
+ mockCpsQueryService.queryDataNodes(dataspaceName, anchorName, cpsPath, { descendantsOption -> {
+ assert descendantsOption.depth == 2}}) >> [dataNode1, dataNode1]
+ when: 'query data nodes API is invoked'
+ def response =
+ mvc.perform(
+ get(dataNodeEndpointV2)
+ .param('cps-path', cpsPath)
+ .param('descendants', '2'))
+ .andReturn().response
+ then: 'the response contains the the datanode in json format'
+ assert response.status == HttpStatus.OK.value()
+ assert response.getContentAsString().contains('{"xpath":{"leaf":"value","leafList":["leaveListElement1","leaveListElement2"]}}')
+ }
+
}
diff --git a/cps-ri/src/main/java/org/onap/cps/spi/entities/FragmentEntity.java b/cps-ri/src/main/java/org/onap/cps/spi/entities/FragmentEntity.java
index 2fdfa0528..2ffbb4ae0 100755
--- a/cps-ri/src/main/java/org/onap/cps/spi/entities/FragmentEntity.java
+++ b/cps-ri/src/main/java/org/onap/cps/spi/entities/FragmentEntity.java
@@ -1,6 +1,6 @@
/*
* ============LICENSE_START=======================================================
- * Copyright (C) 2020 Nordix Foundation.
+ * Copyright (C) 2020-2023 Nordix Foundation.
* Modifications Copyright (C) 2021 Pantheon.tech
* ================================================================================
* Licensed under the Apache License, Version 2.0 (the "License");
@@ -40,6 +40,7 @@ import javax.validation.constraints.NotNull;
import lombok.AllArgsConstructor;
import lombok.Builder;
import lombok.Data;
+import lombok.EqualsAndHashCode;
import lombok.Getter;
import lombok.NoArgsConstructor;
import lombok.Setter;
@@ -58,6 +59,7 @@ import org.hibernate.annotations.TypeDef;
@Entity
@Table(name = "fragment")
@TypeDef(name = "jsonb", typeClass = JsonBinaryType.class)
+@EqualsAndHashCode(onlyExplicitlyIncluded = true)
public class FragmentEntity implements Serializable {
private static final long serialVersionUID = 7737669789097119667L;
@@ -68,6 +70,7 @@ public class FragmentEntity implements Serializable {
@NotNull
@Column(columnDefinition = "text")
+ @EqualsAndHashCode.Include
private String xpath;
@Column(name = "parent_id")
diff --git a/cps-ri/src/main/java/org/onap/cps/spi/impl/CpsDataPersistenceServiceImpl.java b/cps-ri/src/main/java/org/onap/cps/spi/impl/CpsDataPersistenceServiceImpl.java
index 06ee8ecad..d2b7273fe 100644
--- a/cps-ri/src/main/java/org/onap/cps/spi/impl/CpsDataPersistenceServiceImpl.java
+++ b/cps-ri/src/main/java/org/onap/cps/spi/impl/CpsDataPersistenceServiceImpl.java
@@ -61,6 +61,7 @@ import org.onap.cps.spi.model.DataNode;
import org.onap.cps.spi.model.DataNodeBuilder;
import org.onap.cps.spi.repository.AnchorRepository;
import org.onap.cps.spi.repository.DataspaceRepository;
+import org.onap.cps.spi.repository.FragmentNativeRepository;
import org.onap.cps.spi.repository.FragmentQueryBuilder;
import org.onap.cps.spi.repository.FragmentRepository;
import org.onap.cps.spi.utils.SessionManager;
@@ -78,6 +79,7 @@ public class CpsDataPersistenceServiceImpl implements CpsDataPersistenceService
private final FragmentRepository fragmentRepository;
private final JsonObjectMapper jsonObjectMapper;
private final SessionManager sessionManager;
+ private final FragmentNativeRepository fragmentNativeRepositoryImpl;
private static final String REG_EX_FOR_OPTIONAL_LIST_INDEX = "(\\[@[\\s\\S]+?]){0,1})";
@@ -263,17 +265,26 @@ public class CpsDataPersistenceServiceImpl implements CpsDataPersistenceService
final DataspaceEntity dataspaceEntity = dataspaceRepository.getByName(dataspaceName);
final AnchorEntity anchorEntity = anchorRepository.getByDataspaceAndName(dataspaceEntity, anchorName);
- final Set<String> normalizedXpaths = new HashSet<>(xpaths.size());
- for (final String xpath : xpaths) {
+ final Collection<String> nonRootXpaths = new HashSet<>(xpaths);
+ final boolean haveRootXpath = nonRootXpaths.removeIf(CpsDataPersistenceServiceImpl::isRootXpath);
+
+ final Collection<String> normalizedXpaths = new HashSet<>(nonRootXpaths.size());
+ for (final String xpath : nonRootXpaths) {
try {
normalizedXpaths.add(CpsPathUtil.getNormalizedXpath(xpath));
} catch (final PathParsingException e) {
log.warn("Error parsing xpath \"{}\" in getDataNodes: {}", xpath, e.getMessage());
}
}
+ final Collection<FragmentEntity> fragmentEntities =
+ new HashSet<>(fragmentRepository.findByAnchorAndMultipleCpsPaths(anchorEntity.getId(), normalizedXpaths));
+
+ if (haveRootXpath) {
+ final List<FragmentExtract> fragmentExtracts = fragmentRepository.getTopLevelFragments(dataspaceEntity,
+ anchorEntity);
+ fragmentEntities.addAll(FragmentEntityArranger.toFragmentEntityTrees(anchorEntity, fragmentExtracts));
+ }
- final List<FragmentEntity> fragmentEntities =
- fragmentRepository.findByAnchorAndMultipleCpsPaths(anchorEntity.getId(), normalizedXpaths);
return toDataNodes(fragmentEntities, fetchDescendantsOption);
}
@@ -645,7 +656,7 @@ public class CpsDataPersistenceServiceImpl implements CpsDataPersistenceService
private boolean deleteDataNode(final FragmentEntity parentFragmentEntity, final String targetXpath) {
final String normalizedTargetXpath = CpsPathUtil.getNormalizedXpath(targetXpath);
if (parentFragmentEntity.getXpath().equals(normalizedTargetXpath)) {
- fragmentRepository.delete(parentFragmentEntity);
+ fragmentNativeRepositoryImpl.deleteFragmentEntity(parentFragmentEntity.getId());
return true;
}
if (parentFragmentEntity.getChildFragments()
diff --git a/cps-ri/src/main/java/org/onap/cps/spi/repository/FragmentNativeRepository.java b/cps-ri/src/main/java/org/onap/cps/spi/repository/FragmentNativeRepository.java
new file mode 100644
index 000000000..4cfd79dee
--- /dev/null
+++ b/cps-ri/src/main/java/org/onap/cps/spi/repository/FragmentNativeRepository.java
@@ -0,0 +1,28 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2023 Nordix Foundation
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ * SPDX-License-Identifier: Apache-2.0
+ * ============LICENSE_END=========================================================
+ */
+
+package org.onap.cps.spi.repository;
+
+/**
+ * This interface is used in delete fragment entity by id with child using native sql queries.
+ */
+public interface FragmentNativeRepository {
+ void deleteFragmentEntity(long fragmentEntityId);
+}
diff --git a/cps-ri/src/main/java/org/onap/cps/spi/repository/FragmentNativeRepositoryImpl.java b/cps-ri/src/main/java/org/onap/cps/spi/repository/FragmentNativeRepositoryImpl.java
new file mode 100644
index 000000000..57dca568f
--- /dev/null
+++ b/cps-ri/src/main/java/org/onap/cps/spi/repository/FragmentNativeRepositoryImpl.java
@@ -0,0 +1,61 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2023 Nordix Foundation
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ * SPDX-License-Identifier: Apache-2.0
+ * ============LICENSE_END=========================================================
+ */
+
+package org.onap.cps.spi.repository;
+
+import java.sql.PreparedStatement;
+import javax.persistence.EntityManager;
+import javax.persistence.PersistenceContext;
+import org.hibernate.Session;
+import org.springframework.stereotype.Repository;
+
+@Repository
+public class FragmentNativeRepositoryImpl implements FragmentNativeRepository {
+
+ private static final String DROP_FRAGMENT_CONSTRAINT
+ = "ALTER TABLE fragment DROP CONSTRAINT fragment_parent_id_fkey;";
+ private static final String ADD_FRAGMENT_CONSTRAINT_WITH_CASCADE
+ = "ALTER TABLE fragment ADD CONSTRAINT fragment_parent_id_fkey FOREIGN KEY (parent_id) "
+ + "REFERENCES fragment (id) ON DELETE CASCADE;";
+ private static final String DELETE_FRAGMENT = "DELETE FROM fragment WHERE id =?;";
+ private static final String ADD_ORIGINAL_FRAGMENT_CONSTRAINT
+ = "ALTER TABLE fragment ADD CONSTRAINT fragment_parent_id_fkey FOREIGN KEY (parent_id) "
+ + "REFERENCES fragment (id) ON DELETE NO ACTION;";
+
+ @PersistenceContext
+ private EntityManager entityManager;
+
+ @Override
+ public void deleteFragmentEntity(final long fragmentEntityId) {
+ final Session session = entityManager.unwrap(Session.class);
+ session.doWork(connection -> {
+ try (PreparedStatement preparedStatement = connection.prepareStatement(
+ DROP_FRAGMENT_CONSTRAINT
+ + ADD_FRAGMENT_CONSTRAINT_WITH_CASCADE
+ + DELETE_FRAGMENT
+ + DROP_FRAGMENT_CONSTRAINT
+ + ADD_ORIGINAL_FRAGMENT_CONSTRAINT)) {
+ preparedStatement.setLong(1, fragmentEntityId);
+ preparedStatement.executeUpdate();
+ }
+ });
+ }
+}
+
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 b6d2c5d65..ba8425fef 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
@@ -3,6 +3,7 @@
* Copyright (C) 2021-2022 Nordix Foundation
* Modifications Copyright (C) 2021 Pantheon.tech
* Modifications Copyright (C) 2021 Bell Canada.
+ * Modifications Copyright (C) 2023 TechMahindra Ltd.
* ================================================================================
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -22,6 +23,7 @@
package org.onap.cps.spi.impl
import org.onap.cps.spi.CpsDataPersistenceService
+import org.onap.cps.spi.FetchDescendantsOption
import org.onap.cps.spi.exceptions.CpsPathException
import org.springframework.beans.factory.annotation.Autowired
import org.springframework.test.context.jdbc.Sql
@@ -41,7 +43,7 @@ class CpsDataPersistenceQueryDataNodeSpec extends CpsPersistenceSpecBase {
@Sql([CLEAR_DATA, SET_DATA])
def 'Cps Path query for leaf value(s) with : #scenario.'() {
when: 'a query is executed to get a data node by the given cps path'
- def result = objectUnderTest.queryDataNodes(DATASPACE_NAME, ANCHOR_FOR_SHOP_EXAMPLE, cpsPath, includeDescendantsOption)
+ def result = objectUnderTest.queryDataNodes(DATASPACE_NAME, ANCHOR_FOR_SHOP_EXAMPLE, cpsPath, fetchDescendantsOption)
then: 'the correct number of parent nodes are returned'
result.size() == expectedNumberOfParentNodes
then: 'the correct data is returned'
@@ -49,10 +51,12 @@ class CpsDataPersistenceQueryDataNodeSpec extends CpsPersistenceSpecBase {
assert it.getChildDataNodes().size() == expectedNumberOfChildNodes
}
where: 'the following data is used'
- 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 || 3 | 0
+ scenario | cpsPath | fetchDescendantsOption || 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 || 3 | 0
+ 'Integer and level 1 descendants' | '/shops' | new FetchDescendantsOption(1) || 1 | 5
+ 'Integer and level 2 descendants' | '/shops/shop[@id=1]' | new FetchDescendantsOption(2) || 1 | 3
}
@Sql([CLEAR_DATA, SET_DATA])
diff --git a/cps-ri/src/test/groovy/org/onap/cps/spi/impl/CpsDataPersistenceServiceIntegrationSpec.groovy b/cps-ri/src/test/groovy/org/onap/cps/spi/impl/CpsDataPersistenceServiceIntegrationSpec.groovy
index 6252fff56..5f48469c0 100755
--- a/cps-ri/src/test/groovy/org/onap/cps/spi/impl/CpsDataPersistenceServiceIntegrationSpec.groovy
+++ b/cps-ri/src/test/groovy/org/onap/cps/spi/impl/CpsDataPersistenceServiceIntegrationSpec.groovy
@@ -20,6 +20,7 @@
* SPDX-License-Identifier: Apache-2.0
* ============LICENSE_END=========================================================
*/
+
package org.onap.cps.spi.impl
import com.fasterxml.jackson.databind.ObjectMapper
@@ -303,6 +304,7 @@ class CpsDataPersistenceServiceIntegrationSpec extends CpsPersistenceSpecBase {
assert results.size() == expectedResultSize
where: 'following parameters were used'
scenario | inputXpaths || expectedResultSize
+ '0 nodes' | [] || 0
'1 node' | ["/parent-200"] || 1
'2 unique nodes' | ["/parent-200", "/parent-201"] || 2
'3 unique nodes' | ["/parent-200", "/parent-201", "/parent-202"] || 3
@@ -314,6 +316,10 @@ class CpsDataPersistenceServiceIntegrationSpec extends CpsPersistenceSpecBase {
'existing and non-existing xpaths' | ["/parent-200", "/NO-XPATH", "/parent-201"] || 2
'invalid xpath' | ["INVALID XPATH"] || 0
'valid and invalid xpaths' | ["/parent-200", "INVALID XPATH", "/parent-201"] || 2
+ 'root xpath' | ["/"] || 7
+ 'empty (root) xpath' | [""] || 7
+ 'root and top-level xpaths' | ["/", "/parent-200", "/parent-201"] || 7
+ 'root and child xpaths' | ["/", "/parent-200/child-201"] || 8
}
@Sql([CLEAR_DATA, SET_DATA])
diff --git a/cps-ri/src/test/groovy/org/onap/cps/spi/impl/CpsDataPersistenceServiceSpec.groovy b/cps-ri/src/test/groovy/org/onap/cps/spi/impl/CpsDataPersistenceServiceSpec.groovy
index 87e59c60d..5dab87eec 100644
--- a/cps-ri/src/test/groovy/org/onap/cps/spi/impl/CpsDataPersistenceServiceSpec.groovy
+++ b/cps-ri/src/test/groovy/org/onap/cps/spi/impl/CpsDataPersistenceServiceSpec.groovy
@@ -32,6 +32,7 @@ import org.onap.cps.spi.model.DataNode
import org.onap.cps.spi.model.DataNodeBuilder
import org.onap.cps.spi.repository.AnchorRepository
import org.onap.cps.spi.repository.DataspaceRepository
+import org.onap.cps.spi.repository.FragmentNativeRepository
import org.onap.cps.spi.repository.FragmentRepository
import org.onap.cps.spi.utils.SessionManager
import org.onap.cps.utils.JsonObjectMapper
@@ -45,8 +46,10 @@ class CpsDataPersistenceServiceSpec extends Specification {
def mockFragmentRepository = Mock(FragmentRepository)
def jsonObjectMapper = new JsonObjectMapper(new ObjectMapper())
def mockSessionManager = Mock(SessionManager)
+ def stubFragmentNativeRepository = Stub(FragmentNativeRepository)
- def objectUnderTest = Spy(new CpsDataPersistenceServiceImpl(mockDataspaceRepository, mockAnchorRepository, mockFragmentRepository, jsonObjectMapper, mockSessionManager))
+ def objectUnderTest = Spy(new CpsDataPersistenceServiceImpl(mockDataspaceRepository, mockAnchorRepository,
+ mockFragmentRepository, jsonObjectMapper, mockSessionManager, stubFragmentNativeRepository))
def 'Storing data nodes individually when batch operation fails'(){
given: 'two data nodes and supporting repository mock behavior'
diff --git a/cps-ri/src/test/groovy/org/onap/cps/spi/performance/CpsDataPersistenceServiceDeletePerfTest.groovy b/cps-ri/src/test/groovy/org/onap/cps/spi/performance/CpsDataPersistenceServiceDeletePerfTest.groovy
index 5aae285d7..4dd4823c9 100644
--- a/cps-ri/src/test/groovy/org/onap/cps/spi/performance/CpsDataPersistenceServiceDeletePerfTest.groovy
+++ b/cps-ri/src/test/groovy/org/onap/cps/spi/performance/CpsDataPersistenceServiceDeletePerfTest.groovy
@@ -61,8 +61,8 @@ class CpsDataPersistenceServiceDeletePerfTest extends CpsPersistencePerfSpecBase
}
stopWatch.stop()
def deleteDurationInMillis = stopWatch.getTotalTimeMillis()
- then: 'delete duration is under 6000 milliseconds'
- assert deleteDurationInMillis < 6000
+ then: 'delete duration is under 300 milliseconds'
+ assert deleteDurationInMillis < 300
}
def 'Delete 50 grandchildren (that have no descendants)'() {
@@ -74,8 +74,8 @@ class CpsDataPersistenceServiceDeletePerfTest extends CpsPersistencePerfSpecBase
}
stopWatch.stop()
def deleteDurationInMillis = stopWatch.getTotalTimeMillis()
- then: 'delete duration is under 500 milliseconds'
- assert deleteDurationInMillis < 500
+ then: 'delete duration is under 350 milliseconds'
+ assert deleteDurationInMillis < 350
}
def 'Delete 1 large data node with many descendants'() {
@@ -84,8 +84,8 @@ class CpsDataPersistenceServiceDeletePerfTest extends CpsPersistencePerfSpecBase
objectUnderTest.deleteDataNode(PERF_DATASPACE, PERF_ANCHOR, PERF_TEST_PARENT)
stopWatch.stop()
def deleteDurationInMillis = stopWatch.getTotalTimeMillis()
- then: 'delete duration is under 2500 milliseconds'
- assert deleteDurationInMillis < 2500
+ then: 'delete duration is under 250 milliseconds'
+ assert deleteDurationInMillis < 250
}
@Sql([CLEAR_DATA, PERF_TEST_DATA])
@@ -108,8 +108,8 @@ class CpsDataPersistenceServiceDeletePerfTest extends CpsPersistencePerfSpecBase
}
stopWatch.stop()
def deleteDurationInMillis = stopWatch.getTotalTimeMillis()
- then: 'delete duration is under 4000 milliseconds'
- assert deleteDurationInMillis < 4000
+ then: 'delete duration is under 1000 milliseconds'
+ assert deleteDurationInMillis < 1000
}
def 'Delete 10 list elements with keys'() {
@@ -122,8 +122,8 @@ class CpsDataPersistenceServiceDeletePerfTest extends CpsPersistencePerfSpecBase
}
stopWatch.stop()
def deleteDurationInMillis = stopWatch.getTotalTimeMillis()
- then: 'delete duration is under 6000 milliseconds'
- assert deleteDurationInMillis < 6000
+ then: 'delete duration is under 1200 milliseconds'
+ assert deleteDurationInMillis < 1200
}
@Sql([CLEAR_DATA, PERF_TEST_DATA])
diff --git a/cps-ri/src/test/groovy/org/onap/cps/spi/performance/CpsDataPersistenceServicePerfTest.groovy b/cps-ri/src/test/groovy/org/onap/cps/spi/performance/CpsDataPersistenceServicePerfTest.groovy
index 2346239df..040749027 100644
--- a/cps-ri/src/test/groovy/org/onap/cps/spi/performance/CpsDataPersistenceServicePerfTest.groovy
+++ b/cps-ri/src/test/groovy/org/onap/cps/spi/performance/CpsDataPersistenceServicePerfTest.groovy
@@ -29,8 +29,6 @@ import org.onap.cps.spi.repository.FragmentRepository
import org.springframework.beans.factory.annotation.Autowired
import org.springframework.test.context.jdbc.Sql
-import java.util.concurrent.TimeUnit
-
import static org.onap.cps.spi.FetchDescendantsOption.INCLUDE_ALL_DESCENDANTS
import static org.onap.cps.spi.FetchDescendantsOption.OMIT_DESCENDANTS
diff --git a/cps-service/src/main/java/org/onap/cps/spi/FetchDescendantsOption.java b/cps-service/src/main/java/org/onap/cps/spi/FetchDescendantsOption.java
index b80054ac3..0c8cddcd7 100644
--- a/cps-service/src/main/java/org/onap/cps/spi/FetchDescendantsOption.java
+++ b/cps-service/src/main/java/org/onap/cps/spi/FetchDescendantsOption.java
@@ -2,6 +2,7 @@
* ============LICENSE_START=======================================================
* Copyright (C) 2021 Pantheon.tech
* Copyright (C) 2022 Nordix Foundation
+ * Modifications Copyright (C) 2023 TechMahindra Ltd.
* ================================================================================
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -20,7 +21,11 @@
package org.onap.cps.spi;
+import com.google.common.base.Strings;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
import lombok.RequiredArgsConstructor;
+import org.onap.cps.spi.exceptions.DataValidationException;
@RequiredArgsConstructor
public class FetchDescendantsOption {
@@ -29,6 +34,9 @@ public class FetchDescendantsOption {
public static final FetchDescendantsOption OMIT_DESCENDANTS = new FetchDescendantsOption(0);
public static final FetchDescendantsOption INCLUDE_ALL_DESCENDANTS = new FetchDescendantsOption(-1);
+ private static final Pattern FETCH_DESCENDANTS_OPTION_PATTERN =
+ Pattern.compile("^$|^all$|^none$|^[0-9]+$|^-1$");
+
private final int depth;
/**
@@ -58,6 +66,36 @@ public class FetchDescendantsOption {
return nextDescendantsOption;
}
+ /**
+ * get fetch descendants option for given descendant.
+ *
+ * @param fetchDescendantsOptionAsString fetch descendants option string
+ * @return fetch descendants option for given descendant
+ */
+ public static FetchDescendantsOption getFetchDescendantsOption(final String fetchDescendantsOptionAsString) {
+ validateFetchDescendantsOption(fetchDescendantsOptionAsString);
+ if (Strings.isNullOrEmpty(fetchDescendantsOptionAsString)
+ || "0".equals(fetchDescendantsOptionAsString) || "none".equals(fetchDescendantsOptionAsString)) {
+ return FetchDescendantsOption.OMIT_DESCENDANTS;
+ } else if ("-1".equals(fetchDescendantsOptionAsString) || "all".equals(fetchDescendantsOptionAsString)) {
+ return FetchDescendantsOption.INCLUDE_ALL_DESCENDANTS;
+ } else {
+ final Integer depth = Integer.valueOf(fetchDescendantsOptionAsString);
+ return new FetchDescendantsOption(depth);
+ }
+ }
+
+ private static void validateFetchDescendantsOption(final String fetchDescendantsOptionAsString) {
+ if (Strings.isNullOrEmpty(fetchDescendantsOptionAsString)) {
+ return;
+ }
+ final Matcher matcher = FETCH_DESCENDANTS_OPTION_PATTERN.matcher(fetchDescendantsOptionAsString);
+ if (!matcher.matches()) {
+ throw new DataValidationException("FetchDescendantsOption validation error.",
+ fetchDescendantsOptionAsString + " is not valid fetch descendants option");
+ }
+ }
+
private static void validateDepth(final int depth) {
if (depth < -1) {
throw new IllegalArgumentException("A depth of less than minus one is not allowed");
diff --git a/cps-service/src/test/groovy/org/onap/cps/api/impl/CpsQueryServiceImplSpec.groovy b/cps-service/src/test/groovy/org/onap/cps/api/impl/CpsQueryServiceImplSpec.groovy
index 8b232b420..60286b664 100644
--- a/cps-service/src/test/groovy/org/onap/cps/api/impl/CpsQueryServiceImplSpec.groovy
+++ b/cps-service/src/test/groovy/org/onap/cps/api/impl/CpsQueryServiceImplSpec.groovy
@@ -1,6 +1,7 @@
/*
* ============LICENSE_START=======================================================
* Copyright (C) 2021-2022 Nordix Foundation
+ * Modifications Copyright (C) 2023 TechMahindra Ltd.
* ================================================================================
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -43,7 +44,8 @@ class CpsQueryServiceImplSpec extends Specification {
and: 'the CpsValidator is called on the dataspaceName, schemaSetName and anchorName'
1 * mockCpsValidator.validateNameCharacters(dataspaceName, anchorName)
where: 'all fetch descendants options are supported'
- fetchDescendantsOption << [FetchDescendantsOption.OMIT_DESCENDANTS, FetchDescendantsOption.INCLUDE_ALL_DESCENDANTS]
+ fetchDescendantsOption << [FetchDescendantsOption.OMIT_DESCENDANTS, FetchDescendantsOption.INCLUDE_ALL_DESCENDANTS,
+ FetchDescendantsOption.FETCH_DIRECT_CHILDREN_ONLY, new FetchDescendantsOption(10)]
}
}
diff --git a/cps-service/src/test/groovy/org/onap/cps/spi/FetchDescendantsOptionSpec.groovy b/cps-service/src/test/groovy/org/onap/cps/spi/FetchDescendantsOptionSpec.groovy
index 627383561..c4d3dd8b7 100644
--- a/cps-service/src/test/groovy/org/onap/cps/spi/FetchDescendantsOptionSpec.groovy
+++ b/cps-service/src/test/groovy/org/onap/cps/spi/FetchDescendantsOptionSpec.groovy
@@ -1,6 +1,7 @@
/*
* ============LICENSE_START=======================================================
* Copyright (C) 2022 Nordix Foundation
+ * Modifications Copyright (C) 2023 TechMahindra Ltd.
* ================================================================================
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -20,7 +21,7 @@
package org.onap.cps.spi
-
+import org.onap.cps.spi.exceptions.DataValidationException
import spock.lang.Specification
class FetchDescendantsOptionSpec extends Specification {
@@ -72,4 +73,18 @@ class FetchDescendantsOptionSpec extends Specification {
then: 'exception thrown'
thrown IllegalArgumentException
}
+
+ def 'Create fetch descendant option with descendant using #scenario'() {
+ when: 'the next level of depth is not allowed'
+ def FetchDescendantsOption fetchDescendantsOption = FetchDescendantsOption.getFetchDescendantsOption(fetchDescendantsOptionAsString)
+ then: 'fetch descendant object created'
+ assert fetchDescendantsOption.depth == expectedDepth
+ where: 'following parameters are used'
+ scenario | fetchDescendantsOptionAsString || expectedDepth
+ 'all descendants using number' | '-1' || -1
+ 'all descendants using all' | 'all' || -1
+ 'No descendants by default' | '' || 0
+ 'No descendants using none' | 'none' || 0
+ 'til 10th descendants using number' | '10' || 10
+ }
}
diff --git a/docs/api/swagger/cps/openapi.yaml b/docs/api/swagger/cps/openapi.yaml
index ec7d29524..80766cc07 100644
--- a/docs/api/swagger/cps/openapi.yaml
+++ b/docs/api/swagger/cps/openapi.yaml
@@ -1104,16 +1104,16 @@ paths:
status: 500
message: Internal Server Error
details: Internal Server Error occurred
- /{apiVersion}/dataspaces/{dataspace-name}/anchors/{anchor-name}/node:
+ /v1/dataspaces/{dataspace-name}/anchors/{anchor-name}/node:
get:
tags:
- cps-data
summary: Get a node
description: Get a node with an option to retrieve all the children for a given
anchor and dataspace
+ deprecated: true
operationId: getNodeByDataspaceAndAnchor
parameters:
- - $ref: '#/components/parameters/apiVersionInPath'
- name: dataspace-name
in: path
description: dataspace-name
@@ -1199,6 +1199,101 @@ paths:
message: Internal Server Error
details: Internal Server Error occurred
x-codegen-request-body-name: xpath
+ /v2/dataspaces/{dataspace-name}/anchors/{anchor-name}/node:
+ get:
+ tags:
+ - cps-data
+ summary: Get a node
+ description: Get a node with an option to retrieve all the children for a given
+ anchor and dataspace
+ operationId: getNodeByDataspaceAndAnchorV2
+ parameters:
+ - name: dataspace-name
+ in: path
+ description: dataspace-name
+ required: true
+ schema:
+ type: string
+ example: my-dataspace
+ - name: anchor-name
+ in: path
+ description: anchor-name
+ required: true
+ schema:
+ type: string
+ example: my-anchor
+ - name: xpath
+ in: query
+ description: "For more details on xpath, please refer https://docs.onap.org/projects/onap-cps/en/latest/cps-path.html"
+ required: false
+ schema:
+ type: string
+ default: /
+ examples:
+ container xpath:
+ value: /shops/bookstore
+ list attributes xpath:
+ value: "/shops/bookstore/categories[@code=1]"
+ - name: descendants
+ in: query
+ description: descendants
+ required: false
+ schema:
+ type: string
+ example: 3
+ default: none
+ pattern: '^all$|^none$|^[0-9]+$|^-1$'
+ responses:
+ "200":
+ description: OK
+ content:
+ application/json:
+ schema:
+ type: object
+ examples:
+ dataSample:
+ $ref: '#/components/examples/dataSample'
+ "400":
+ description: Bad Request
+ content:
+ application/json:
+ schema:
+ $ref: '#/components/schemas/ErrorMessage'
+ example:
+ status: 400
+ message: Bad Request
+ details: The provided request is not valid
+ "401":
+ description: Unauthorized
+ content:
+ application/json:
+ schema:
+ $ref: '#/components/schemas/ErrorMessage'
+ example:
+ status: 401
+ message: Unauthorized request
+ details: This request is unauthorized
+ "403":
+ description: Forbidden
+ content:
+ application/json:
+ schema:
+ $ref: '#/components/schemas/ErrorMessage'
+ example:
+ status: 403
+ message: Request Forbidden
+ details: This request is forbidden
+ "500":
+ description: Internal Server Error
+ content:
+ application/json:
+ schema:
+ $ref: '#/components/schemas/ErrorMessage'
+ example:
+ status: 500
+ message: Internal Server Error
+ details: Internal Server Error occurred
+ x-codegen-request-body-name: xpath
/{apiVersion}/dataspaces/{dataspace-name}/anchors/{anchor-name}/nodes:
put:
tags:
@@ -1972,6 +2067,101 @@ paths:
message: Internal Server Error
details: Internal Server Error occurred
x-codegen-request-body-name: xpath
+ /v2/dataspaces/{dataspace-name}/anchors/{anchor-name}/nodes/query:
+ get:
+ tags:
+ - cps-query
+ summary: Query data nodes
+ description: Query data nodes for the given dataspace and anchor using CPS path
+ operationId: getNodesByDataspaceAndAnchorAndCpsPathV2
+ parameters:
+ - name: dataspace-name
+ in: path
+ description: dataspace-name
+ required: true
+ schema:
+ type: string
+ example: my-dataspace
+ - name: anchor-name
+ in: path
+ description: anchor-name
+ required: true
+ schema:
+ type: string
+ example: my-anchor
+ - name: cps-path
+ in: query
+ description: "For more details on cps path, please refer https://docs.onap.org/projects/onap-cps/en/latest/cps-path.html"
+ required: false
+ schema:
+ type: string
+ default: /
+ examples:
+ container cps path:
+ value: //bookstore
+ list attributes cps path:
+ value: "//categories[@code=1]"
+ - name: descendants
+ in: query
+ description: descendants
+ required: false
+ schema:
+ type: string
+ pattern: '^all$|^none$|^[0-9]+$|^-1$'
+ example: false
+ default: none
+ example: 3
+ responses:
+ "200":
+ description: OK
+ content:
+ application/json:
+ schema:
+ type: object
+ examples:
+ dataSample:
+ $ref: '#/components/examples/dataSample'
+ "400":
+ description: Bad Request
+ content:
+ application/json:
+ schema:
+ $ref: '#/components/schemas/ErrorMessage'
+ example:
+ status: 400
+ message: Bad Request
+ details: The provided request is not valid
+ "401":
+ description: Unauthorized
+ content:
+ application/json:
+ schema:
+ $ref: '#/components/schemas/ErrorMessage'
+ example:
+ status: 401
+ message: Unauthorized request
+ details: This request is unauthorized
+ "403":
+ description: Forbidden
+ content:
+ application/json:
+ schema:
+ $ref: '#/components/schemas/ErrorMessage'
+ example:
+ status: 403
+ message: Request Forbidden
+ details: This request is forbidden
+ "500":
+ description: Internal Server Error
+ content:
+ application/json:
+ schema:
+ $ref: '#/components/schemas/ErrorMessage'
+ example:
+ status: 500
+ message: Internal Server Error
+ details: Internal Server Error occurred
+ x-codegen-request-body-name: xpath
components:
parameters:
apiVersionInPath: