summaryrefslogtreecommitdiffstats
path: root/dmi-service/src/test
diff options
context:
space:
mode:
Diffstat (limited to 'dmi-service/src/test')
-rw-r--r--dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/api/kafka/MessagingBaseSpec.groovy79
-rw-r--r--dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/config/DmiConfigurationSpec.groovy66
-rw-r--r--dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/config/DmiPluginConfigSpec.groovy52
-rw-r--r--dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/config/kafka/KafkaConfigSpec.groovy62
-rw-r--r--dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/datajobs/rest/controller/DmiDatajobsRestControllerSpec.groovy69
-rw-r--r--dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/async/AsyncTaskExecutorIntegrationSpec.groovy110
-rw-r--r--dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/avc/AvcEventExecutorIntegrationSpec.groovy61
-rw-r--r--dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/cmsubscription/CmNotificationSubscriptionDmiInEventConsumerSpec.groovy149
-rw-r--r--dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/cmsubscription/CmNotificationSubscriptionDmiOutEventToCloudEventMapperSpec.groovy69
-rw-r--r--dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/mapper/CloudEventMapperSpec.groovy53
-rw-r--r--dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/rest/controller/ControllerSecuritySpec.groovy76
-rw-r--r--dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/rest/controller/DmiRestControllerSpec.groovy406
-rw-r--r--dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/DmiServiceImplSpec.groovy273
-rw-r--r--dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/YangResourceExtractorSpec.groovy98
-rw-r--r--dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/client/NcmpRestClientSpec.groovy57
-rw-r--r--dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/client/SdncRestconfClientSpec.groovy102
-rw-r--r--dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/operation/SdncOperationsSpec.groovy176
-rw-r--r--dmi-service/src/test/java/org/onap/cps/ncmp/dmi/TestUtils.java54
-rw-r--r--dmi-service/src/test/java/org/onap/cps/ncmp/dmi/rest/controller/TestController.java35
-rw-r--r--dmi-service/src/test/resources/ModuleSchema.json15
-rw-r--r--dmi-service/src/test/resources/application.yml79
-rw-r--r--dmi-service/src/test/resources/cmNotificationSubscriptionCreationEvent.json43
-rw-r--r--dmi-service/src/test/resources/createDataWithNormalChar.json8
-rw-r--r--dmi-service/src/test/resources/createDataWithSpecialChar.json8
-rw-r--r--dmi-service/src/test/resources/deleteData.json8
-rw-r--r--dmi-service/src/test/resources/moduleResources.json18
-rw-r--r--dmi-service/src/test/resources/patchData.json8
-rw-r--r--dmi-service/src/test/resources/readData.json9
-rw-r--r--dmi-service/src/test/resources/updateData.json8
29 files changed, 2251 insertions, 0 deletions
diff --git a/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/api/kafka/MessagingBaseSpec.groovy b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/api/kafka/MessagingBaseSpec.groovy
new file mode 100644
index 00000000..13dd043d
--- /dev/null
+++ b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/api/kafka/MessagingBaseSpec.groovy
@@ -0,0 +1,79 @@
+/*
+ * ============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.dmi.api.kafka
+
+import io.cloudevents.CloudEvent
+import io.cloudevents.kafka.CloudEventDeserializer
+import io.cloudevents.kafka.CloudEventSerializer
+import org.apache.kafka.clients.consumer.KafkaConsumer
+import org.apache.kafka.common.serialization.StringDeserializer
+import org.apache.kafka.common.serialization.StringSerializer
+import org.springframework.kafka.core.DefaultKafkaProducerFactory
+import org.springframework.kafka.core.KafkaTemplate
+import org.springframework.kafka.support.serializer.JsonSerializer
+import org.springframework.test.context.DynamicPropertyRegistry
+import org.springframework.test.context.DynamicPropertySource
+import org.testcontainers.containers.KafkaContainer
+import org.testcontainers.utility.DockerImageName
+import spock.lang.Specification
+
+class MessagingBaseSpec extends Specification {
+
+ 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(valueSerializer) {
+ return [('bootstrap.servers'): kafkaTestContainer.getBootstrapServers().split(',')[0],
+ ('retries') : 0,
+ ('batch-size') : 16384,
+ ('linger.ms') : 1,
+ ('buffer.memory') : 33554432,
+ ('key.serializer') : StringSerializer,
+ ('value.serializer') : valueSerializer]
+ }
+
+ def consumerConfigProperties(consumerGroupId, valueDeserializer) {
+ return [('bootstrap.servers') : kafkaTestContainer.getBootstrapServers().split(',')[0],
+ ('key.deserializer') : StringDeserializer,
+ ('value.deserializer'): valueDeserializer,
+ ('auto.offset.reset') : 'earliest',
+ ('group.id') : consumerGroupId
+ ]
+ }
+
+ def kafkaTemplate = new KafkaTemplate<>(new DefaultKafkaProducerFactory<Integer, String>(producerConfigProperties(JsonSerializer)))
+ def kafkaConsumer = new KafkaConsumer<>(consumerConfigProperties('ncmp-test-group', StringDeserializer))
+
+ def cloudEventKafkaTemplate = new KafkaTemplate(new DefaultKafkaProducerFactory<String, CloudEvent>(producerConfigProperties(CloudEventSerializer)))
+ def cloudEventKafkaConsumer = new KafkaConsumer<>(consumerConfigProperties('ncmp-test-group', CloudEventDeserializer))
+
+ @DynamicPropertySource
+ static void registerKafkaProperties(DynamicPropertyRegistry dynamicPropertyRegistry) {
+ dynamicPropertyRegistry.add('spring.kafka.bootstrap-servers', kafkaTestContainer::getBootstrapServers)
+ }
+}
diff --git a/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/config/DmiConfigurationSpec.groovy b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/config/DmiConfigurationSpec.groovy
new file mode 100644
index 00000000..9d80b71f
--- /dev/null
+++ b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/config/DmiConfigurationSpec.groovy
@@ -0,0 +1,66 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2021 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.dmi.config
+
+import org.springframework.beans.factory.annotation.Autowired
+import org.springframework.boot.test.context.SpringBootTest
+import org.springframework.boot.web.client.RestTemplateBuilder
+import org.springframework.test.context.ContextConfiguration
+import spock.lang.Specification
+
+@SpringBootTest
+@ContextConfiguration(classes = [DmiConfiguration.CpsProperties, DmiConfiguration.SdncProperties])
+class DmiConfigurationSpec extends Specification {
+
+ @Autowired
+ DmiConfiguration.CpsProperties cpsProperties
+
+ @Autowired
+ DmiConfiguration.SdncProperties sdncProperties
+
+ def 'CPS properties configuration.'() {
+ expect: 'CPS properties are set to values in test configuration yaml file'
+ cpsProperties.baseUrl == 'some url for cps'
+ cpsProperties.dmiRegistrationUrl == 'some registration url'
+ cpsProperties.authUsername == 'some cps core user'
+ cpsProperties.authPassword == 'some cps core password'
+ }
+
+ def 'SDNC properties configuration.'() {
+ expect: 'SDNC properties are set to values in test configuration yaml file'
+ sdncProperties.authUsername == 'test'
+ sdncProperties.authPassword == 'test'
+ sdncProperties.baseUrl == 'http://test'
+ sdncProperties.topologyId == 'test-topology'
+ }
+
+ def 'Rest template building.'() {
+ given: 'a DMI configuration'
+ DmiConfiguration objectUnderTest = new DmiConfiguration()
+ and: 'a rest template builder'
+ RestTemplateBuilder mockRestTemplateBuilder = Spy(RestTemplateBuilder)
+ when: 'rest template method is invoked'
+ objectUnderTest.restTemplate(mockRestTemplateBuilder)
+ then: 'DMI configuration uses the build method on the template builder'
+ 1 * mockRestTemplateBuilder.build()
+ }
+
+}
diff --git a/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/config/DmiPluginConfigSpec.groovy b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/config/DmiPluginConfigSpec.groovy
new file mode 100644
index 00000000..c09403d7
--- /dev/null
+++ b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/config/DmiPluginConfigSpec.groovy
@@ -0,0 +1,52 @@
+/*
+ * ============LICENSE_START=======================================================
+ * 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.
+ * 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.dmi.config
+
+import org.springdoc.core.models.GroupedOpenApi
+import org.springframework.beans.factory.annotation.Autowired
+import org.springframework.boot.test.context.SpringBootTest
+import org.springframework.test.context.ContextConfiguration
+import spock.lang.Specification
+
+@SpringBootTest
+@ContextConfiguration(classes = [DmiPluginConfig.DmiPluginProperties])
+class DmiPluginConfigSpec extends Specification {
+
+ @Autowired
+ DmiPluginConfig.DmiPluginProperties dmiPluginProperties
+
+ def 'DMI plugin properties configuration.'() {
+ expect: 'DMI plugin properties are set to values in test configuration yaml file'
+ dmiPluginProperties.dmiServiceUrl == 'some url for the dmi service'
+ }
+
+ def 'DMI plugin api creation.'() {
+ given: 'a DMI plugin configuration'
+ DmiPluginConfig objectUnderTest = new DmiPluginConfig()
+ when: 'the api method is invoked'
+ def result = objectUnderTest.api()
+ then: 'a spring web plugin docket is returned'
+ result instanceof GroupedOpenApi
+ and: 'it is named "dmi-plugin-api"'
+ result.group == 'dmi-plugin-api'
+ }
+
+}
diff --git a/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/config/kafka/KafkaConfigSpec.groovy b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/config/kafka/KafkaConfigSpec.groovy
new file mode 100644
index 00000000..a3bf52b3
--- /dev/null
+++ b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/config/kafka/KafkaConfigSpec.groovy
@@ -0,0 +1,62 @@
+/*
+ * ============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.dmi.config.kafka
+
+import io.cloudevents.CloudEvent
+import io.cloudevents.kafka.CloudEventDeserializer
+import io.cloudevents.kafka.CloudEventSerializer
+import org.spockframework.spring.EnableSharedInjection
+import org.springframework.beans.factory.annotation.Autowired
+import org.springframework.boot.autoconfigure.kafka.KafkaProperties
+import org.springframework.boot.context.properties.EnableConfigurationProperties
+import org.springframework.boot.test.context.SpringBootTest
+import org.springframework.kafka.core.KafkaTemplate
+import org.springframework.kafka.support.serializer.JsonDeserializer
+import org.springframework.kafka.support.serializer.JsonSerializer
+import spock.lang.Shared
+import spock.lang.Specification
+
+@SpringBootTest(classes = [KafkaProperties, KafkaConfig])
+@EnableSharedInjection
+@EnableConfigurationProperties
+class KafkaConfigSpec extends Specification {
+
+ @Shared
+ @Autowired
+ KafkaTemplate<String, String> legacyEventKafkaTemplate
+
+ @Shared
+ @Autowired
+ KafkaTemplate<String, CloudEvent> cloudEventKafkaTemplate
+
+ def 'Verify kafka template serializer and deserializer configuration for #eventType.'() {
+ expect: 'kafka template is instantiated'
+ assert kafkaTemplateInstance.properties['beanName'] == beanName
+ and: 'verify event key and value serializer'
+ assert kafkaTemplateInstance.properties['producerFactory'].configs['value.serializer'].asType(String.class).contains(valueSerializer.getCanonicalName())
+ and: 'verify event key and value deserializer'
+ assert kafkaTemplateInstance.properties['consumerFactory'].configs['spring.deserializer.value.delegate.class'].asType(String.class).contains(delegateDeserializer.getCanonicalName())
+ where: 'the following event type is used'
+ eventType | kafkaTemplateInstance || beanName | valueSerializer | delegateDeserializer
+ 'legacy event' | legacyEventKafkaTemplate || 'legacyEventKafkaTemplate' | JsonSerializer | JsonDeserializer
+ 'cloud event' | cloudEventKafkaTemplate || 'cloudEventKafkaTemplate' | CloudEventSerializer | CloudEventDeserializer
+ }
+}
diff --git a/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/datajobs/rest/controller/DmiDatajobsRestControllerSpec.groovy b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/datajobs/rest/controller/DmiDatajobsRestControllerSpec.groovy
new file mode 100644
index 00000000..c55f53c1
--- /dev/null
+++ b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/datajobs/rest/controller/DmiDatajobsRestControllerSpec.groovy
@@ -0,0 +1,69 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2024 Nordix Foundation
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * 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.dmi.datajobs.rest.controller
+
+import org.onap.cps.ncmp.dmi.config.WebSecurityConfig
+import org.springframework.beans.factory.annotation.Autowired
+import org.springframework.beans.factory.annotation.Value
+import org.springframework.boot.test.autoconfigure.web.servlet.WebMvcTest
+import org.springframework.context.annotation.Import
+import org.springframework.http.HttpStatus
+import org.springframework.security.test.context.support.WithMockUser
+import org.springframework.test.web.servlet.MockMvc
+import spock.lang.Specification
+
+import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.post
+
+@Import(WebSecurityConfig)
+@WebMvcTest(DmiDatajobsRestController.class)
+@WithMockUser
+class DmiDatajobsRestControllerSpec extends Specification{
+
+ @Autowired
+ private MockMvc mvc
+
+ @Value('${rest.api.dmi-base-path}/v1')
+ def basePathV1
+
+ def 'write request should return 501 HTTP Status' () {
+ given: 'URL to write a data job'
+ def getModuleUrl = "${basePathV1}/writeJob/001"
+ when: 'the request is posted'
+ def response = mvc.perform(
+ post(getModuleUrl)
+ .contentType('application/3gpp-json-patch+json')
+ ).andReturn().response
+ then: 'response value is Not Implemented'
+ response.status == HttpStatus.NOT_IMPLEMENTED.value()
+ }
+
+ def 'read request should return 501 HTTP Status' () {
+ given: 'URL to write a data job'
+ def getModuleUrl = "${basePathV1}/readJob/001"
+ when: 'the request is posted'
+ def response = mvc.perform(
+ post(getModuleUrl)
+ .contentType('application/3gpp-json-patch+json')
+ ).andReturn().response
+ then: 'response value is Not Implemented'
+ response.status == HttpStatus.NOT_IMPLEMENTED.value()
+ }
+}
diff --git a/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/async/AsyncTaskExecutorIntegrationSpec.groovy b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/async/AsyncTaskExecutorIntegrationSpec.groovy
new file mode 100644
index 00000000..12ca05cf
--- /dev/null
+++ b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/async/AsyncTaskExecutorIntegrationSpec.groovy
@@ -0,0 +1,110 @@
+/*
+ * ============LICENSE_START=======================================================
+ * 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.
+ * 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.dmi.notifications.async
+
+import com.fasterxml.jackson.databind.ObjectMapper
+import org.onap.cps.ncmp.dmi.api.kafka.MessagingBaseSpec
+import org.onap.cps.ncmp.dmi.exception.HttpClientRequestException
+import org.onap.cps.ncmp.dmi.model.DataAccessRequest
+import org.onap.cps.ncmp.event.model.DmiAsyncRequestResponseEvent
+import org.spockframework.spring.SpringBean
+import org.springframework.boot.test.context.SpringBootTest
+import org.springframework.http.HttpStatus
+import org.springframework.test.annotation.DirtiesContext
+import org.testcontainers.spock.Testcontainers
+
+import java.time.Duration
+import java.util.function.Supplier
+
+@SpringBootTest(classes = [AsyncTaskExecutor, DmiAsyncRequestResponseEventProducer])
+@Testcontainers
+@DirtiesContext
+class AsyncTaskExecutorIntegrationSpec extends MessagingBaseSpec {
+
+ @SpringBean
+ DmiAsyncRequestResponseEventProducer cpsAsyncRequestResponseEventProducer =
+ new DmiAsyncRequestResponseEventProducer(kafkaTemplate)
+
+ def spiedObjectMapper = Spy(ObjectMapper)
+ def mockSupplier = Mock(Supplier)
+
+ def objectUnderTest = new AsyncTaskExecutor(cpsAsyncRequestResponseEventProducer)
+
+ private static final String TEST_TOPIC = 'test-topic'
+
+ def setup() {
+ cpsAsyncRequestResponseEventProducer.dmiNcmpTopic = TEST_TOPIC
+ kafkaConsumer.subscribe([TEST_TOPIC] as List<String>)
+ }
+
+ def cleanup() {
+ kafkaConsumer.close()
+ }
+
+ def 'Publish and Subscribe message - success'() {
+ when: 'a successful event is published'
+ objectUnderTest.publishAsyncEvent(TEST_TOPIC, '12345','{}', 'OK', '200')
+ and: 'the topic is polled'
+ def records = kafkaConsumer.poll(Duration.ofMillis(1500))
+ then: 'the record received is the event sent'
+ def record = records.iterator().next()
+ DmiAsyncRequestResponseEvent event = spiedObjectMapper.readValue(record.value(), DmiAsyncRequestResponseEvent)
+ and: 'the status & code matches expected'
+ assert event.getEventContent().getResponseStatus() == 'OK'
+ assert event.getEventContent().getResponseCode() == '200'
+ }
+
+ def 'Publish and Subscribe message - failure'() {
+ when: 'a failure event is published'
+ def exception = new HttpClientRequestException('some cm handle', 'Node not found', HttpStatus.INTERNAL_SERVER_ERROR)
+ objectUnderTest.publishAsyncFailureEvent(TEST_TOPIC, '67890', exception)
+ and: 'the topic is polled'
+ def records = kafkaConsumer.poll(Duration.ofMillis(1500))
+ then: 'the record received is the event sent'
+ def record = records.iterator().next()
+ DmiAsyncRequestResponseEvent event = spiedObjectMapper.readValue(record.value(), DmiAsyncRequestResponseEvent)
+ and: 'the status & code matches expected'
+ assert event.getEventContent().getResponseStatus() == 'Internal Server Error'
+ assert event.getEventContent().getResponseCode() == '500'
+ }
+
+ def 'Execute an Async Task using asyncTaskExecutor and throw an error'() {
+ given: 'A task to be executed'
+ def requestId = '123456'
+ def operationEnum = DataAccessRequest.OperationEnum.CREATE
+ def timeOut = 100
+ when: 'AsyncTask has been executed'
+ objectUnderTest.executeAsyncTask(taskSupplierForFailingTask(), TEST_TOPIC, requestId, operationEnum, timeOut)
+ def records = kafkaConsumer.poll(Duration.ofMillis(1500))
+ then: 'the record received is the event sent'
+ def record = records.iterator().next()
+ DmiAsyncRequestResponseEvent event = spiedObjectMapper.readValue(record.value(), DmiAsyncRequestResponseEvent)
+ and: 'the status & code matches expected'
+ assert event.getEventContent().getResponseStatus() == 'Internal Server Error'
+ assert event.getEventContent().getResponseCode() == '500'
+
+ }
+
+ def taskSupplierForFailingTask() {
+ return () -> { throw new RuntimeException('original exception message') }
+ }
+
+} \ No newline at end of file
diff --git a/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/avc/AvcEventExecutorIntegrationSpec.groovy b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/avc/AvcEventExecutorIntegrationSpec.groovy
new file mode 100644
index 00000000..a7557bb9
--- /dev/null
+++ b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/avc/AvcEventExecutorIntegrationSpec.groovy
@@ -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.ncmp.dmi.notifications.avc
+
+import com.fasterxml.jackson.databind.ObjectMapper
+import io.cloudevents.core.CloudEventUtils
+import io.cloudevents.jackson.PojoCloudEventDataMapper
+import org.onap.cps.ncmp.dmi.api.kafka.MessagingBaseSpec
+import org.onap.cps.ncmp.events.avc1_0_0.AvcEvent
+import org.spockframework.spring.SpringBean
+import org.springframework.boot.test.context.SpringBootTest
+import org.springframework.test.annotation.DirtiesContext
+import org.testcontainers.spock.Testcontainers
+
+import java.time.Duration
+
+@SpringBootTest(classes = [DmiDataAvcEventProducer])
+@Testcontainers
+@DirtiesContext
+class AvcEventExecutorIntegrationSpec extends MessagingBaseSpec {
+
+ @SpringBean
+ DmiDataAvcEventProducer dmiDataAvcEventProducer = new DmiDataAvcEventProducer(cloudEventKafkaTemplate)
+
+ def dmiService = new DmiDataAvcEventSimulationController(dmiDataAvcEventProducer)
+
+ def objectMapper = new ObjectMapper()
+
+ def 'Publish Avc Event'() {
+ given: 'a simulated event'
+ dmiService.simulateEvents(1)
+ and: 'a consumer subscribed to dmi-cm-events topic'
+ cloudEventKafkaConsumer.subscribe(['dmi-cm-events'])
+ when: 'the next event record is consumed'
+ def record = cloudEventKafkaConsumer.poll(Duration.ofMillis(1500)).iterator().next()
+ then: 'record has correct topic'
+ assert record.topic == 'dmi-cm-events'
+ and: 'the record value can be mapped to an avcEvent'
+ def dmiDataAvcEvent = record.value()
+ def convertedAvcEvent = CloudEventUtils.mapData(dmiDataAvcEvent, PojoCloudEventDataMapper.from(objectMapper, AvcEvent.class)).getValue()
+ assert convertedAvcEvent != null
+ }
+} \ No newline at end of file
diff --git a/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/cmsubscription/CmNotificationSubscriptionDmiInEventConsumerSpec.groovy b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/cmsubscription/CmNotificationSubscriptionDmiInEventConsumerSpec.groovy
new file mode 100644
index 00000000..f1f476f6
--- /dev/null
+++ b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/cmsubscription/CmNotificationSubscriptionDmiInEventConsumerSpec.groovy
@@ -0,0 +1,149 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2024 Nordix Foundation
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * 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.dmi.notifications.cmsubscription
+
+import ch.qos.logback.classic.Level
+import ch.qos.logback.classic.Logger
+import ch.qos.logback.classic.spi.ILoggingEvent
+import ch.qos.logback.core.read.ListAppender
+import com.fasterxml.jackson.databind.ObjectMapper
+import io.cloudevents.CloudEvent
+import io.cloudevents.core.builder.CloudEventBuilder
+import org.apache.kafka.clients.consumer.ConsumerRecord
+import org.onap.cps.ncmp.dmi.TestUtils
+import org.onap.cps.ncmp.dmi.api.kafka.MessagingBaseSpec
+import org.onap.cps.ncmp.dmi.notifications.cmsubscription.model.CmNotificationSubscriptionStatus
+import org.onap.cps.ncmp.dmi.notifications.mapper.CloudEventMapper
+import org.onap.cps.ncmp.events.cmnotificationsubscription_merge1_0_0.dmi_to_ncmp.CmNotificationSubscriptionDmiOutEvent
+import org.onap.cps.ncmp.events.cmnotificationsubscription_merge1_0_0.dmi_to_ncmp.Data
+import org.onap.cps.ncmp.events.cmnotificationsubscription_merge1_0_0.ncmp_to_dmi.CmNotificationSubscriptionDmiInEvent
+import org.slf4j.LoggerFactory
+import org.spockframework.spring.SpringBean
+import org.springframework.boot.test.context.SpringBootTest
+import org.springframework.test.annotation.DirtiesContext
+import org.testcontainers.spock.Testcontainers
+
+import java.sql.Timestamp
+import java.time.Duration
+import java.time.OffsetDateTime
+import java.time.ZoneId
+
+
+@Testcontainers
+@DirtiesContext
+class CmNotificationSubscriptionDmiInEventConsumerSpec extends MessagingBaseSpec {
+ def objectMapper = new ObjectMapper()
+ def testTopic = 'dmi-ncmp-cm-avc-subscription'
+ def testDmiName = 'test-ncmp-dmi'
+
+ @SpringBean
+ CmNotificationSubscriptionDmiInEventConsumer objectUnderTest = new CmNotificationSubscriptionDmiInEventConsumer(cloudEventKafkaTemplate)
+
+ def logger = Spy(ListAppender<ILoggingEvent>)
+
+ void setup() {
+ ((Logger) LoggerFactory.getLogger(CloudEventMapper.class)).addAppender(logger)
+ logger.start()
+ }
+
+ void cleanup() {
+ ((Logger) LoggerFactory.getLogger(CloudEventMapper.class)).detachAndStopAllAppenders()
+ }
+
+ def 'Sends subscription cloud event response successfully.'() {
+ given: 'an subscription event response'
+ objectUnderTest.dmiName = testDmiName
+ objectUnderTest.cmNotificationSubscriptionDmiOutTopic = testTopic
+ def correlationId = 'test-subscriptionId#test-ncmp-dmi'
+ def cmSubscriptionDmiOutEventData = new Data(statusCode: subscriptionStatusCode, statusMessage: subscriptionStatusMessage)
+ def subscriptionEventResponse =
+ new CmNotificationSubscriptionDmiOutEvent().withData(cmSubscriptionDmiOutEventData)
+ and: 'consumer has a subscription'
+ kafkaConsumer.subscribe([testTopic] as List<String>)
+ when: 'an event is published'
+ def eventKey = UUID.randomUUID().toString()
+ objectUnderTest.createAndSendCmNotificationSubscriptionDmiOutEvent(eventKey, "subscriptionCreatedStatus", correlationId, subscriptionAcceptanceType)
+ and: 'topic is polled'
+ def records = kafkaConsumer.poll(Duration.ofMillis(1500))
+ then: 'poll returns one record and close kafkaConsumer'
+ assert records.size() == 1
+ def record = records.iterator().next()
+ kafkaConsumer.close()
+ and: 'the record value matches the expected event value'
+ def expectedValue = objectMapper.writeValueAsString(subscriptionEventResponse)
+ assert expectedValue == record.value
+ assert eventKey == record.key
+ where: 'given #scenario'
+ scenario | subscriptionAcceptanceType | subscriptionStatusCode | subscriptionStatusMessage
+ 'Subscription is Accepted' | CmNotificationSubscriptionStatus.ACCEPTED | '1' | 'ACCEPTED'
+ 'Subscription is Rejected' | CmNotificationSubscriptionStatus.REJECTED | '104' | 'REJECTED'
+ }
+
+ def 'Consume valid message.'() {
+ given: 'an event'
+ objectUnderTest.dmiName = testDmiName
+ def eventKey = UUID.randomUUID().toString()
+ def timestamp = new Timestamp(1679521929511)
+ def jsonData = TestUtils.getResourceFileContent('cmNotificationSubscriptionCreationEvent.json')
+ def subscriptionEvent = objectMapper.readValue(jsonData, CmNotificationSubscriptionDmiInEvent.class)
+ objectUnderTest.cmNotificationSubscriptionDmiOutTopic = testTopic
+ def cloudEvent = CloudEventBuilder.v1().withId(UUID.randomUUID().toString()).withSource(URI.create('test-ncmp-dmi'))
+ .withType(subscriptionType)
+ .withDataSchema(URI.create("urn:cps:" + CmNotificationSubscriptionDmiInEvent.class.getName() + ":1.0.0"))
+ .withExtension("correlationid", eventKey)
+ .withTime(OffsetDateTime.ofInstant(timestamp.toInstant(), ZoneId.of("UTC")))
+ .withData(objectMapper.writeValueAsBytes(subscriptionEvent)).build()
+ def testEventSent = new ConsumerRecord<String, CloudEvent>('topic-name', 0, 0, eventKey, cloudEvent)
+ when: 'the valid event is consumed'
+ objectUnderTest.consumeCmNotificationSubscriptionDmiInEvent(testEventSent)
+ then: 'no exception is thrown'
+ noExceptionThrown()
+ where: 'given #scenario'
+ scenario | subscriptionType
+ 'Subscription Create Event' | "subscriptionCreated"
+ 'Subscription Delete Event' | "subscriptionDeleted"
+ }
+
+ def 'Consume invalid message.'() {
+ given: 'an invalid event body'
+ objectUnderTest.dmiName = testDmiName
+ def eventKey = UUID.randomUUID().toString()
+ def timestamp = new Timestamp(1679521929511)
+ def invalidJsonBody = "/////"
+ objectUnderTest.cmNotificationSubscriptionDmiOutTopic = testTopic
+ def cloudEvent = CloudEventBuilder.v1().withId(UUID.randomUUID().toString()).withSource(URI.create('test-ncmp-dmi'))
+ .withType("subscriptionCreated")
+ .withDataSchema(URI.create("urn:cps:org.onap.ncmp.dmi.cm.subscription:1.0.0"))
+ .withTime(OffsetDateTime.ofInstant(timestamp.toInstant(), ZoneId.of("UTC")))
+ .withExtension("correlationid", eventKey).withData(objectMapper.writeValueAsBytes(invalidJsonBody)).build()
+ def testEventSent = new ConsumerRecord<String, CloudEvent>('topic-name', 0, 0, eventKey, cloudEvent)
+ when: 'the invalid event is consumed'
+ objectUnderTest.consumeCmNotificationSubscriptionDmiInEvent(testEventSent)
+ then: 'exception is thrown and event is logged'
+ def loggingEvent = getLoggingEvent()
+ assert loggingEvent.level == Level.ERROR
+ assert loggingEvent.formattedMessage.contains('Unable to map cloud event to target event class type')
+ }
+
+ def getLoggingEvent() {
+ return logger.list[0]
+ }
+} \ No newline at end of file
diff --git a/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/cmsubscription/CmNotificationSubscriptionDmiOutEventToCloudEventMapperSpec.groovy b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/cmsubscription/CmNotificationSubscriptionDmiOutEventToCloudEventMapperSpec.groovy
new file mode 100644
index 00000000..8ca629f1
--- /dev/null
+++ b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/cmsubscription/CmNotificationSubscriptionDmiOutEventToCloudEventMapperSpec.groovy
@@ -0,0 +1,69 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2024 Nordix Foundation
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * 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.dmi.notifications.cmsubscription
+
+import com.fasterxml.jackson.databind.ObjectMapper
+import io.cloudevents.core.builder.CloudEventBuilder
+import org.onap.cps.ncmp.dmi.exception.CloudEventConstructionException
+import org.onap.cps.ncmp.events.cmnotificationsubscription_merge1_0_0.dmi_to_ncmp.CmNotificationSubscriptionDmiOutEvent
+import org.onap.cps.ncmp.events.cmnotificationsubscription_merge1_0_0.dmi_to_ncmp.Data
+import org.spockframework.spring.SpringBean
+import org.springframework.beans.factory.annotation.Autowired
+import org.springframework.boot.test.context.SpringBootTest
+import spock.lang.Specification
+
+@SpringBootTest(classes = [ObjectMapper])
+class CmNotificationSubscriptionDmiOutEventToCloudEventMapperSpec extends Specification {
+
+ @Autowired
+ def objectMapper = new ObjectMapper()
+
+ @SpringBean
+ CmNotificationSubscriptionDmiOutEventToCloudEventMapper objectUnderTest = new CmNotificationSubscriptionDmiOutEventToCloudEventMapper()
+
+ def 'Convert a Cm Subscription DMI Out Event to CloudEvent successfully.'() {
+ given: 'a Cm Subscription DMI Out Event and an event key'
+ def dmiName = 'test-ncmp-dmi'
+ def correlationId = 'subscription1#test-ncmp-dmi'
+ def cmSubscriptionDmiOutEventData = new Data(statusCode: "1", statusMessage: "accepted")
+ def cmSubscriptionDmiOutEvent =
+ new CmNotificationSubscriptionDmiOutEvent().withData(cmSubscriptionDmiOutEventData)
+ when: 'a Cm Subscription DMI Out Event is converted'
+ def result = objectUnderTest.toCloudEvent(cmSubscriptionDmiOutEvent, "subscriptionCreatedStatus", dmiName, correlationId)
+ then: 'Cm Subscription DMI Out Event is converted as expected'
+ def expectedCloudEvent = CloudEventBuilder.v1().withId(UUID.randomUUID().toString()).withSource(URI.create('test-ncmp-dmi'))
+ .withType("subscriptionCreated")
+ .withDataSchema(URI.create("urn:cps:" + CmNotificationSubscriptionDmiOutEvent.class.getName() + ":1.0.0"))
+ .withExtension("correlationid", correlationId)
+ .withData(objectMapper.writeValueAsBytes(cmSubscriptionDmiOutEvent)).build()
+ assert expectedCloudEvent.data == result.data
+ }
+
+ def 'Map the Cloud Event to data of the subscription event with null parameters causes an exception'() {
+ given: 'an empty subscription response event and event key'
+ def correlationId = 'subscription1#test-ncmp-dmi'
+ def cmSubscriptionDmiOutEvent = new CmNotificationSubscriptionDmiOutEvent()
+ when: 'the cm subscription dmi out Event map to data of cloud event'
+ objectUnderTest.toCloudEvent(cmSubscriptionDmiOutEvent, "subscriptionCreatedStatus", null , correlationId)
+ then: 'a run time exception is thrown'
+ thrown(CloudEventConstructionException)
+ }
+}
diff --git a/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/mapper/CloudEventMapperSpec.groovy b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/mapper/CloudEventMapperSpec.groovy
new file mode 100644
index 00000000..0b404776
--- /dev/null
+++ b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/notifications/mapper/CloudEventMapperSpec.groovy
@@ -0,0 +1,53 @@
+/*
+ * ============LICENSE_START========================================================
+ * Copyright (c) 2024 Nordix Foundation.
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * 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.dmi.notifications.mapper
+
+import com.fasterxml.jackson.databind.ObjectMapper
+import io.cloudevents.core.builder.CloudEventBuilder
+import org.onap.cps.ncmp.events.cmnotificationsubscription_merge1_0_0.client_to_ncmp.CmNotificationSubscriptionNcmpInEvent
+import org.springframework.beans.factory.annotation.Autowired
+import org.springframework.boot.test.context.SpringBootTest
+import spock.lang.Specification
+
+@SpringBootTest(classes = [ObjectMapper])
+class CloudEventMapperSpec extends Specification {
+
+ @Autowired
+ ObjectMapper objectMapper
+
+ def 'Cloud event to Target event type when it is #scenario'() {
+ expect: 'Events mapped correctly'
+ assert mappedCloudEvent == (CloudEventMapper.toTargetEvent(testCloudEvent(), targetClass) != null)
+ where: 'below are the scenarios'
+ scenario | targetClass || mappedCloudEvent
+ 'valid concrete type' | CmNotificationSubscriptionNcmpInEvent.class || true
+ 'invalid concrete type' | ArrayList.class || false
+ }
+
+ def testCloudEvent() {
+ return CloudEventBuilder.v1().withData(objectMapper.writeValueAsBytes(new CmNotificationSubscriptionNcmpInEvent()))
+ .withId("cmhandle1")
+ .withSource(URI.create('test-source'))
+ .withDataSchema(URI.create('test'))
+ .withType('org.onap.cm.events.cm-subscription')
+ .build()
+ }
+}
diff --git a/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/rest/controller/ControllerSecuritySpec.groovy b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/rest/controller/ControllerSecuritySpec.groovy
new file mode 100644
index 00000000..3f5d4a80
--- /dev/null
+++ b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/rest/controller/ControllerSecuritySpec.groovy
@@ -0,0 +1,76 @@
+/*
+ * ============LICENSE_START=======================================================
+ * 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.
+ * 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.dmi.rest.controller
+
+import org.onap.cps.ncmp.dmi.config.WebSecurityConfig
+import org.springframework.context.annotation.Import
+
+import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.get
+
+import org.springframework.beans.factory.annotation.Autowired
+import org.springframework.boot.test.autoconfigure.web.servlet.WebMvcTest
+import org.springframework.http.HttpStatus
+import org.springframework.test.web.servlet.MockMvc
+import spock.lang.Specification
+
+@WebMvcTest(controllers = TestController.class)
+@Import(WebSecurityConfig)
+class ControllerSecuritySpec extends Specification {
+
+ @Autowired
+ MockMvc mvc
+
+ def testEndpoint = '/test'
+
+ def 'Get request with valid authentication'() {
+ when: 'request is sent with authentication'
+ def response = mvc.perform(
+ get(testEndpoint).header("Authorization", 'Basic Y3BzdXNlcjpjcHNyMGNrcyE=')
+ ).andReturn().response
+ then: 'HTTP OK status code is returned'
+ assert response.status == HttpStatus.OK.value()
+ }
+
+ def 'Get request without authentication'() {
+ when: 'request is sent without authentication'
+ def response = mvc.perform(get(testEndpoint)).andReturn().response
+ then: 'HTTP Unauthorized status code is returned'
+ assert response.status == HttpStatus.UNAUTHORIZED.value()
+ }
+
+ def 'Get request with invalid authentication'() {
+ when: 'request is sent with invalid authentication'
+ def response = mvc.perform(
+ get(testEndpoint).header("Authorization", 'Basic invalid auth')
+ ).andReturn().response
+ then: 'HTTP Unauthorized status code is returned'
+ assert response.status == HttpStatus.UNAUTHORIZED.value()
+ }
+
+ def 'Security Config #scenario permit URIs'() {
+ expect: 'can create a web security configuration'
+ new WebSecurityConfig(permitUris,'user','password')
+ where: 'the following string of permit URIs is provided'
+ scenario | permitUris
+ 'with' | 'a,b'
+ 'without' | ''
+ }
+}
diff --git a/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/rest/controller/DmiRestControllerSpec.groovy b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/rest/controller/DmiRestControllerSpec.groovy
new file mode 100644
index 00000000..a519de7b
--- /dev/null
+++ b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/rest/controller/DmiRestControllerSpec.groovy
@@ -0,0 +1,406 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2021-2023 Nordix Foundation
+ * Modifications Copyright (C) 2021-2022 Bell Canada
+ * ================================================================================
+ * 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.dmi.rest.controller
+
+
+import ch.qos.logback.classic.Logger
+import ch.qos.logback.classic.spi.ILoggingEvent
+import ch.qos.logback.core.read.ListAppender
+import org.onap.cps.ncmp.dmi.TestUtils
+import org.onap.cps.ncmp.dmi.config.WebSecurityConfig
+import org.onap.cps.ncmp.dmi.exception.DmiException
+import org.onap.cps.ncmp.dmi.exception.ModuleResourceNotFoundException
+import org.onap.cps.ncmp.dmi.exception.ModulesNotFoundException
+import org.onap.cps.ncmp.dmi.model.ModuleSet
+import org.onap.cps.ncmp.dmi.model.ModuleSetSchemasInner
+import org.onap.cps.ncmp.dmi.model.YangResource
+import org.onap.cps.ncmp.dmi.model.YangResources
+import org.onap.cps.ncmp.dmi.notifications.async.AsyncTaskExecutor
+import org.onap.cps.ncmp.dmi.notifications.async.DmiAsyncRequestResponseEventProducer
+import org.onap.cps.ncmp.dmi.service.DmiService
+import org.onap.cps.ncmp.dmi.service.model.ModuleReference
+import org.slf4j.LoggerFactory
+import org.spockframework.spring.SpringBean
+import org.springframework.beans.factory.annotation.Autowired
+import org.springframework.beans.factory.annotation.Value
+import org.springframework.boot.test.autoconfigure.web.servlet.WebMvcTest
+import org.springframework.context.annotation.Import
+import org.springframework.http.HttpStatus
+import org.springframework.http.MediaType
+import org.springframework.kafka.core.KafkaTemplate
+import org.springframework.security.test.context.support.WithMockUser
+import org.springframework.test.web.servlet.MockMvc
+import spock.lang.Specification
+
+import static org.onap.cps.ncmp.dmi.model.DataAccessRequest.OperationEnum.CREATE
+import static org.onap.cps.ncmp.dmi.model.DataAccessRequest.OperationEnum.DELETE
+import static org.onap.cps.ncmp.dmi.model.DataAccessRequest.OperationEnum.PATCH
+import static org.onap.cps.ncmp.dmi.model.DataAccessRequest.OperationEnum.READ
+import static org.onap.cps.ncmp.dmi.model.DataAccessRequest.OperationEnum.UPDATE
+import static org.springframework.http.HttpStatus.BAD_REQUEST
+import static org.springframework.http.HttpStatus.CREATED
+import static org.springframework.http.HttpStatus.INTERNAL_SERVER_ERROR
+import static org.springframework.http.HttpStatus.NO_CONTENT
+import static org.springframework.http.HttpStatus.OK
+import static org.springframework.test.web.servlet.request.MockMvcRequestBuilders.post
+
+@Import(WebSecurityConfig)
+@WebMvcTest(DmiRestController.class)
+@WithMockUser
+class DmiRestControllerSpec extends Specification {
+
+ @Autowired
+ private MockMvc mvc
+
+ @SpringBean
+ DmiService mockDmiService = Mock()
+
+ @SpringBean
+ DmiAsyncRequestResponseEventProducer cpsAsyncRequestResponseEventProducer = new DmiAsyncRequestResponseEventProducer(Mock(KafkaTemplate))
+
+ @SpringBean
+ AsyncTaskExecutor asyncTaskExecutor = new AsyncTaskExecutor(cpsAsyncRequestResponseEventProducer)
+
+ def logger = Spy(ListAppender<ILoggingEvent>)
+
+ void setup() {
+ ((Logger) LoggerFactory.getLogger(DmiRestController.class)).addAppender(logger)
+ logger.start()
+ }
+
+ void cleanup() {
+ ((Logger) LoggerFactory.getLogger(DmiRestController.class)).detachAndStopAllAppenders()
+ }
+
+ @Value('${rest.api.dmi-base-path}/v1')
+ def basePathV1
+
+ def 'Get all modules.'() {
+ given: 'URL for getting all modules and some request data'
+ def getModuleUrl = "$basePathV1/ch/node1/modules"
+ def someValidJson = '{}'
+ and: 'DMI service returns some module'
+ def moduleSetSchema = new ModuleSetSchemasInner(namespace:'some-namespace',
+ moduleName:'some-moduleName',
+ revision:'some-revision')
+ def moduleSetSchemasList = [moduleSetSchema] as List<ModuleSetSchemasInner>
+ def moduleSet = new ModuleSet()
+ moduleSet.schemas(moduleSetSchemasList)
+ mockDmiService.getModulesForCmHandle('node1') >> moduleSet
+ when: 'the request is posted'
+ def response = mvc.perform(post(getModuleUrl)
+ .contentType(MediaType.APPLICATION_JSON).content(someValidJson))
+ .andReturn().response
+ then: 'status is OK'
+ response.status == OK.value()
+ and: 'the response content matches the result from the DMI service'
+ response.getContentAsString() == '{"schemas":[{"moduleName":"some-moduleName","revision":"some-revision","namespace":"some-namespace"}]}'
+ }
+
+ def 'Get all modules with exception handling of #scenario.'() {
+ given: 'URL for getting all modules and some request data'
+ def getModuleUrl = "$basePathV1/ch/node1/modules"
+ def someValidJson = '{}'
+ and: 'a #exception is thrown during the process'
+ mockDmiService.getModulesForCmHandle('node1') >> { throw exception }
+ when: 'the request is posted'
+ def response = mvc.perform( post(getModuleUrl)
+ .contentType(MediaType.APPLICATION_JSON).content(someValidJson))
+ .andReturn().response
+ then: 'response status is #expectedResponse'
+ response.status == expectedResponse.value()
+ where: 'the scenario is #scenario'
+ scenario | exception || expectedResponse
+ 'dmi service exception' | new DmiException('','') || HttpStatus.INTERNAL_SERVER_ERROR
+ 'no modules found' | new ModulesNotFoundException('','') || HttpStatus.NOT_FOUND
+ 'any other runtime exception' | new RuntimeException() || HttpStatus.INTERNAL_SERVER_ERROR
+ 'runtime exception with cause' | new RuntimeException('', new RuntimeException()) || HttpStatus.INTERNAL_SERVER_ERROR
+ }
+
+ def 'Register given list.'() {
+ given: 'register cm handle url and cmHandles'
+ def registerCmhandlesPost = "${basePathV1}/inventory/cmHandles"
+ def cmHandleJson = '{"cmHandles":["node1", "node2"]}'
+ when: 'the request is posted'
+ def response = mvc.perform(
+ post(registerCmhandlesPost)
+ .contentType(MediaType.APPLICATION_JSON)
+ .content(cmHandleJson)
+ ).andReturn().response
+ then: 'register cm handles in dmi service is invoked with correct parameters'
+ 1 * mockDmiService.registerCmHandles(_ as List<String>)
+ and: 'response status is created'
+ response.status == CREATED.value()
+ }
+
+ def 'register cm handles called with empty content.'() {
+ given: 'register cm handle url and empty json'
+ def registerCmhandlesPost = "${basePathV1}/inventory/cmHandles"
+ def emptyJson = '{"cmHandles":[]}'
+ when: 'the request is posted'
+ def response = mvc.perform(
+ post(registerCmhandlesPost).contentType(MediaType.APPLICATION_JSON)
+ .content(emptyJson)
+ ).andReturn().response
+ then: 'response status is "bad request"'
+ response.status == BAD_REQUEST.value()
+ and: 'dmi service is not called'
+ 0 * mockDmiService.registerCmHandles(_)
+ }
+
+ def 'Retrieve module resources.'() {
+ given: 'URL to get module resources'
+ def getModulesEndpoint = "$basePathV1/ch/some-cm-handle/moduleResources"
+ and: 'request data to get some modules'
+ String jsonData = TestUtils.getResourceFileContent('moduleResources.json')
+ and: 'the DMI service returns the yang resources'
+ ModuleReference moduleReference1 = new ModuleReference(name: 'ietf-yang-library', revision: '2016-06-21')
+ ModuleReference moduleReference2 = new ModuleReference(name: 'nc-notifications', revision: '2008-07-14')
+ def moduleReferences = [moduleReference1, moduleReference2]
+ def yangResources = new YangResources()
+ def yangResource = new YangResource(yangSource: '"some-data"', moduleName: 'NAME', revision: 'REVISION')
+ yangResources.add(yangResource)
+ mockDmiService.getModuleResources('some-cm-handle', moduleReferences) >> yangResources
+ when: 'the request is posted'
+ def response = mvc.perform(post(getModulesEndpoint)
+ .contentType(MediaType.APPLICATION_JSON)
+ .content(jsonData)).andReturn().response
+ then: 'a OK status is returned'
+ response.status == OK.value()
+ and: 'the response content matches the result from the DMI service'
+ response.getContentAsString() == '[{"yangSource":"\\"some-data\\"","moduleName":"NAME","revision":"REVISION"}]'
+ }
+
+ def 'Retrieve module resources with exception handling.'() {
+ given: 'URL to get module resources'
+ def getModulesEndpoint = "$basePathV1/ch/some-cm-handle/moduleResources"
+ and: 'request data to get some modules'
+ String jsonData = TestUtils.getResourceFileContent('moduleResources.json')
+ and: 'the system throws a not-found exception (during the processing)'
+ mockDmiService.getModuleResources('some-cm-handle', _) >> { throw Mock(ModuleResourceNotFoundException.class) }
+ when: 'the request is posted'
+ def response = mvc.perform(post(getModulesEndpoint)
+ .contentType(MediaType.APPLICATION_JSON)
+ .content(jsonData)).andReturn().response
+ then: 'a not found status is returned'
+ response.status == HttpStatus.NOT_FOUND.value()
+ }
+
+ def 'Retrieve module resources and ensure module set tag is logged.'() {
+ given: 'URL to get module resources'
+ def getModulesEndpoint = "$basePathV1/ch/some-cm-handle/moduleResources"
+ and: 'request data to get some modules'
+ String jsonData = TestUtils.getResourceFileContent('moduleResources.json')
+ and: 'the DMI service returns the yang resources'
+ def moduleReferences = []
+ def yangResources = new YangResources()
+ def yangResource = new YangResource()
+ yangResources.add(yangResource)
+ mockDmiService.getModuleResources('some-cm-handle', moduleReferences) >> yangResources
+ when: 'the request is posted'
+ mvc.perform(post(getModulesEndpoint)
+ .contentType(MediaType.APPLICATION_JSON)
+ .content(jsonData))
+ then: 'the module set tag is logged'
+ def loggingMessage = getLoggingMessage(0)
+ assert loggingMessage.contains('module-set-tag1')
+ }
+
+ def 'Get resource data for pass-through operational.'() {
+ given: 'Get resource data url and some request data'
+ def getResourceDataForCmHandleUrl = "${basePathV1}/ch/some-cmHandle/data/ds/ncmp-datastore:passthrough-operational" +
+ "?resourceIdentifier=parent/child&options=(fields=myfields,depth=5)"
+ def someValidJson = '{}'
+ when: 'the request is posted'
+ def response = mvc.perform(
+ post(getResourceDataForCmHandleUrl).contentType(MediaType.APPLICATION_JSON).content(someValidJson)
+ ).andReturn().response
+ then: 'response status is ok'
+ response.status == OK.value()
+ and: 'dmi service method to get resource data is invoked once'
+ 1 * mockDmiService.getResourceData('some-cmHandle',
+ 'parent/child',
+ '(fields=myfields,depth=5)',
+ 'content=all')
+ }
+
+ def 'Get resource data for pass-through operational with write request (invalid).'() {
+ given: 'Get resource data url'
+ def getResourceDataForCmHandleUrl = "${basePathV1}/ch/some-cmHandle/data/ds/ncmp-datastore:passthrough-operational" +
+ "?resourceIdentifier=parent/child&options=(fields=myfields,depth=5)"
+ and: 'an invalid write request data for "create" operation'
+ def jsonData = '{"operation":"create"}'
+ when: 'the request is posted'
+ def response = mvc.perform(
+ post(getResourceDataForCmHandleUrl).contentType(MediaType.APPLICATION_JSON).content(jsonData)
+ ).andReturn().response
+ then: 'response status is bad request'
+ response.status == BAD_REQUEST.value()
+ and: 'dmi service is not invoked'
+ 0 * mockDmiService.getResourceData(*_)
+ }
+
+ def 'Get resource data for invalid datastore'() {
+ given: 'Get resource data url'
+ def getResourceDataForCmHandleUrl = "${basePathV1}/ch/some-cmHandle/data/ds/dummy-datastore" +
+ "?resourceIdentifier=parent/child&options=(fields=myfields,depth=5)"
+ and: 'an invalid write request data for "create" operation'
+ def jsonData = '{"operation":"create"}'
+ when: 'the request is posted'
+ def response = mvc.perform(
+ post(getResourceDataForCmHandleUrl).contentType(MediaType.APPLICATION_JSON).content(jsonData)
+ ).andReturn().response
+ then: 'response status is internal server error'
+ response.status == INTERNAL_SERVER_ERROR.value()
+ and: 'response contains expected error message'
+ response.contentAsString.contains('dummy-datastore is an invalid datastore name')
+ }
+
+ def 'data with #scenario operation using passthrough running.'() {
+ given: 'write data for passthrough running url'
+ def writeDataForPassthroughRunning = "${basePathV1}/ch/some-cmHandle/data/ds/ncmp-datastore:passthrough-running" +
+ "?resourceIdentifier=some-resourceIdentifier"
+ and: 'request data for #scenario'
+ def jsonData = TestUtils.getResourceFileContent(requestBodyFile)
+ and: 'dmi service is called'
+ mockDmiService.writeData(operationEnum, 'some-cmHandle',
+ 'some-resourceIdentifier', dataType,
+ 'normal request body' ) >> '{some-json}'
+ when: 'the request is posted'
+ def response = mvc.perform(
+ post(writeDataForPassthroughRunning).contentType(MediaType.APPLICATION_JSON)
+ .content(jsonData)
+ ).andReturn().response
+ then: 'response status is #expectedResponseStatus'
+ response.status == expectedResponseStatus
+ and: 'the response content matches the result from the DMI service'
+ response.getContentAsString() == expectedJsonResponse
+ where: 'given request body and data'
+ scenario | requestBodyFile | operationEnum | dataType || expectedResponseStatus | expectedJsonResponse
+ 'Create' | 'createDataWithNormalChar.json' | CREATE | 'application/json' || CREATED.value() | '{some-json}'
+ 'Update' | 'updateData.json' | UPDATE | 'application/json' || OK.value() | '{some-json}'
+ 'Delete' | 'deleteData.json' | DELETE | 'application/json' || NO_CONTENT.value() | '{some-json}'
+ 'Read' | 'readData.json' | READ | 'application/json' || OK.value() | ''
+ 'Patch' | 'patchData.json' | PATCH | 'application/yang.patch+json' || OK.value() | '{some-json}'
+ }
+
+ def 'Create data using passthrough for special characters.'(){
+ given: 'create data for cmHandle url'
+ def writeDataForCmHandlePassthroughRunning = "${basePathV1}/ch/some-cmHandle/data/ds/ncmp-datastore:passthrough-running" +
+ "?resourceIdentifier=some-resourceIdentifier"
+ and: 'request data with special characters'
+ def jsonData = TestUtils.getResourceFileContent('createDataWithSpecialChar.json')
+ and: 'dmi service returns data'
+ mockDmiService.writeData(CREATE, 'some-cmHandle', 'some-resourceIdentifier', 'application/json',
+ 'data with quote \" and new line \n') >> '{some-json}'
+ when: 'the request is posted'
+ def response = mvc.perform(
+ post(writeDataForCmHandlePassthroughRunning).contentType(MediaType.APPLICATION_JSON).content(jsonData)
+ ).andReturn().response
+ then: 'response status is CREATED'
+ response.status == CREATED.value()
+ and: 'the response content matches the result from the DMI service'
+ response.getContentAsString() == '{some-json}'
+ }
+
+ def 'PassThrough Returns OK when topic is used for async'(){
+ given: 'Passthrough read URL and request data with a topic (parameter)'
+ def readPassThroughUrl ="${basePathV1}/ch/some-cmHandle/data/ds/ncmp-datastore:" +
+ resourceIdentifier +
+ '?resourceIdentifier=some-resourceIdentifier&topic=test-topic'
+ def jsonData = TestUtils.getResourceFileContent('readData.json')
+ when: 'the request is posted'
+ def response = mvc.perform(
+ post(readPassThroughUrl).contentType(MediaType.APPLICATION_JSON).content(jsonData)
+ ).andReturn().response
+ then: 'response status is OK'
+ assert response.status == HttpStatus.NO_CONTENT.value()
+ where: 'the following values are used'
+ resourceIdentifier << ['passthrough-operational', 'passthrough-running']
+ }
+
+ def 'PassThrough logs module set tag'(){
+ given: 'Passthrough read URL and request data with a module set tag (parameter)'
+ def readPassThroughUrl ="${basePathV1}/ch/some-cmHandle/data/ds/ncmp-datastore:" +
+ 'passthrough-running?resourceIdentifier=some-resourceIdentifier'
+ def jsonData = TestUtils.getResourceFileContent('readData.json')
+ when: 'the request is posted'
+ mvc.perform(
+ post(readPassThroughUrl).contentType(MediaType.APPLICATION_JSON).content(jsonData))
+ then: 'response status is OK'
+ def loggingMessage = getLoggingMessage(0)
+ assert loggingMessage.contains('module-set-tag-example')
+ }
+
+ def 'Get resource data for pass-through running with #scenario value in resource identifier param.'() {
+ given: 'Get resource data url'
+ def getResourceDataForCmHandleUrl = "${basePathV1}/ch/some-cmHandle/data/ds/ncmp-datastore:passthrough-running" +
+ "?resourceIdentifier="+resourceIdentifier+"&options=(fields=myfields,depth=5)"
+ and: 'some valid json data'
+ def json = '{"cmHandleProperties" : { "prop1" : "value1", "prop2" : "value2"}}'
+ when: 'the request is posted'
+ def response = mvc.perform(
+ post(getResourceDataForCmHandleUrl).contentType(MediaType.APPLICATION_JSON).content(json)
+ ).andReturn().response
+ then: 'response status is ok'
+ response.status == OK.value()
+ and: 'dmi service method to get resource data is invoked once with correct parameters'
+ 1 * mockDmiService.getResourceData('some-cmHandle',
+ resourceIdentifier,
+ '(fields=myfields,depth=5)',
+ 'content=config')
+ where: 'tokens are used in the resource identifier parameter'
+ scenario | resourceIdentifier
+ '/' | 'id/with/slashes'
+ '?' | 'idWith?'
+ ',' | 'idWith,'
+ '=' | 'idWith='
+ '[]' | 'idWith[]'
+ '? needs to be encoded as %3F' | 'idWith%3F'
+
+ }
+
+ def 'Execute a data operation for a list of operations.'() {
+ given: 'an endpoint for a data operation request with list of cmhandles in request body'
+ def resourceDataUrl = "$basePathV1/data?topic=client-topic-name&requestId=some-requestId"
+ and: 'list of operation details are received into request body'
+ def dataOperationRequestBody = '[{"operation": "read", "operationId": "14", "datastore": "ncmp-datastore:passthrough-operational", "options": "some options", "resourceIdentifier": "some resourceIdentifier",' +
+ '"cmHandles": [ {"id": "cmHandle123", "moduleSetTag": "module-set-tag1", "cmHandleProperties": { "myProp`": "some value", "otherProp": "other value"}}]}]'
+ when: 'the dmi resource data for dataOperation api is called.'
+ def response = mvc.perform(
+ post(resourceDataUrl).contentType(MediaType.APPLICATION_JSON).content(dataOperationRequestBody)
+ ).andReturn().response
+ then: 'the resource data operation endpoint returns the not implemented response'
+ assert response.status == 501
+ and: 'the job details are correctly received (logged)'
+ assert getLoggingMessage(1).contains('some-requestId')
+ assert getLoggingMessage(2).contains('client-topic-name')
+ assert getLoggingMessage(4).contains('some resourceIdentifier')
+ assert getLoggingMessage(5).contains('module-set-tag1')
+ and: 'the operation Id is correctly received (logged)'
+ assert getLoggingMessage(6).contains('14')
+ }
+
+ def getLoggingMessage(int index) {
+ return logger.list[index].formattedMessage
+ }
+} \ No newline at end of file
diff --git a/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/DmiServiceImplSpec.groovy b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/DmiServiceImplSpec.groovy
new file mode 100644
index 00000000..8531d35f
--- /dev/null
+++ b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/DmiServiceImplSpec.groovy
@@ -0,0 +1,273 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2021-2022 Nordix Foundation
+ * Modifications Copyright (C) 2021-2022 Bell Canada
+ * ================================================================================
+ * 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.dmi.service
+
+import com.fasterxml.jackson.core.JsonProcessingException
+import com.fasterxml.jackson.databind.ObjectMapper
+import com.fasterxml.jackson.databind.ObjectWriter
+import org.onap.cps.ncmp.dmi.config.DmiPluginConfig
+import org.onap.cps.ncmp.dmi.exception.CmHandleRegistrationException
+import org.onap.cps.ncmp.dmi.exception.DmiException
+import org.onap.cps.ncmp.dmi.exception.ModuleResourceNotFoundException
+import org.onap.cps.ncmp.dmi.exception.ModulesNotFoundException
+import org.onap.cps.ncmp.dmi.exception.HttpClientRequestException
+import org.onap.cps.ncmp.dmi.service.model.ModuleReference
+import org.onap.cps.ncmp.dmi.model.YangResource
+import org.onap.cps.ncmp.dmi.model.YangResources
+import org.onap.cps.ncmp.dmi.service.client.NcmpRestClient
+import org.onap.cps.ncmp.dmi.service.model.ModuleSchema
+import org.onap.cps.ncmp.dmi.service.operation.SdncOperations
+import org.springframework.http.HttpStatus
+import org.springframework.http.ResponseEntity
+import spock.lang.Specification
+
+import static org.onap.cps.ncmp.dmi.model.DataAccessRequest.OperationEnum.CREATE
+import static org.onap.cps.ncmp.dmi.model.DataAccessRequest.OperationEnum.UPDATE
+
+class DmiServiceImplSpec extends Specification {
+
+
+ def mockNcmpRestClient = Mock(NcmpRestClient)
+ def mockDmiPluginProperties = Mock(DmiPluginConfig.DmiPluginProperties)
+ def spyObjectMapper = Spy(ObjectMapper)
+ def mockObjectMapper = Mock(ObjectMapper)
+ def mockSdncOperations = Mock(SdncOperations)
+ def objectUnderTest = new DmiServiceImpl(mockDmiPluginProperties, mockNcmpRestClient, mockSdncOperations, spyObjectMapper)
+
+ def 'Register cm handles with ncmp.'() {
+ given: 'some cm-handle ids'
+ def givenCmHandlesList = ['node1', 'node2']
+ and: 'json payload'
+ def expectedJson = '{"dmiPlugin":"test-dmi-service","createdCmHandles":[{"cmHandle":"node1"},{"cmHandle":"node2"}]}'
+ and: 'process returns "test-dmi-service" for service name'
+ mockDmiPluginProperties.getDmiServiceUrl() >> 'test-dmi-service'
+ when: 'the cm handles are registered'
+ objectUnderTest.registerCmHandles(givenCmHandlesList)
+ then: 'register cm handle with ncmp is called with the expected json and return "created" status'
+ 1 * mockNcmpRestClient.registerCmHandlesWithNcmp(expectedJson) >> new ResponseEntity<>(HttpStatus.CREATED)
+ }
+
+ def 'Register cm handles with ncmp called with exception #scenario.'() {
+ given: 'some cm-handle ids'
+ def cmHandlesList = ['node1', 'node2']
+ and: 'process returns "test-dmi-service" for service name'
+ mockDmiPluginProperties.getDmiServiceUrl() >> 'test-dmi-service'
+ and: 'returns #responseEntity'
+ mockNcmpRestClient.registerCmHandlesWithNcmp(_ as String) >> responseEntity
+ when: 'the cm handles are registered'
+ objectUnderTest.registerCmHandles(cmHandlesList)
+ then: 'a registration exception is thrown'
+ thrown(CmHandleRegistrationException.class)
+ where: 'given #scenario'
+ scenario | responseEntity
+ 'ncmp rest client returns bad request' | new ResponseEntity<>(HttpStatus.BAD_REQUEST)
+ 'ncmp rest client returns internal server error' | new ResponseEntity<>(HttpStatus.INTERNAL_SERVER_ERROR)
+ }
+
+ def 'Register cm handles with ncmp with wrong data.'() {
+ given: 'some cm-handle ids'
+ def cmHandlesList = ['node1', 'node2']
+ and: ' "JsonProcessingException" occurs during parsing'
+ objectUnderTest.objectMapper = mockObjectMapper
+ mockObjectMapper.writeValueAsString(_) >> { throw new JsonProcessingException('some error.') }
+ when: 'the cmHandles are registered'
+ objectUnderTest.registerCmHandles(cmHandlesList)
+ then: 'a dmi exception is thrown'
+ thrown(DmiException.class)
+ }
+
+ def ' Get modules for a cm-handle.'() {
+ given: 'a cm handle'
+ def cmHandle = 'node1'
+ and: 'process returns one module schema for the cmhandle'
+ def moduleSchema = new ModuleSchema(
+ identifier: "example-identifier",
+ namespace: "example:namespace",
+ version: "example-version")
+ mockSdncOperations.getModuleSchemasFromNode(cmHandle) >> List.of(moduleSchema)
+ when: 'modules for cmHandle is requested'
+ def result = objectUnderTest.getModulesForCmHandle(cmHandle)
+ then: 'one module is returned'
+ result.schemas.size() == 1
+ and: 'module has expected values'
+ with(result.schemas[0]) {
+ it.getRevision() == moduleSchema.getVersion()
+ it.getModuleName() == moduleSchema.getIdentifier()
+ it.getNamespace() == moduleSchema.getNamespace();
+ }
+ }
+
+ def 'no modules found for the cmhandle.'() {
+ given: 'cm handle id'
+ def cmHandle = 'node1'
+ and: 'process returns no modules'
+ mockSdncOperations.getModuleSchemasFromNode(cmHandle) >> Collections.emptyList();
+ when: 'modules for cm-handle is requested'
+ objectUnderTest.getModulesForCmHandle(cmHandle)
+ then: 'module not found exception is thrown'
+ thrown(ModulesNotFoundException)
+ }
+
+ def 'Get multiple module resources.'() {
+ given: 'a cmHandle'
+ def cmHandle = 'some-cmHandle'
+ and: 'multiple module references'
+ def moduleReference1 = new ModuleReference(name: 'name-1', revision: 'revision-1')
+ def moduleReference2 = new ModuleReference(name: 'name-2', revision: 'revision-2')
+ def moduleList = [moduleReference1, moduleReference2]
+ when: 'module resources is requested'
+ def result = objectUnderTest.getModuleResources(cmHandle, moduleList)
+ then: 'SDNC operation service is called same number of module references given'
+ 2 * mockSdncOperations.getModuleResource(cmHandle, _) >>> [new ResponseEntity<String>('{"ietf-netconf-monitoring:output": {"data": "some-data1"}}', HttpStatus.OK),
+ new ResponseEntity<String>('{"ietf-netconf-monitoring:output": {"data": "some-data2"}}', HttpStatus.OK)]
+ and: 'the result contains the expected properties'
+ def yangResources = new YangResources()
+ def yangResource1 = new YangResource(yangSource: 'some-data1', moduleName: 'name-1', revision: 'revision-1')
+ def yangResource2 = new YangResource(yangSource: 'some-data2', moduleName: 'name-2', revision: 'revision-2')
+ yangResources.add(yangResource1)
+ yangResources.add(yangResource2)
+ assert result == yangResources
+ }
+
+ def 'Get a module resource with module resource not found exception for #scenario.'() {
+ given: 'a cmHandle and module reference list'
+ def cmHandle = 'some-cmHandle'
+ def moduleReference = new ModuleReference(name: 'NAME', revision: 'REVISION')
+ def moduleList = [moduleReference]
+ when: 'module resources is requested'
+ objectUnderTest.getModuleResources(cmHandle, moduleList)
+ then: 'SDNC operation service is called once with a response body that contains no data'
+ 1 * mockSdncOperations.getModuleResource(cmHandle, _) >> new ResponseEntity<String>(responseBody, HttpStatus.OK)
+ and: 'an exception is thrown'
+ thrown(ModuleResourceNotFoundException)
+ where: 'the following values are returned'
+ scenario | responseBody
+ 'a response body containing no data object' | '{"ietf-netconf-monitoring:output": {"null": "some-data"}}'
+ 'a response body containing no ietf-netconf-monitoring:output object' | '{"null": {"data": "some-data"}}'
+ }
+
+ def 'Get module resources when sdnc returns #scenario response.'() {
+ given: 'sdnc returns a #scenario response'
+ mockSdncOperations.getModuleResource(_ as String, _ as String) >> new ResponseEntity<String>('some-response-body', httpStatus)
+ when: 'module resources is requested'
+ objectUnderTest.getModuleResources('some-cmHandle', [new ModuleReference()] as LinkedList<ModuleReference>)
+ then: '#expectedException is thrown'
+ thrown(expectedException)
+ where: 'the following values are returned'
+ scenario | httpStatus || expectedException
+ 'not found' | HttpStatus.NOT_FOUND || ModuleResourceNotFoundException
+ 'internal server error' | HttpStatus.INTERNAL_SERVER_ERROR || DmiException
+ }
+
+ def 'Get module resources with JSON processing exception.'() {
+ given: 'a json processing exception during process'
+ def mockObjectWriter = Mock(ObjectWriter)
+ spyObjectMapper.writer() >> mockObjectWriter
+ mockObjectWriter.withRootName(_) >> mockObjectWriter
+ def jsonProcessingException = new JsonProcessingException('')
+ mockObjectWriter.writeValueAsString(_) >> { throw jsonProcessingException }
+ when: 'module resources is requested'
+ objectUnderTest.getModuleResources('some-cmHandle', [new ModuleReference()] as LinkedList<ModuleReference>)
+ then: 'an exception is thrown'
+ def thrownException = thrown(DmiException.class)
+ and: 'the exception has the expected message and details'
+ thrownException.message == 'Unable to process JSON.'
+ thrownException.details == 'JSON exception occurred when creating the module request.'
+ and: 'the cause is the original json processing exception'
+ thrownException.cause == jsonProcessingException
+ }
+
+ def 'Get resource data for passthrough operational.'() {
+ given: 'sdnc operation returns OK response'
+ mockSdncOperations.getResouceDataForOperationalAndRunning(
+ 'someCmHandle',
+ 'someResourceId',
+ '(fields=x/y/z,depth=10,test=abc)',
+ 'content=all') >> new ResponseEntity<>('response json', HttpStatus.OK)
+ when: 'resource data is requested'
+ def response = objectUnderTest.getResourceData(
+ 'someCmHandle',
+ 'someResourceId',
+ '(fields=x/y/z,depth=10,test=abc)',
+ 'content=all')
+ then: 'response matches the response returned from the SDNC service'
+ response == 'response json'
+ }
+
+ def 'Get resource data with not found exception.'() {
+ given: 'sdnc operation returns "NOT_FOUND" response'
+ mockSdncOperations.getResouceDataForOperationalAndRunning(*_) >> new ResponseEntity<>(HttpStatus.NOT_FOUND)
+ when: 'resource data is requested'
+ objectUnderTest.getResourceData('someCmHandle', 'someResourceId',
+ '(fields=x/y/z,depth=10,test=abc)', 'content=config')
+ then: 'http client request exception'
+ thrown(HttpClientRequestException.class)
+ }
+
+ def 'Get resource data for passthrough running.'() {
+ given: 'sdnc operation returns OK response'
+ mockSdncOperations.getResouceDataForOperationalAndRunning(*_) >> new ResponseEntity<>('response json', HttpStatus.OK)
+ when: 'resource data is requested'
+ def response = objectUnderTest.getResourceData(
+ 'someCmHandle',
+ 'someResourceId',
+ '(fields=x/y/z,depth=10,test=abc)',
+ 'content=config')
+ then: 'response have expected json'
+ response == 'response json'
+ }
+
+ def 'Write resource data for passthrough running with a #scenario from sdnc.'() {
+ given: 'sdnc returns a response with #scenario'
+ mockSdncOperations.writeData(operationEnum, _, _, _, _) >> new ResponseEntity<String>('response json', httpResponse)
+ when: 'resource data is written to sdnc'
+ def response = objectUnderTest.writeData(operationEnum, 'some-cmHandle',
+ 'some-resourceIdentifier', 'some-dataType', '{some-data}')
+ then: 'the response matches the expected data'
+ response == 'response json'
+ where: 'the following values are used'
+ scenario | httpResponse | operationEnum
+ '200 OK with an update operation' | HttpStatus.OK | UPDATE
+ '201 CREATED with a create operation' | HttpStatus.CREATED | CREATE
+ }
+
+ def 'Write resource data with special characters.'() {
+ given: 'sdnc returns a created response'
+ mockSdncOperations.writeData(CREATE, 'some-cmHandle',
+ 'some-resourceIdentifier', 'some-dataType', 'data with quote " and \n new line') >> new ResponseEntity<String>('response json', HttpStatus.CREATED)
+ when: 'resource data is written to sdnc'
+ def response = objectUnderTest.writeData(CREATE, 'some-cmHandle',
+ 'some-resourceIdentifier', 'some-dataType', 'data with quote " and \n new line')
+ then: 'the response matches the expected data'
+ response == 'response json'
+ }
+
+ def 'Write resource data for passthrough running with a 500 response from sdnc.'() {
+ given: 'sdnc returns internal server error response'
+ mockSdncOperations.writeData(CREATE, _, _, _, _) >> new ResponseEntity<String>('response json', HttpStatus.INTERNAL_SERVER_ERROR)
+ when: 'resource data is written to sdnc'
+ objectUnderTest.writeData(CREATE, 'some-cmHandle',
+ 'some-resourceIdentifier', 'some-dataType', _ as String)
+ then: 'a dmi exception is thrown'
+ thrown(DmiException.class)
+ }
+} \ No newline at end of file
diff --git a/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/YangResourceExtractorSpec.groovy b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/YangResourceExtractorSpec.groovy
new file mode 100644
index 00000000..656cfcb5
--- /dev/null
+++ b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/YangResourceExtractorSpec.groovy
@@ -0,0 +1,98 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2021 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.dmi.service
+
+import com.google.gson.JsonSyntaxException
+import org.onap.cps.ncmp.dmi.exception.ModuleResourceNotFoundException
+import org.onap.cps.ncmp.dmi.service.model.ModuleReference
+import org.springframework.http.ResponseEntity
+import spock.lang.Specification
+
+class YangResourceExtractorSpec extends Specification {
+
+ static def BACK_SLASH = '\\';
+ static def NEW_LINE = '\n';
+ static def QUOTE = '"';
+ static def TAB = '\t';
+
+ static def YANG_ESCAPED_NEW_LINE = '\\n'
+ static def YANG_ESCAPED_BACK_SLASH = '\\\\'
+ static def YANG_ESCAPED_QUOTE = '\\"'
+ static def YANG_ESCAPED_TAB = '\\t'
+
+ static def SDNC_OUTPUT_JSON_NAME = '"ietf-netconf-monitoring:output"'
+
+ def moduleReference = new ModuleReference(name: 'test', revision: 'rev')
+ def responseEntity = Mock(ResponseEntity)
+
+ def 'Extract yang resource with escaped characters in the source.'() {
+ given: 'a response entity with a data field of value #jsonValue'
+ responseEntity.getBody() >> '{' + SDNC_OUTPUT_JSON_NAME + ': { "data": "' + jsonValue + '" }}'
+ when: 'the yang resource is extracted'
+ def result = YangResourceExtractor.toYangResource(moduleReference, responseEntity)
+ then: 'the yang source string is as expected'
+ result.getYangSource() == expectedString
+ where: 'the following data is used'
+ jsonValue || expectedString
+ 'line1' + YANG_ESCAPED_NEW_LINE + 'line2' || 'line1' + NEW_LINE + 'line2'
+ 'a' + YANG_ESCAPED_BACK_SLASH+'b' || 'a'+BACK_SLASH +'b'
+ 'a' + YANG_ESCAPED_QUOTE + 'b' || 'a'+QUOTE+'b'
+ 'a' + YANG_ESCAPED_TAB + 'b' || 'a'+TAB+'b'
+ }
+
+ def 'Extract yang resource with escaped characters in the source inside escaped double quotes.'() {
+ given: 'a response entity with a data field of value #jsonValue wrapped in escaped double quotes'
+ responseEntity.getBody() >> '{' + SDNC_OUTPUT_JSON_NAME + ': { "data": "' + YANG_ESCAPED_QUOTE + jsonValue + YANG_ESCAPED_QUOTE + '" }}'
+ when: 'the yang resource is extracted'
+ def result = YangResourceExtractor.toYangResource(moduleReference, responseEntity)
+ then: 'the yang source string is as expected'
+ result.getYangSource() == expectedString
+ where: 'the following data is used'
+ jsonValue || expectedString
+ 'line1' + YANG_ESCAPED_NEW_LINE + 'line2' || '"line1' + NEW_LINE + 'line2"'
+ 'a' + YANG_ESCAPED_BACK_SLASH+'b' || '"a'+BACK_SLASH +'b"'
+ 'a' + YANG_ESCAPED_QUOTE + 'b' || '"a'+QUOTE+'b"'
+ 'a' + YANG_ESCAPED_TAB + 'b' || '"a'+TAB+'b"'
+ }
+
+ def 'Attempt to extract yang resource with un-escaped double quotes in the source.'() {
+ given: 'a response entity with a data field with unescaped double quotes'
+ responseEntity.getBody() >> '{' + SDNC_OUTPUT_JSON_NAME + ': { "data": "' + QUOTE + 'some data' + QUOTE + '" }}'
+ when: 'Json is converted to String'
+ YangResourceExtractor.toYangResource(moduleReference, responseEntity)
+ then: 'the output of the method is equal to the output from the test template'
+ thrown(JsonSyntaxException)
+ }
+
+ def 'Attempt to extract yang resource without #without.'() {
+ given: 'a response entity with a body without #without'
+ responseEntity.getBody() >> jsonBody
+ when: 'Json is converted to String'
+ YangResourceExtractor.toYangResource(moduleReference, responseEntity)
+ then: 'the output of the method is equal to the output from the test template'
+ thrown(ModuleResourceNotFoundException)
+ where:
+ without | jsonBody
+ 'data' | '{' + SDNC_OUTPUT_JSON_NAME + ': { "something": "else" }}'
+ SDNC_OUTPUT_JSON_NAME | '{"something:else": { "data": "data" }}'
+ }
+
+}
diff --git a/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/client/NcmpRestClientSpec.groovy b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/client/NcmpRestClientSpec.groovy
new file mode 100644
index 00000000..4d7e27e2
--- /dev/null
+++ b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/client/NcmpRestClientSpec.groovy
@@ -0,0 +1,57 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2021-2022 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.dmi.service.client
+
+import org.onap.cps.ncmp.dmi.config.DmiConfiguration
+import org.springframework.http.HttpMethod
+import org.springframework.http.ResponseEntity
+import org.springframework.web.client.RestTemplate
+import spock.lang.Specification
+
+class NcmpRestClientSpec extends Specification {
+ def objectUnderTest = new NcmpRestClient(mockCpsProperties, mockRestTemplate)
+ def mockCpsProperties = Mock(DmiConfiguration.CpsProperties)
+ def mockRestTemplate = Mock(RestTemplate)
+
+ def setup() {
+ objectUnderTest.cpsProperties = mockCpsProperties
+ objectUnderTest.restTemplate = mockRestTemplate
+ }
+
+ def 'Register a cm handle.'() {
+ given: 'some request data'
+ def someRequestData = 'some request data'
+ and: 'configuration data'
+ mockCpsProperties.baseUrl >> 'http://some-uri'
+ mockCpsProperties.dmiRegistrationUrl >> 'some-url'
+ mockCpsProperties.authUsername >> 'some-username'
+ mockCpsProperties.authPassword >> 'some-password'
+ and: 'the rest template returns a valid response entity'
+ def mockResponseEntity = Mock(ResponseEntity)
+ when: 'registering a cm handle'
+ def result = objectUnderTest.registerCmHandlesWithNcmp(someRequestData)
+ then: 'the rest template is called with the correct uri and original request data in the body'
+ 1 * mockRestTemplate.exchange({ it.toString() == 'http://some-uri/some-url' },
+ HttpMethod.POST, { it.body.contains(someRequestData) }, String.class) >> mockResponseEntity
+ and: 'the output of the method is equal to the output from the rest template service'
+ result == mockResponseEntity
+ }
+} \ No newline at end of file
diff --git a/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/client/SdncRestconfClientSpec.groovy b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/client/SdncRestconfClientSpec.groovy
new file mode 100644
index 00000000..f334f780
--- /dev/null
+++ b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/client/SdncRestconfClientSpec.groovy
@@ -0,0 +1,102 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2021-2022 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.dmi.service.client
+
+import org.onap.cps.ncmp.dmi.config.DmiConfiguration
+import org.springframework.http.HttpEntity
+import org.springframework.http.HttpHeaders
+import org.springframework.http.HttpMethod
+import org.springframework.http.ResponseEntity
+import org.springframework.web.client.RestTemplate
+import spock.lang.Specification
+
+import static org.springframework.http.HttpMethod.GET
+import static org.springframework.http.HttpMethod.POST
+import static org.springframework.http.HttpMethod.DELETE
+import static org.springframework.http.HttpMethod.PUT
+
+class SdncRestconfClientSpec extends Specification {
+
+ def mockSdncProperties = Mock(DmiConfiguration.SdncProperties)
+ def mockRestTemplate = Mock(RestTemplate)
+ def objectUnderTest = new SdncRestconfClient(mockSdncProperties, mockRestTemplate)
+
+ def 'SDNC GET operation.'() {
+ given: 'a get resource url'
+ def getResourceUrl = '/getResourceUrl'
+ and: 'test configuration data'
+ setupTestConfigurationData()
+ and: 'the process returns a valid response entity'
+ def mockResponseEntity = Mock(ResponseEntity)
+ mockRestTemplate.exchange({ it.toString() == 'http://some-uri/getResourceUrl' },
+ HttpMethod.GET, _ as HttpEntity, String.class) >> mockResponseEntity
+ when: 'the resource is fetched'
+ def result = objectUnderTest.getOperation(getResourceUrl)
+ then: 'the output of the method is equal to the output from the rest template service'
+ result == mockResponseEntity
+ }
+
+ def 'SDNC #scenario operation called.'() {
+ given: 'some request data'
+ def someRequestData = 'some request data'
+ and: 'a url for get module resources'
+ def getModuleResourceUrl = '/getModuleResourceUrl'
+ and: 'test configuration data'
+ setupTestConfigurationData()
+ and: 'the process returns a valid response entity'
+ def mockResponseEntity = Mock(ResponseEntity)
+ when: 'the resource is fetched'
+ def result = objectUnderTest.httpOperationWithJsonData(expectedHttpMethod, getModuleResourceUrl, someRequestData, new HttpHeaders())
+ then: 'the rest template is called with the correct uri and json in the body'
+ 1 * mockRestTemplate.exchange({ it.toString() == 'http://some-uri/getModuleResourceUrl' },
+ expectedHttpMethod, { it.body.contains(someRequestData) }, String.class) >> mockResponseEntity
+ and: 'the output of the method is the same as the output from the rest template service'
+ result == mockResponseEntity
+ where: 'the following values are used'
+ scenario || expectedHttpMethod
+ 'POST' || POST
+ 'PUT' || PUT
+ 'GET' || GET
+ 'DELETE' || DELETE
+ }
+
+ def 'SDNC GET operation with headers.'() {
+ given: 'a get url'
+ def getResourceUrl = '/getResourceUrl'
+ and: 'test configuration data'
+ setupTestConfigurationData()
+ and: 'the process returns a valid response entity'
+ def mockResponseEntity = Mock(ResponseEntity)
+ mockRestTemplate.exchange({ it.toString() == 'http://some-uri/getResourceUrl' },
+ HttpMethod.GET, _ as HttpEntity, String.class) >> mockResponseEntity
+ when: 'the resource is fetched with headers'
+ def result = objectUnderTest.getOperation(getResourceUrl, new HttpHeaders())
+ then: 'the output of the method is equal to the output from the rest template service'
+ result == mockResponseEntity
+ }
+
+ def setupTestConfigurationData() {
+ mockSdncProperties.baseUrl >> 'http://some-uri'
+ mockSdncProperties.authUsername >> 'some-username'
+ mockSdncProperties.authPassword >> 'some-password'
+ mockSdncProperties.topologyId >> 'some-topology-id'
+ }
+} \ No newline at end of file
diff --git a/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/operation/SdncOperationsSpec.groovy b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/operation/SdncOperationsSpec.groovy
new file mode 100644
index 00000000..9dcb72e6
--- /dev/null
+++ b/dmi-service/src/test/groovy/org/onap/cps/ncmp/dmi/service/operation/SdncOperationsSpec.groovy
@@ -0,0 +1,176 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2021-2022 Nordix Foundation
+ * Modifications Copyright (C) 2021-2022 Bell Canada
+ * ================================================================================
+ * 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.dmi.service.operation
+
+import org.onap.cps.ncmp.dmi.TestUtils
+import org.onap.cps.ncmp.dmi.config.DmiConfiguration
+import org.onap.cps.ncmp.dmi.exception.SdncException
+import org.onap.cps.ncmp.dmi.service.client.SdncRestconfClient
+import org.spockframework.spring.SpringBean
+import org.springframework.beans.factory.annotation.Autowired
+import org.springframework.boot.test.context.SpringBootTest
+import org.springframework.http.HttpHeaders
+import org.springframework.http.HttpMethod
+import org.springframework.http.HttpStatus
+import org.springframework.http.ResponseEntity
+import org.springframework.test.context.ContextConfiguration
+import spock.lang.Specification
+
+import static org.onap.cps.ncmp.dmi.model.DataAccessRequest.OperationEnum.CREATE
+import static org.onap.cps.ncmp.dmi.model.DataAccessRequest.OperationEnum.DELETE
+import static org.onap.cps.ncmp.dmi.model.DataAccessRequest.OperationEnum.PATCH
+import static org.onap.cps.ncmp.dmi.model.DataAccessRequest.OperationEnum.UPDATE
+import static org.onap.cps.ncmp.dmi.model.DataAccessRequest.OperationEnum.READ
+
+@SpringBootTest
+@ContextConfiguration(classes = [DmiConfiguration.SdncProperties, SdncOperations])
+class SdncOperationsSpec extends Specification {
+
+ @SpringBean
+ SdncRestconfClient mockSdncRestClient = Mock()
+
+ @Autowired
+ SdncOperations objectUnderTest
+
+ def 'get modules from node.'() {
+ given: 'a node id and url'
+ def nodeId = 'node1'
+ def expectedUrl = '/rests/data/network-topology:network-topology/topology=test-topology/node=node1/yang-ext:mount/ietf-netconf-monitoring:netconf-state/schemas'
+ and: 'sdnc returns one module during process'
+ mockSdncRestClient.getOperation(expectedUrl) >>
+ ResponseEntity.ok(TestUtils.getResourceFileContent('ModuleSchema.json'))
+ when: 'module schemas from node are fetched'
+ def moduleSchemas = objectUnderTest.getModuleSchemasFromNode(nodeId)
+ then: 'one module is found'
+ moduleSchemas.size() == 1
+ and: 'module schema has expected values'
+ with(moduleSchemas[0]) {
+ it.getIdentifier() == "example-identifier"
+ it.getNamespace() == "example:namespace"
+ it.getVersion() == "example-version"
+ it.getFormat() == "example-format"
+ it.getLocation() == ["example-location"]
+ }
+ }
+
+ def 'No modules from Node: SDNC Response - #scenario .'() {
+ given: 'node id and url'
+ def nodeId = 'node1'
+ def expectedUrl = '/rests/data/network-topology:network-topology/topology=test-topology/node=node1/yang-ext:mount/ietf-netconf-monitoring:netconf-state/schemas'
+ and: 'sdnc operation returns #scenario'
+ mockSdncRestClient.getOperation(expectedUrl) >> ResponseEntity.ok(responseBody)
+ when: 'the module schemas are requested'
+ def moduleSchemas = objectUnderTest.getModuleSchemasFromNode(nodeId)
+ then: 'no module schemas are returned'
+ moduleSchemas.size() == 0
+ where:
+ scenario | responseBody
+ 'null response body' | null
+ 'empty response body ' | ''
+ 'no module schema' | '{ "ietf-netconf-monitoring:schemas" : { "schema" : [] } } '
+ }
+
+ def 'Error handling - modules from node: #scenario'() {
+ given: 'node id and url'
+ def nodeId = 'node1'
+ def expectedUrl = '/rests/data/network-topology:network-topology/topology=test-topology/node=node1/yang-ext:mount/ietf-netconf-monitoring:netconf-state/schemas'
+ and: '#scenario is returned during process'
+ mockSdncRestClient.getOperation(expectedUrl) >> new ResponseEntity<>(sdncResponseBody, sdncHttpStatus)
+ when: 'module schemas from node are fetched'
+ objectUnderTest.getModuleSchemasFromNode(nodeId)
+ then: 'SDNCException is thrown'
+ def thrownException = thrown(SdncException)
+ thrownException.getDetails().contains(expectedExceptionDetails)
+ where:
+ scenario | sdncHttpStatus | sdncResponseBody || expectedExceptionDetails
+ 'failed response from SDNC' | HttpStatus.BAD_REQUEST | '{ "errorMessage" : "incorrect input"}' || '{ "errorMessage" : "incorrect input"}'
+ 'invalid json response' | HttpStatus.OK | 'invalid-json' || 'SDNC response is not in the expected format'
+ 'response in unexpected json schema' | HttpStatus.OK | '{ "format" : "incorrect" }' || 'SDNC response is not in the expected format'
+ }
+
+ def 'Get module resources from SDNC.'() {
+ given: 'node id and url'
+ def nodeId = 'some-node'
+ def expectedUrl = '/rests/operations/network-topology:network-topology/topology=test-topology/node=some-node/yang-ext:mount/ietf-netconf-monitoring:get-schema'
+ when: 'module resource is fetched with the expected parameters'
+ objectUnderTest.getModuleResource(nodeId, 'some-json-data')
+ then: 'the SDNC Rest client is invoked with the correct parameters'
+ 1 * mockSdncRestClient.httpOperationWithJsonData(HttpMethod.POST, expectedUrl, 'some-json-data', _ as HttpHeaders)
+ }
+
+ def 'Get resource data from node to SDNC.'() {
+ given: 'expected url'
+ def expectedUrl = '/rests/data/network-topology:network-topology/topology=test-topology/node=node1/yang-ext:mount/testResourceId?a=1&b=2&content=testContent'
+ when: 'resource data is fetched for given node ID'
+ objectUnderTest.getResouceDataForOperationalAndRunning('node1', 'testResourceId',
+ '(a=1,b=2)', 'content=testContent')
+ then: 'the SDNC get operation is executed with the correct URL'
+ 1 * mockSdncRestClient.getOperation(expectedUrl)
+ }
+
+ def 'Write resource data with #scenario operation to SDNC.'() {
+ given: 'expected url'
+ def expectedUrl = '/rests/data/network-topology:network-topology/topology=test-topology/node=node1/yang-ext:mount/testResourceId'
+ when: 'write resource data for passthrough running is called'
+ objectUnderTest.writeData(operationEnum, 'node1', 'testResourceId', 'application/json', 'requestData')
+ then: 'the #expectedHttpMethod operation is executed with the correct parameters'
+ 1 * mockSdncRestClient.httpOperationWithJsonData(expectedHttpMethod, expectedUrl, 'requestData', _ as HttpHeaders)
+ where: 'the following values are used'
+ scenario | operationEnum || expectedHttpMethod
+ 'Create' | CREATE || HttpMethod.POST
+ 'Update' | UPDATE || HttpMethod.PUT
+ 'Read' | READ || HttpMethod.GET
+ 'Delete' | DELETE || HttpMethod.DELETE
+ 'Patch' | PATCH || HttpMethod.PATCH
+ }
+
+ def 'build query param list for SDNC where options #scenario'() {
+ when: 'query param list is built'
+ def result = objectUnderTest.buildQueryParamMap(optionsParamInQuery, 'd=4')
+ .toSingleValueMap().toString()
+ then: 'result matches the expected result'
+ result == expectedResult
+ where: 'following parameters are used'
+ scenario | optionsParamInQuery || expectedResult
+ 'is single key-value pair' | '(a=x)' || '[a:x, d:4]'
+ 'is multiple key-value pairs' | '(a=x,b=y,c=z)' || '[a:x, b:y, c:z, d:4]'
+ 'has / as special char' | '(a=x,b=y,c=t/z)' || '[a:x, b:y, c:t/z, d:4]'
+ 'has " as special char' | '(a=x,b=y,c="z")' || '[a:x, b:y, c:"z", d:4]'
+ 'has [] as special char' | '(a=x,b=y,c=[z])' || '[a:x, b:y, c:[z], d:4]'
+ 'has = in value' | '(a=(x=y),b=x=y)' || '[a:(x=y), b:x=y, d:4]'
+ 'is empty' | '' || '[:]'
+ 'is null' | null || '[:]'
+ }
+
+ def 'options parameters contains a comma #scenario'() {
+ when: 'query param list is built with #scenario'
+ def result = objectUnderTest.buildQueryParamMap(optionsParamInQuery, 'd=4').toSingleValueMap()
+ then: 'expect 2 elements from options where we are ignoring empty query param value, +1 from content query param (2+1) = 3 elements'
+ def expectedNoOfElements = 3
+ and: 'results contains equal elements as expected'
+ result.size() == expectedNoOfElements
+ where: 'following parameters are used'
+ scenario | optionsParamInQuery
+ '"," in value' | '(a=(x,y),b=y)'
+ '"," in string value' | '(a="x,y",b=y)'
+ }
+}
diff --git a/dmi-service/src/test/java/org/onap/cps/ncmp/dmi/TestUtils.java b/dmi-service/src/test/java/org/onap/cps/ncmp/dmi/TestUtils.java
new file mode 100644
index 00000000..c10d91a5
--- /dev/null
+++ b/dmi-service/src/test/java/org/onap/cps/ncmp/dmi/TestUtils.java
@@ -0,0 +1,54 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2021 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.dmi;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+
+/**
+ * Common convenience methods for testing.
+ */
+public class TestUtils {
+
+ /**
+ * Convert a file in the test resource folder to file.
+ *
+ * @param filename to name of the file in test/resources
+ * @return the file
+ * @throws IOException when there is an IO issue
+ */
+ public static File readFile(final String filename) {
+ return new File(ClassLoader.getSystemClassLoader().getResource(filename).getFile());
+ }
+
+ /**
+ * Convert a file in the test resource folder to a string.
+ *
+ * @param filename to name of the file in test/resources
+ * @return the content of the file as a String
+ * @throws IOException when there is an IO issue
+ */
+ public static String getResourceFileContent(final String filename) throws IOException {
+ final File file = readFile(filename);
+ return new String(Files.readAllBytes(file.toPath()));
+ }
+}
diff --git a/dmi-service/src/test/java/org/onap/cps/ncmp/dmi/rest/controller/TestController.java b/dmi-service/src/test/java/org/onap/cps/ncmp/dmi/rest/controller/TestController.java
new file mode 100644
index 00000000..5240e239
--- /dev/null
+++ b/dmi-service/src/test/java/org/onap/cps/ncmp/dmi/rest/controller/TestController.java
@@ -0,0 +1,35 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2021 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.dmi.rest.controller;
+
+import org.springframework.http.HttpStatus;
+import org.springframework.http.ResponseEntity;
+import org.springframework.web.bind.annotation.GetMapping;
+import org.springframework.web.bind.annotation.RestController;
+
+@RestController
+public class TestController {
+
+ @GetMapping("/test")
+ ResponseEntity<String> test() {
+ return new ResponseEntity<>(HttpStatus.OK);
+ }
+}
diff --git a/dmi-service/src/test/resources/ModuleSchema.json b/dmi-service/src/test/resources/ModuleSchema.json
new file mode 100644
index 00000000..50c67154
--- /dev/null
+++ b/dmi-service/src/test/resources/ModuleSchema.json
@@ -0,0 +1,15 @@
+{
+ "ietf-netconf-monitoring:schemas": {
+ "schema": [
+ {
+ "identifier": "example-identifier",
+ "version": "example-version",
+ "format": "example-format",
+ "namespace": "example:namespace",
+ "location": [
+ "example-location"
+ ]
+ }
+ ]
+ }
+} \ No newline at end of file
diff --git a/dmi-service/src/test/resources/application.yml b/dmi-service/src/test/resources/application.yml
new file mode 100644
index 00000000..ddc2b45f
--- /dev/null
+++ b/dmi-service/src/test/resources/application.yml
@@ -0,0 +1,79 @@
+# ============LICENSE_START=======================================================
+# 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.
+# 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=========================================================
+
+rest:
+ api:
+ dmi-base-path: /dmi
+
+security:
+ permit-uri: /actuator/**,/swagger-ui/**,/swagger-resources/**,/v3/api-docs
+ auth:
+ username: cpsuser
+ password: cpsr0cks!
+
+sdnc:
+ baseUrl: http://test
+ topologyId: test-topology
+ auth:
+ username: test
+ password: test
+
+cps-core:
+ baseUrl: some url for cps
+ dmiRegistrationUrl: some registration url
+ auth:
+ username: some cps core user
+ password: some cps core password
+
+dmi:
+ service:
+ url: some url for the dmi service
+ avc:
+ subscription-topic: ncmp-dmi-cm-avc-subscription
+ subscription-response-topic: dmi-ncmp-cm-avc-subscription
+
+spring:
+ application:
+ name: ncmp-dmi-plugin
+ mvc:
+ pathmatch:
+ matching-strategy: ANT_PATH_MATCHER
+ kafka:
+ bootstrap-servers: ${KAFKA_BOOTSTRAP_SERVER}
+ security:
+ protocol: PLAINTEXT
+ producer:
+ key-serializer: org.apache.kafka.common.serialization.StringSerializer
+ value-serializer: io.cloudevents.kafka.CloudEventSerializer
+ client-id: ncmp-dmi-plugin
+ consumer:
+ group-id: ${NCMP_CONSUMER_GROUP_ID:ncmp-group}
+ key-deserializer: org.springframework.kafka.support.serializer.ErrorHandlingDeserializer
+ value-deserializer: org.springframework.kafka.support.serializer.ErrorHandlingDeserializer
+ properties:
+ spring.deserializer.key.delegate.class: org.apache.kafka.common.serialization.StringDeserializer
+ spring.deserializer.value.delegate.class: io.cloudevents.kafka.CloudEventDeserializer
+ spring.json.use.type.headers: false
+
+app:
+ ncmp:
+ async:
+ topic: ${NCMP_ASYNC_M2M_TOPIC:ncmp-async-m2m}
+
+logging:
+ format: json
diff --git a/dmi-service/src/test/resources/cmNotificationSubscriptionCreationEvent.json b/dmi-service/src/test/resources/cmNotificationSubscriptionCreationEvent.json
new file mode 100644
index 00000000..3b780976
--- /dev/null
+++ b/dmi-service/src/test/resources/cmNotificationSubscriptionCreationEvent.json
@@ -0,0 +1,43 @@
+{
+ "data": {
+ "cmhandles": [
+ {
+ "cmhandleId": "CMHandle1",
+ "private-properties": {
+ "prop1": "prop-value"
+ }
+ },
+ {
+ "cmhandleId": "CMHandle2",
+ "private-properties": {
+ "prop-x": "prop-valuex",
+ "prop-p": "prop-valuep"
+ }
+ },
+ {
+ "cmhandleId": "CMHandle3",
+ "private-properties": {
+ "prop-y": "prop-valuey"
+ }
+ }
+ ],
+ "predicates": [
+ {
+ "targetFilter": [
+ "CMHandle1",
+ "CMHandle2",
+ "CMHandle3"
+ ],
+ "scopeFilter": {
+ "datastore": "ncmp-datastore:passthrough-running",
+ "xpath-filter": [
+ "//_3gpp-nr-nrm-gnbdufunction:GNBDUFunction/_3gpp-nr-nrm-nrcelldu:NRCellDU/",
+ "//_3gpp-nr-nrm-gnbcuupfunction:GNBCUUPFunction//",
+ "//_3gpp-nr-nrm-gnbcucpfunction:GNBCUCPFunction/_3gpp-nr-nrm-nrcelldu:NRCellCU//",
+ "//_3gpp-nr-nrm-nrsectorcarrier:NRSectorCarrier//"
+ ]
+ }
+ }
+ ]
+ }
+} \ No newline at end of file
diff --git a/dmi-service/src/test/resources/createDataWithNormalChar.json b/dmi-service/src/test/resources/createDataWithNormalChar.json
new file mode 100644
index 00000000..31cdf1c5
--- /dev/null
+++ b/dmi-service/src/test/resources/createDataWithNormalChar.json
@@ -0,0 +1,8 @@
+{
+ "operation": "create",
+ "dataType": "application/json",
+ "data": "normal request body",
+ "cmHandleProperties": {
+ "some-property": "some-property-value"
+ }
+} \ No newline at end of file
diff --git a/dmi-service/src/test/resources/createDataWithSpecialChar.json b/dmi-service/src/test/resources/createDataWithSpecialChar.json
new file mode 100644
index 00000000..1e7622ee
--- /dev/null
+++ b/dmi-service/src/test/resources/createDataWithSpecialChar.json
@@ -0,0 +1,8 @@
+{
+ "operation": "create",
+ "dataType": "application/json",
+ "data": "data with quote \" and new line \n",
+ "cmHandleProperties": {
+ "some-property": "some-property-value"
+ }
+} \ No newline at end of file
diff --git a/dmi-service/src/test/resources/deleteData.json b/dmi-service/src/test/resources/deleteData.json
new file mode 100644
index 00000000..2233fa01
--- /dev/null
+++ b/dmi-service/src/test/resources/deleteData.json
@@ -0,0 +1,8 @@
+{
+ "operation": "delete",
+ "dataType": "application/json",
+ "data": "normal request body",
+ "cmHandleProperties": {
+ "some-property": "some-property-value"
+ }
+} \ No newline at end of file
diff --git a/dmi-service/src/test/resources/moduleResources.json b/dmi-service/src/test/resources/moduleResources.json
new file mode 100644
index 00000000..23adfcba
--- /dev/null
+++ b/dmi-service/src/test/resources/moduleResources.json
@@ -0,0 +1,18 @@
+{
+ "data": {
+ "modules": [
+ {
+ "name": "ietf-yang-library",
+ "revision": "2016-06-21"
+ },
+ {
+ "name": "nc-notifications",
+ "revision": "2008-07-14"
+ }
+ ]
+ },
+ "cmHandleProperties": {
+ "subsystemId": "system-001"
+ },
+ "moduleSetTag": "module-set-tag1"
+}
diff --git a/dmi-service/src/test/resources/patchData.json b/dmi-service/src/test/resources/patchData.json
new file mode 100644
index 00000000..e5466eaf
--- /dev/null
+++ b/dmi-service/src/test/resources/patchData.json
@@ -0,0 +1,8 @@
+{
+ "operation": "patch",
+ "dataType": "application/yang.patch+json",
+ "data": "normal request body",
+ "cmHandleProperties": {
+ "some-property": "some-property-value"
+ }
+} \ No newline at end of file
diff --git a/dmi-service/src/test/resources/readData.json b/dmi-service/src/test/resources/readData.json
new file mode 100644
index 00000000..53f6d2ed
--- /dev/null
+++ b/dmi-service/src/test/resources/readData.json
@@ -0,0 +1,9 @@
+{
+ "operation": "read",
+ "dataType": "application/json",
+ "data": "normal request body",
+ "cmHandleProperties": {
+ "some-property": "some-property-value"
+ },
+ "moduleSetTag": "module-set-tag-example"
+} \ No newline at end of file
diff --git a/dmi-service/src/test/resources/updateData.json b/dmi-service/src/test/resources/updateData.json
new file mode 100644
index 00000000..7cbf4f0c
--- /dev/null
+++ b/dmi-service/src/test/resources/updateData.json
@@ -0,0 +1,8 @@
+{
+ "operation": "update",
+ "dataType": "application/json",
+ "data": "normal request body",
+ "cmHandleProperties": {
+ "some-property": "some-property-value"
+ }
+} \ No newline at end of file