diff options
Diffstat (limited to 'dmi-service/src/test')
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 |