summaryrefslogtreecommitdiffstats
path: root/cps-ncmp-service/src/test/groovy
diff options
context:
space:
mode:
Diffstat (limited to 'cps-ncmp-service/src/test/groovy')
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplSpec.groovy42
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/async/NcmpAsyncBatchEventConsumerSpec.groovy104
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/client/DmiRestClientSpec.groovy6
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandlerImplSpec.groovy16
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCreatorSpec.groovy18
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsPublisherSpec.groovy40
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsServiceSpec.groovy34
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/operations/DmiDataOperationsSpec.groovy43
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/operations/DmiModelOperationsSpec.groovy16
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/ResourceDataBatchRequestUtilsSpec.groovy80
-rw-r--r--cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/RestQueryParametersValidatorSpec.groovy14
11 files changed, 332 insertions, 81 deletions
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplSpec.groovy
index 3d8e9cb2e8..79f7e50e76 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/NetworkCmProxyDataServiceImplSpec.groovy
@@ -33,11 +33,13 @@ import org.onap.cps.ncmp.api.inventory.CompositeState
import org.onap.cps.ncmp.api.inventory.InventoryPersistence
import org.onap.cps.ncmp.api.inventory.LockReasonCategory
import org.onap.cps.ncmp.api.inventory.DataStoreSyncState
+import org.onap.cps.ncmp.api.models.BatchOperationDefinition
import org.onap.cps.ncmp.api.models.CmHandleQueryApiParameters
import org.onap.cps.ncmp.api.models.CmHandleQueryServiceParameters
import org.onap.cps.ncmp.api.models.ConditionApiProperties
import org.onap.cps.ncmp.api.models.DmiPluginRegistration
import org.onap.cps.ncmp.api.models.NcmpServiceCmHandle
+import org.onap.cps.ncmp.api.models.ResourceDataBatchRequest
import org.onap.cps.spi.exceptions.CpsException
import org.onap.cps.spi.model.ConditionProperties
import spock.lang.Shared
@@ -54,8 +56,8 @@ import spock.lang.Specification
import static org.onap.cps.ncmp.api.impl.operations.DatastoreType.PASSTHROUGH_OPERATIONAL
import static org.onap.cps.ncmp.api.impl.operations.DatastoreType.PASSTHROUGH_RUNNING
-import static org.onap.cps.ncmp.api.impl.operations.OperationEnum.CREATE
-import static org.onap.cps.ncmp.api.impl.operations.OperationEnum.UPDATE
+import static org.onap.cps.ncmp.api.impl.operations.OperationType.CREATE
+import static org.onap.cps.ncmp.api.impl.operations.OperationType.UPDATE
class NetworkCmProxyDataServiceImplSpec extends Specification {
@@ -133,21 +135,13 @@ class NetworkCmProxyDataServiceImplSpec extends Specification {
response == '{dmi-response}'
}
- def 'Get bulk resource data for #datastoreName from DMI.'() {
+ def 'Get batch resource data for #datastoreName from DMI.'() {
given: 'cpsDataService returns valid data node'
- mockDataNode()
- and: 'DMI returns valid response and data'
- mockDmiDataOperations.getResourceDataFromDmi(datastoreName, ['testCmHandle'],
- 'testResourceId', OPTIONS_PARAM,'some topic','requestId') >>
- new ResponseEntity<>('{dmi-bulk-response}', HttpStatus.OK)
+ def resourceDataBatchRequest = getResourceDataBatchRequest(datastoreName)
when: 'get batch resource data is called'
- def response = objectUnderTest.getResourceDataForCmHandleBatch(datastoreName, ['testCmHandle'],
- 'testResourceId',
- OPTIONS_PARAM,
- 'some topic',
- 'requestId')
- then: 'get bulk resource data returns expected response'
- response == '{dmi-bulk-response}'
+ objectUnderTest.requestResourceDataForCmHandleBatch('some topic', resourceDataBatchRequest, 'requestId')
+ then: 'get batch resource data returns expected response'
+ 1 * mockDmiDataOperations.requestResourceDataFromDmi('some topic', resourceDataBatchRequest, 'requestId')
where: 'the following data stores are used'
datastoreName << [PASSTHROUGH_RUNNING.datastoreName, PASSTHROUGH_OPERATIONAL.datastoreName]
}
@@ -373,4 +367,22 @@ class NetworkCmProxyDataServiceImplSpec extends Specification {
mockCpsDataService.getDataNodes('NCMP-Admin', 'ncmp-dmi-registry',
cmHandleXPath, FetchDescendantsOption.INCLUDE_ALL_DESCENDANTS) >> dataNode
}
+
+ def getResourceDataBatchRequest(datastore) {
+ def resourceDataBatchRequest = new ResourceDataBatchRequest()
+ def batchOperationDefinitions = new ArrayList()
+ batchOperationDefinitions.add(getBatchOperationDefinition(datastore))
+ resourceDataBatchRequest.setBatchOperationDefinitions(batchOperationDefinitions)
+ }
+
+ def getBatchOperationDefinition(datastore) {
+ def batchOperationDefinition = new BatchOperationDefinition()
+ batchOperationDefinition.setOperation("read")
+ batchOperationDefinition.setOperationId("operational-12")
+ batchOperationDefinition.setDatastore(datastore)
+ def targetIds = new ArrayList()
+ targetIds.add("some-cm-handle")
+ batchOperationDefinition.setCmHandleIds(targetIds)
+ return batchOperationDefinition
+ }
}
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/async/NcmpAsyncBatchEventConsumerSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/async/NcmpAsyncBatchEventConsumerSpec.groovy
new file mode 100644
index 0000000000..65c43a011d
--- /dev/null
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/async/NcmpAsyncBatchEventConsumerSpec.groovy
@@ -0,0 +1,104 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2023 Nordix Foundation
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ * SPDX-License-Identifier: Apache-2.0
+ * ============LICENSE_END=========================================================
+ */
+
+package org.onap.cps.ncmp.api.impl.async
+
+import com.fasterxml.jackson.databind.ObjectMapper
+import org.apache.commons.lang3.SerializationUtils
+import org.apache.kafka.clients.consumer.ConsumerRecord
+import org.apache.kafka.clients.consumer.KafkaConsumer
+import org.apache.kafka.common.header.internals.RecordHeader
+import org.onap.cps.ncmp.api.impl.events.EventsPublisher
+import org.onap.cps.ncmp.api.kafka.MessagingBaseSpec
+import org.onap.cps.ncmp.events.async.BatchDataResponseEventV1
+import org.onap.cps.ncmp.utils.TestUtils
+import org.onap.cps.utils.JsonObjectMapper
+import org.spockframework.spring.SpringBean
+import org.springframework.beans.factory.annotation.Autowired
+import org.springframework.boot.test.context.SpringBootTest
+import org.springframework.kafka.listener.adapter.RecordFilterStrategy
+import org.springframework.test.annotation.DirtiesContext
+import org.testcontainers.spock.Testcontainers
+
+import java.time.Duration
+
+@SpringBootTest(classes = [EventsPublisher, NcmpAsyncBatchEventConsumer, BatchRecordFilterStrategy,JsonObjectMapper,
+ ObjectMapper])
+@Testcontainers
+@DirtiesContext
+class NcmpAsyncBatchEventConsumerSpec extends MessagingBaseSpec {
+
+ @SpringBean
+ EventsPublisher asyncBatchEventPublisher = new EventsPublisher<BatchDataResponseEventV1>(kafkaTemplate)
+
+ @SpringBean
+ NcmpAsyncBatchEventConsumer asyncBatchEventConsumer = new NcmpAsyncBatchEventConsumer(asyncBatchEventPublisher)
+
+ @Autowired
+ JsonObjectMapper jsonObjectMapper
+
+ @Autowired
+ RecordFilterStrategy<Object, Object> recordFilterStrategy
+
+ def kafkaConsumer = new KafkaConsumer<>(consumerConfigProperties('test'))
+ def static clientTopic = 'client-topic'
+ def static batchEventType = 'org.onap.cps.ncmp.events.async.BatchDataResponseEventV1'
+
+ def 'Consume and publish event to client specified topic'() {
+ given: 'consumer subscribing to client topic'
+ kafkaConsumer.subscribe([clientTopic])
+ and: 'consumer record for batch event'
+ def consumerRecordIn = createConsumerRecord(batchEventType)
+ when: 'the batch event is consumed and published to client specified topic'
+ asyncBatchEventConsumer.consumeAndPublish(consumerRecordIn)
+ and: 'the client specified topic is polled'
+ def consumerRecordOut = kafkaConsumer.poll(Duration.ofMillis(1500))[0]
+ then: 'verifying consumed event operationID is same as published event operationID'
+ def operationIdIn = consumerRecordIn.value.event.batchResponses[0].operationId
+ def operationIdOut = jsonObjectMapper.convertJsonString((String)consumerRecordOut.value(), BatchDataResponseEventV1.class).event.batchResponses[0].operationId
+ assert operationIdIn == operationIdOut
+ }
+
+ def 'Filter an event with type #eventType'() {
+ given: 'consumer record for event with type #eventType'
+ def consumerRecord = createConsumerRecord(eventType)
+ when: 'while consuming the topic ncmp-async-m2m it executes the filter strategy'
+ def result = recordFilterStrategy.filter(consumerRecord)
+ then: 'the event is #description'
+ assert result == expectedResult
+ where: 'filter the event based on the eventType #eventType'
+ description | eventType || expectedResult
+ 'not filtered(the consumer will see the event)' | batchEventType || false
+ 'filtered(the consumer will not see the event)' | 'wrongType' || true
+ }
+
+ def createConsumerRecord(eventTypeAsString) {
+ def jsonData = TestUtils.getResourceFileContent('batchDataEvent.json')
+ def testEventSent = jsonObjectMapper.convertJsonString(jsonData, BatchDataResponseEventV1.class)
+ def eventTarget = SerializationUtils.serialize(clientTopic)
+ def eventType = SerializationUtils.serialize(eventTypeAsString)
+ def eventId = SerializationUtils.serialize('12345')
+ def consumerRecord = new ConsumerRecord<String, Object>(clientTopic, 1, 1L, '123', testEventSent)
+ consumerRecord.headers().add(new RecordHeader('eventId', eventId))
+ consumerRecord.headers().add(new RecordHeader('eventTarget', eventTarget))
+ consumerRecord.headers().add(new RecordHeader('eventType', eventType))
+ return consumerRecord
+ }
+}
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/client/DmiRestClientSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/client/DmiRestClientSpec.groovy
index b38ca10f7b..6b0355eee8 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/client/DmiRestClientSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/client/DmiRestClientSpec.groovy
@@ -34,9 +34,9 @@ import org.springframework.web.client.HttpServerErrorException
import org.springframework.web.client.RestTemplate
import spock.lang.Specification
-import static org.onap.cps.ncmp.api.impl.operations.OperationEnum.READ
-import static org.onap.cps.ncmp.api.impl.operations.OperationEnum.PATCH
-import static org.onap.cps.ncmp.api.impl.operations.OperationEnum.CREATE
+import static org.onap.cps.ncmp.api.impl.operations.OperationType.READ
+import static org.onap.cps.ncmp.api.impl.operations.OperationType.PATCH
+import static org.onap.cps.ncmp.api.impl.operations.OperationType.CREATE
@SpringBootTest
@ContextConfiguration(classes = [NcmpConfiguration.DmiProperties, DmiRestClient])
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandlerImplSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandlerImplSpec.groovy
index f660be7103..e449d65ac2 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandlerImplSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCmHandleStateHandlerImplSpec.groovy
@@ -54,7 +54,7 @@ class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
then: 'state is saved using inventory persistence'
expectedCallsToInventoryPersistence * mockInventoryPersistence.saveCmHandleState(cmHandleId, _)
and: 'event service is called to publish event'
- expectedCallsToEventService * mockLcmEventsService.publishLcmEvent(cmHandleId, _)
+ expectedCallsToEventService * mockLcmEventsService.publishLcmEvent(cmHandleId, _, _)
where: 'state change parameters are provided'
stateChange | fromCmHandleState | toCmHandleState || expectedCallsToInventoryPersistence | expectedCallsToEventService
'ADVISED to READY' | ADVISED | READY || 1 | 1
@@ -73,7 +73,7 @@ class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
then: 'state is saved using inventory persistence'
1 * mockInventoryPersistence.saveCmHandle(yangModelCmHandle)
and: 'event service is called to publish event'
- 1 * mockLcmEventsService.publishLcmEvent(cmHandleId, _)
+ 1 * mockLcmEventsService.publishLcmEvent(cmHandleId, _, _)
}
def 'Update and Publish Events on State Change from LOCKED to ADVISED'() {
@@ -90,7 +90,7 @@ class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
}
}
and: 'event service is called to publish event'
- 1 * mockLcmEventsService.publishLcmEvent(cmHandleId, _)
+ 1 * mockLcmEventsService.publishLcmEvent(cmHandleId, _, _)
}
def 'Update and Publish Events on State Change to READY'() {
@@ -111,7 +111,7 @@ class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
}
}
and: 'event service is called to publish event'
- 1 * mockLcmEventsService.publishLcmEvent(cmHandleId, _)
+ 1 * mockLcmEventsService.publishLcmEvent(cmHandleId, _, _)
}
def 'Update cmHandle state to "DELETING"' (){
@@ -125,7 +125,7 @@ class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
and: 'method to persist cm handle state is called once'
1 * mockInventoryPersistence.saveCmHandleState(yangModelCmHandle.getId(), yangModelCmHandle.getCompositeState())
and: 'the method to publish Lcm event is called once'
- 1 * mockLcmEventsService.publishLcmEvent(cmHandleId, _)
+ 1 * mockLcmEventsService.publishLcmEvent(cmHandleId, _, _)
}
def 'Update cmHandle state to "DELETED"' (){
@@ -137,7 +137,7 @@ class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
then: 'the cm handle state is as expected'
yangModelCmHandle.getCompositeState().getCmHandleState() == DELETED
and: 'the method to publish Lcm event is called once'
- 1 * mockLcmEventsService.publishLcmEvent(cmHandleId, _)
+ 1 * mockLcmEventsService.publishLcmEvent(cmHandleId, _, _)
}
def 'No state change and no event to be published'() {
@@ -167,7 +167,7 @@ class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
}
}
and: 'event service is called to publish event'
- 2 * mockLcmEventsService.publishLcmEvent(_, _)
+ 2 * mockLcmEventsService.publishLcmEvent(_, _, _)
}
@@ -183,7 +183,7 @@ class LcmEventsCmHandleStateHandlerImplSpec extends Specification {
}
}
and: 'event service is called to publish event'
- 2 * mockLcmEventsService.publishLcmEvent(_, _)
+ 2 * mockLcmEventsService.publishLcmEvent(_, _, _)
}
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCreatorSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCreatorSpec.groovy
index f4adfc587c..6d7d6250f1 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCreatorSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsCreatorSpec.groovy
@@ -20,10 +20,11 @@
package org.onap.cps.ncmp.api.impl.events.lcm
+import org.mapstruct.factory.Mappers
import org.onap.cps.ncmp.api.inventory.CmHandleState
import org.onap.cps.ncmp.api.inventory.CompositeState
import org.onap.cps.ncmp.api.models.NcmpServiceCmHandle
-import org.onap.ncmp.cmhandle.event.lcm.Values
+import org.onap.cps.ncmp.events.lcm.v1.Values
import spock.lang.Specification
import static org.onap.cps.ncmp.api.inventory.CmHandleState.ADVISED
@@ -32,7 +33,9 @@ import static org.onap.cps.ncmp.api.inventory.CmHandleState.READY
class LcmEventsCreatorSpec extends Specification {
- def objectUnderTest = new LcmEventsCreator()
+ LcmEventHeaderMapper lcmEventsHeaderMapper = Mappers.getMapper(LcmEventHeaderMapper)
+
+ def objectUnderTest = new LcmEventsCreator(lcmEventsHeaderMapper)
def cmHandleId = 'test-cm-handle'
def 'Map the LcmEvent for #operation'() {
@@ -159,4 +162,15 @@ class LcmEventsCreatorSpec extends Specification {
'null to null' | null | null
}
+
+ def 'Map the LcmEventHeader'() {
+ given: 'NCMP cm handle details with current and old details'
+ def existingNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(cmHandleState: ADVISED))
+ def targetNcmpServiceCmHandle = new NcmpServiceCmHandle(cmHandleId: cmHandleId, compositeState: new CompositeState(cmHandleState: READY))
+ when: 'the event header is populated'
+ def result = objectUnderTest.populateLcmEventHeader(cmHandleId, targetNcmpServiceCmHandle, existingNcmpServiceCmHandle)
+ then: 'the header has fields populated'
+ assert result.eventCorrelationId == cmHandleId
+ assert result.eventId != null
+ }
} \ No newline at end of file
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsPublisherSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsPublisherSpec.groovy
index 7c9464dccb..93741261f6 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsPublisherSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsPublisherSpec.groovy
@@ -24,14 +24,15 @@ import com.fasterxml.jackson.databind.ObjectMapper
import org.apache.kafka.clients.consumer.KafkaConsumer
import org.onap.cps.ncmp.api.impl.events.EventsPublisher
import org.onap.cps.ncmp.api.kafka.MessagingBaseSpec
+import org.onap.cps.ncmp.events.lcm.v1.Event
+import org.onap.cps.ncmp.events.lcm.v1.LcmEvent
import org.onap.cps.ncmp.utils.TestUtils
import org.onap.cps.utils.JsonObjectMapper
-import org.onap.ncmp.cmhandle.event.lcm.Event
-import org.onap.ncmp.cmhandle.event.lcm.LcmEvent
import org.spockframework.spring.SpringBean
import org.springframework.beans.factory.annotation.Autowired
import org.springframework.boot.test.context.SpringBootTest
import org.springframework.test.annotation.DirtiesContext
+import org.springframework.util.SerializationUtils
import org.testcontainers.spock.Testcontainers
import java.time.Duration
@@ -55,19 +56,35 @@ class LcmEventsPublisherSpec extends MessagingBaseSpec {
def 'Produce and Consume Lcm Event'() {
given: 'event key and event data'
def eventKey = 'lcm'
+ def eventId = 'test-uuid'
+ def eventCorrelationId = 'cmhandle-test'
+ def eventSource = 'org.onap.ncmp'
+ def eventTime = '2022-12-31T20:30:40.000+0000'
+ def eventType = 'org.onap.ncmp.cmhandle.lcm.event'
+ def eventSchema = 'org.onap.ncmp.cmhandle.lcm.event'
+ def eventSchemaVersion = 'v1'
def eventData = new LcmEvent(
- eventId: 'test-uuid',
- eventCorrelationId: 'cmhandle-as-correlationid',
- eventSource: 'org.onap.ncmp',
- eventTime: '2022-12-31T20:30:40.000+0000',
- eventType: 'org.onap.ncmp.cmhandle.lcm.event',
- eventSchema: 'org.onap.ncmp.cmhandle.lcm.event',
- eventSchemaVersion: 'v1',
+ eventId: eventId,
+ eventCorrelationId: eventCorrelationId,
+ eventSource: eventSource,
+ eventTime: eventTime,
+ eventType: eventType,
+ eventSchema: eventSchema,
+ eventSchemaVersion: eventSchemaVersion,
event: new Event(cmHandleId: 'cmhandle-test'))
+ and: 'we have a event header'
+ def eventHeader = [
+ eventId : eventId,
+ eventCorrelationId: eventCorrelationId,
+ eventSource : eventSource,
+ eventTime : eventTime,
+ eventType : eventType,
+ eventSchema : eventSchema,
+ eventSchemaVersion: eventSchemaVersion]
and: 'consumer has a subscription'
kafkaConsumer.subscribe([testTopic] as List<String>)
when: 'an event is published'
- lcmEventsPublisher.publishEvent(testTopic, eventKey, eventData)
+ lcmEventsPublisher.publishEvent(testTopic, eventKey, eventHeader, eventData)
and: 'topic is polled'
def records = kafkaConsumer.poll(Duration.ofMillis(1500))
then: 'poll returns one record'
@@ -79,5 +96,8 @@ class LcmEventsPublisherSpec extends MessagingBaseSpec {
def expectedJsonString = TestUtils.getResourceFileContent('expectedLcmEvent.json')
def expectedLcmEvent = jsonObjectMapper.convertJsonString(expectedJsonString, LcmEvent.class)
assert expectedLcmEvent == jsonObjectMapper.convertJsonString(record.value, LcmEvent.class)
+ and: 'record header matches the expected parameters'
+ assert SerializationUtils.deserialize(record.headers().lastHeader('eventId').value()) == eventId
+ assert SerializationUtils.deserialize(record.headers().lastHeader('eventCorrelationId').value()) == eventCorrelationId
}
}
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsServiceSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsServiceSpec.groovy
index 65f4d50c68..2d3f8ac516 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsServiceSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/events/lcm/LcmEventsServiceSpec.groovy
@@ -21,26 +21,42 @@
package org.onap.cps.ncmp.api.impl.events.lcm
import org.onap.cps.ncmp.api.impl.events.EventsPublisher
-import org.onap.ncmp.cmhandle.event.lcm.LcmEvent
+import org.onap.cps.ncmp.events.lcm.v1.LcmEvent
+import org.onap.cps.ncmp.events.lcm.v1.LcmEventHeader
+import org.onap.cps.utils.JsonObjectMapper
import org.springframework.kafka.KafkaException
import spock.lang.Specification
class LcmEventsServiceSpec extends Specification {
def mockLcmEventsPublisher = Mock(EventsPublisher)
+ def mockJsonObjectMapper = Mock(JsonObjectMapper)
- def objectUnderTest = new LcmEventsService(mockLcmEventsPublisher)
+ def objectUnderTest = new LcmEventsService(mockLcmEventsPublisher, mockJsonObjectMapper)
def 'Create and Publish lcm event where events are #scenario'() {
given: 'a cm handle id and Lcm Event'
def cmHandleId = 'test-cm-handle-id'
- def lcmEvent = new LcmEvent(eventId: UUID.randomUUID().toString(), eventCorrelationId: cmHandleId)
+ def eventId = UUID.randomUUID().toString()
+ def lcmEvent = new LcmEvent(eventId: eventId, eventCorrelationId: cmHandleId)
+ and: 'we also have a lcm event header'
+ def lcmEventHeader = new LcmEventHeader(eventId: eventId, eventCorrelationId: cmHandleId)
and: 'notificationsEnabled is #notificationsEnabled and it will be true as default'
objectUnderTest.notificationsEnabled = notificationsEnabled
+ and: 'lcm event header is transformed to headers map'
+ mockJsonObjectMapper.convertToValueType(lcmEventHeader, Map.class) >> ['eventId': eventId, 'eventCorrelationId': cmHandleId]
when: 'service is called to publish lcm event'
- objectUnderTest.publishLcmEvent('test-cm-handle-id', lcmEvent)
+ objectUnderTest.publishLcmEvent('test-cm-handle-id', lcmEvent, lcmEventHeader)
then: 'publisher is called #expectedTimesMethodCalled times'
- expectedTimesMethodCalled * mockLcmEventsPublisher.publishEvent(_, cmHandleId, lcmEvent)
+ expectedTimesMethodCalled * mockLcmEventsPublisher.publishEvent(_, cmHandleId, _, lcmEvent) >> {
+ args -> {
+ def eventHeaders = (args[2] as Map<String,Object>)
+ assert eventHeaders.containsKey('eventId')
+ assert eventHeaders.containsKey('eventCorrelationId')
+ assert eventHeaders.get('eventId') == eventId
+ assert eventHeaders.get('eventCorrelationId') == cmHandleId
+ }
+ }
where: 'the following values are used'
scenario | notificationsEnabled || expectedTimesMethodCalled
'enabled' | true || 1
@@ -50,12 +66,14 @@ class LcmEventsServiceSpec extends Specification {
def 'Unable to send message'(){
given: 'a cm handle id and Lcm Event and notification enabled'
def cmHandleId = 'test-cm-handle-id'
- def lcmEvent = new LcmEvent(eventId: UUID.randomUUID().toString(), eventCorrelationId: cmHandleId)
+ def eventId = UUID.randomUUID().toString()
+ def lcmEvent = new LcmEvent(eventId: eventId, eventCorrelationId: cmHandleId)
+ def lcmEventHeader = new LcmEventHeader(eventId: eventId, eventCorrelationId: cmHandleId)
objectUnderTest.notificationsEnabled = true
when: 'publisher set to throw an exception'
- mockLcmEventsPublisher.publishEvent(*_) >> { throw new KafkaException('publishing failed')}
+ mockLcmEventsPublisher.publishEvent(_, _, _, _) >> { throw new KafkaException('publishing failed')}
and: 'an event is publised'
- objectUnderTest.publishLcmEvent(cmHandleId, lcmEvent)
+ objectUnderTest.publishLcmEvent(cmHandleId, lcmEvent, lcmEventHeader)
then: 'the exception is just logged and not bubbled up'
noExceptionThrown()
}
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/operations/DmiDataOperationsSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/operations/DmiDataOperationsSpec.groovy
index 5fd4fbd43f..9343666260 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/operations/DmiDataOperationsSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/operations/DmiDataOperationsSpec.groovy
@@ -24,6 +24,8 @@ package org.onap.cps.ncmp.api.impl.operations
import com.fasterxml.jackson.databind.ObjectMapper
import org.onap.cps.ncmp.api.impl.config.NcmpConfiguration
import org.onap.cps.ncmp.api.impl.utils.DmiServiceUrlBuilder
+import org.onap.cps.ncmp.api.models.ResourceDataBatchRequest
+import org.onap.cps.ncmp.utils.TestUtils
import org.onap.cps.utils.JsonObjectMapper
import org.spockframework.spring.SpringBean
import org.springframework.beans.factory.annotation.Autowired
@@ -35,9 +37,9 @@ import spock.lang.Shared
import static org.onap.cps.ncmp.api.impl.operations.DatastoreType.PASSTHROUGH_OPERATIONAL
import static org.onap.cps.ncmp.api.impl.operations.DatastoreType.PASSTHROUGH_RUNNING
-import static org.onap.cps.ncmp.api.impl.operations.OperationEnum.CREATE
-import static org.onap.cps.ncmp.api.impl.operations.OperationEnum.READ
-import static org.onap.cps.ncmp.api.impl.operations.OperationEnum.UPDATE
+import static org.onap.cps.ncmp.api.impl.operations.OperationType.CREATE
+import static org.onap.cps.ncmp.api.impl.operations.OperationType.READ
+import static org.onap.cps.ncmp.api.impl.operations.OperationType.UPDATE
@SpringBootTest
@ContextConfiguration(classes = [NcmpConfiguration.DmiProperties, DmiDataOperations])
@@ -50,8 +52,6 @@ class DmiDataOperationsSpec extends DmiOperationsBaseSpec {
def NO_REQUEST_ID = null
@Shared
def OPTIONS_PARAM = '(a=1,b=2)'
- @Shared
- def expectedBulkRequestAsJson = '{"operation": "read","data": {"fe1c1f1a070c4ce5bbfda7198592a0d3": {"neType": "RadioNode"},"b8e42eed0d9541ed8d8839e8eb86b3e0": {"neType": "RadioNode"}},"requestId": "bbb67474-f705-410a-93d1-b2844e7f45fd"}'
@SpringBean
JsonObjectMapper spiedJsonObjectMapper = Spy(new JsonObjectMapper(new ObjectMapper()))
@@ -82,23 +82,26 @@ class DmiDataOperationsSpec extends DmiOperationsBaseSpec {
'datastore running with properties' | [yangModelCmHandleProperty] | PASSTHROUGH_RUNNING | OPTIONS_PARAM || '{"operation":"read","cmHandleProperties":{"prop1":"val1"}}' | 'passthrough-running' | '&options=(a=1,b=2)'
}
- def 'call get bulk resource data for #dataStore from DMI service with topic #scenario.'() {
- given: 'collection of yang model cm Handles'
+ def 'call get batch resource data from DMI service #scenario.'() {
+ given: 'collection of yang model cm Handles and resource data batch request'
mockYangModelCmHandleCollectionRetrieval([yangModelCmHandleProperty])
- and: 'a positive response from DMI service when it is called with the expected parameters'
+ def resourceDataBatchRequestJsonData = TestUtils.getResourceFileContent('resourceDataBatchRequest.json')
+ def resourceDataBatchRequest = spiedJsonObjectMapper.convertJsonString(resourceDataBatchRequestJsonData, ResourceDataBatchRequest.class)
+ resourceDataBatchRequest.batchOperationDefinitions[0].cmHandleIds = [cmHandleId]
+ def requestBodyAsJsonStringArg = null
+ and: 'a positive response from DMI service when it is called with valid request parameters'
def responseFromDmi = new ResponseEntity<Object>(HttpStatus.ACCEPTED)
- def expectedDmiBulkResourceDataUrl = "ncmp/v1/batch/data/ds/${dataStore}?resourceIdentifier=parent/child%26options=(a=1,b=2)&topic=my-topic-name&options=(fields=schemas/schema)"
- mockDmiRestClient.postOperationWithJsonData(expectedDmiBulkResourceDataUrl, expectedBulkRequestAsJson, READ) >> responseFromDmi
- dmiServiceUrlBuilder.getBulkRequestUrl(_, _) >> expectedDmiBulkResourceDataUrl
- when: 'get resource data for bulk cm handle is invoked'
- def result = objectUnderTest.getResourceDataFromDmi( dataStore.datastoreName, [cmHandleId], resourceIdentifier,
- OPTIONS_PARAM, 'some-topic','requestId')
- then: 'the result is the response from the DMI service'
- assert result == responseFromDmi
- where: 'the following parameters are used'
- scenario | dataStore
- 'datastore operational' | PASSTHROUGH_OPERATIONAL
- 'datastore running' | PASSTHROUGH_RUNNING
+ def expectedDmiBatchResourceDataUrl = "ncmp/v1/data/topic=my-topic-name"
+ def expectedBatchRequestAsJson = '[{"operation":"read","operationId":"operational-14","datastore":"ncmp-datastore:passthrough-operational","options":"some option","resourceIdentifier":"some resource identifier","cmHandles":[{"id":"some-cm-handle","cmHandleProperties":{"prop1":"val1"}}]}]'
+ mockDmiRestClient.postOperationWithJsonData(expectedDmiBatchResourceDataUrl, _, READ.operationName) >> responseFromDmi
+ dmiServiceUrlBuilder.getBatchRequestUrl(_, _) >> expectedDmiBatchResourceDataUrl
+ when: 'get resource data for batch of cm handles are invoked'
+ objectUnderTest.requestResourceDataFromDmi('my-topic-name', resourceDataBatchRequest, 'requestId')
+ then: 'wait a little to allow execution of service method by task executor (on separate thread)'
+ Thread.sleep(100)
+ then: 'validate ncmp generated dmi request body json args'
+ 1 * mockDmiRestClient.postOperationWithJsonData(expectedDmiBatchResourceDataUrl, _, READ) >> { args -> requestBodyAsJsonStringArg = args[1] }
+ assert requestBodyAsJsonStringArg == expectedBatchRequestAsJson
}
def 'call get all resource data.'() {
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/operations/DmiModelOperationsSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/operations/DmiModelOperationsSpec.groovy
index ed74ad3342..d1025f9d65 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/operations/DmiModelOperationsSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/operations/DmiModelOperationsSpec.groovy
@@ -36,7 +36,7 @@ import org.springframework.http.ResponseEntity
import org.springframework.test.context.ContextConfiguration
import spock.lang.Shared
-import static org.onap.cps.ncmp.api.impl.operations.OperationEnum.READ
+import static org.onap.cps.ncmp.api.impl.operations.OperationType.READ
@SpringBootTest
@ContextConfiguration(classes = [NcmpConfiguration.DmiProperties, DmiModelOperations])
@@ -103,9 +103,9 @@ class DmiModelOperationsSpec extends DmiOperationsBaseSpec {
then: 'the result is the response from DMI service'
assert result == []
where: 'the following DMI properties are used'
- scenario | dmiProperties || expectedAdditionalPropertiesInRequest
- 'with properties' | [yangModelCmHandleProperty] || '{"prop1":"val1"}'
- 'without properties' | [] || '{}'
+ scenario | dmiProperties || expectedAdditionalPropertiesInRequest
+ 'with properties' | [yangModelCmHandleProperty] || '{"prop1":"val1"}'
+ 'without properties' | [] || '{}'
}
def 'Retrieving yang resources.'() {
@@ -154,10 +154,10 @@ class DmiModelOperationsSpec extends DmiOperationsBaseSpec {
then: 'the result is the response from DMI service'
assert result == [mod1:'some yang source']
where: 'the following DMI properties are used'
- scenario | dmiProperties | unknownModuleReferences || expectedAdditionalPropertiesInRequest | expectedModuleReferencesInRequest
- 'with module references and properties' | [yangModelCmHandleProperty] | newModuleReferences || '{"prop1":"val1"}' | '{"name":"mod1","revision":"A"},{"name":"mod2","revision":"X"}'
- 'without module references' | [yangModelCmHandleProperty] | [] || '{"prop1":"val1"}' | ''
- 'without properties' | [] | newModuleReferences || '{}' | '{"name":"mod1","revision":"A"},{"name":"mod2","revision":"X"}'
+ scenario | dmiProperties | unknownModuleReferences || expectedAdditionalPropertiesInRequest | expectedModuleReferencesInRequest
+ 'with module references and properties' | [yangModelCmHandleProperty] | newModuleReferences || '{"prop1":"val1"}' | '{"name":"mod1","revision":"A"},{"name":"mod2","revision":"X"}'
+ 'without module references' | [yangModelCmHandleProperty] | [] || '{"prop1":"val1"}' | ''
+ 'without properties' | [] | newModuleReferences || '{}' | '{"name":"mod1","revision":"A"},{"name":"mod2","revision":"X"}'
}
def 'Retrieving yang resources from DMI with null DMI properties.'() {
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/ResourceDataBatchRequestUtilsSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/ResourceDataBatchRequestUtilsSpec.groovy
new file mode 100644
index 0000000000..e65874930b
--- /dev/null
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/ResourceDataBatchRequestUtilsSpec.groovy
@@ -0,0 +1,80 @@
+/*
+ * ============LICENSE_START=======================================================
+ * Copyright (C) 2023 Nordix Foundation
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ * SPDX-License-Identifier: Apache-2.0
+ * ============LICENSE_END=========================================================
+ */
+
+package org.onap.cps.ncmp.api.impl.utils
+
+import com.fasterxml.jackson.databind.ObjectMapper
+import org.onap.cps.ncmp.api.impl.yangmodels.YangModelCmHandle
+import org.onap.cps.ncmp.api.inventory.CmHandleState
+import org.onap.cps.ncmp.api.inventory.CompositeStateBuilder
+import org.onap.cps.ncmp.api.models.ResourceDataBatchRequest
+import org.onap.cps.ncmp.utils.TestUtils
+import org.onap.cps.utils.JsonObjectMapper
+import org.spockframework.spring.SpringBean
+import spock.lang.Specification
+
+class ResourceDataBatchRequestUtilsSpec extends Specification {
+
+ @SpringBean
+ JsonObjectMapper jsonObjectMapper = new JsonObjectMapper(new ObjectMapper())
+
+ def 'Process per operation in batch request with #serviceName.'() {
+ given: 'batch request with 3 operations'
+ def resourceDataBatchRequestJsonData = TestUtils.getResourceFileContent('resourceDataBatchRequest.json')
+ def resourceDataBatchRequest = jsonObjectMapper.convertJsonString(resourceDataBatchRequestJsonData, ResourceDataBatchRequest.class)
+ and: '4 known cm handles: ch1-dmi1, ch2-dmi1, ch3-dmi2, ch4-dmi2'
+ def yangModelCmHandles = getYangModelCmHandles()
+ when: 'Operation in batch request is processed'
+ def operationsOutPerDmiServiceName = ResourceDataBatchRequestUtils.processPerOperationInBatchRequest(resourceDataBatchRequest, yangModelCmHandles)
+ and: 'converted to a json node'
+ def dmiBatchRequestBody = jsonObjectMapper.asJsonString(operationsOutPerDmiServiceName.get(serviceName))
+ def dmiBatchRequestBodyAsJsonNode = jsonObjectMapper.convertToJsonNode(dmiBatchRequestBody).get(operationIndex)
+ then: 'it contains the correct operation details'
+ assert dmiBatchRequestBodyAsJsonNode.get('operation').asText() == 'read'
+ assert dmiBatchRequestBodyAsJsonNode.get('operationId').asText() == expectedOperationId
+ assert dmiBatchRequestBodyAsJsonNode.get('datastore').asText() == expectedDatastore
+ and: 'the correct cm handles (just for #serviceName)'
+ assert dmiBatchRequestBodyAsJsonNode.get('cmHandles').size() == expectedCmHandleIds.size()
+ expectedCmHandleIds.each {
+ dmiBatchRequestBodyAsJsonNode.get('cmHandles').toString().contains(it)
+ }
+ where: 'the following dmi service and operations are checked'
+ serviceName | operationIndex || expectedOperationId | expectedDatastore | expectedCmHandleIds
+ 'dmi1' | 0 || 'operational-14' | 'ncmp-datastore:passthrough-operational' | ['ch6-dmi1']
+ 'dmi1' | 1 || 'running-12' | 'ncmp-datastore:passthrough-running' | ['ch1-dmi1', 'ch2-dmi1']
+ 'dmi1' | 2 || 'operational-15' | 'ncmp-datastore:passthrough-operational' | ['ch6-dmi1']
+ 'dmi2' | 0 || 'operational-14' | 'ncmp-datastore:passthrough-operational' | ['ch3-dmi2']
+ 'dmi2' | 1 || 'running-12' | 'ncmp-datastore:passthrough-running' | ['ch7-dmi2']
+ 'dmi2' | 2 || 'operational-15' | 'ncmp-datastore:passthrough-operational' | ['ch4-dmi2']
+ }
+
+ static def getYangModelCmHandles() {
+ def dmiProperties = [new YangModelCmHandle.Property('prop', 'some DMI property')]
+ def readyState = new CompositeStateBuilder().withCmHandleState(CmHandleState.READY).withLastUpdatedTimeNow().build()
+ return [new YangModelCmHandle(id: 'ch1-dmi1', dmiServiceName: 'dmi1', dmiProperties: dmiProperties, compositeState: readyState),
+ new YangModelCmHandle(id: 'ch2-dmi1', dmiServiceName: 'dmi1', dmiProperties: dmiProperties, compositeState: readyState),
+ new YangModelCmHandle(id: 'ch6-dmi1', dmiServiceName: 'dmi1', dmiProperties: dmiProperties, compositeState: readyState),
+ new YangModelCmHandle(id: 'ch8-dmi1', dmiServiceName: 'dmi1', dmiProperties: dmiProperties, compositeState: readyState),
+ new YangModelCmHandle(id: 'ch3-dmi2', dmiServiceName: 'dmi2', dmiProperties: dmiProperties, compositeState: readyState),
+ new YangModelCmHandle(id: 'ch4-dmi2', dmiServiceName: 'dmi2', dmiProperties: dmiProperties, compositeState: readyState),
+ new YangModelCmHandle(id: 'ch7-dmi2', dmiServiceName: 'dmi2', dmiProperties: dmiProperties, compositeState: readyState),
+ ]
+ }
+}
diff --git a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/RestQueryParametersValidatorSpec.groovy b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/RestQueryParametersValidatorSpec.groovy
index e1055bb217..dc471e64fa 100644
--- a/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/RestQueryParametersValidatorSpec.groovy
+++ b/cps-ncmp-service/src/test/groovy/org/onap/cps/ncmp/api/impl/utils/RestQueryParametersValidatorSpec.groovy
@@ -64,13 +64,13 @@ class RestQueryParametersValidatorSpec extends Specification {
and: 'the exception details contain the correct significant term '
thrown.details.contains(expectedWordInDetails)
where:
- scenario | conditionName | conditionParameters || expectedWordInDetails
- 'unknown condition name' | 'unknownCondition' | [['key':'value']] || 'conditionName'
- 'no condition name' | '' | [['key':'value']] || 'conditionName'
- 'empty properties' | 'validConditionName' | [[ : ]] || 'conditionsParameter'
- 'empty conditions' | 'validConditionName' | [[:]] || 'conditionsParameter'
- 'too many properties' | 'validConditionName' | [[key1:'value1', key2:'value2']] || 'conditionsParameter'
- 'empty key' | 'validConditionName' | [['':'wrong']] || 'conditionsParameter'
+ scenario | conditionName | conditionParameters || expectedWordInDetails
+ 'unknown condition name' | 'unknownCondition' | [['key': 'value']] || 'conditionName'
+ 'no condition name' | '' | [['key': 'value']] || 'conditionName'
+ 'empty properties' | 'validConditionName' | [[:]] || 'conditionsParameter'
+ 'empty conditions' | 'validConditionName' | [[:]] || 'conditionsParameter'
+ 'too many properties' | 'validConditionName' | [[key1: 'value1', key2: 'value2']] || 'conditionsParameter'
+ 'empty key' | 'validConditionName' | [['': 'wrong']] || 'conditionsParameter'
}
def 'CM Handle Query validation: validate module name condition properties - valid query.'() {