aboutsummaryrefslogtreecommitdiffstats
path: root/sources/hv-collector-core/src/test
diff options
context:
space:
mode:
authorFilip Krzywka <filip.krzywka@nokia.com>2019-02-28 17:33:02 +0100
committerFilip Krzywka <filip.krzywka@nokia.com>2019-03-11 14:51:58 +0100
commit1422bed4c1a002e663fd7c2c4c204831e5f7aa9a (patch)
tree18247d0af6cb8aa615934969c24249227e07d9a2 /sources/hv-collector-core/src/test
parentcc581b7f4833fe47ad3eb846d432ca0ffa1c66f4 (diff)
Use CBS by means of SDK in place of Consul
- changed IO creation in main to fix error with too early calling changeState method on collector HealthState - increased coverage a little with few tests - corrected coverage-report pom file to include new modules - temporarily changed to 1.1.4-SNAPSHOT version of sdk due to need of new version of CBSLookup Change-Id: Ic73b46cf881ab4fabf52bef0327b09082aa90dc6 Issue-ID: DCAEGEN2-1302 Signed-off-by: Filip Krzywka <filip.krzywka@nokia.com>
Diffstat (limited to 'sources/hv-collector-core/src/test')
-rw-r--r--sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/ConfigurationProviderTest.kt (renamed from sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/ConsulConfigurationProviderTest.kt)112
-rw-r--r--sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/ProtobufSerializerTest.kt65
-rw-r--r--sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/VesMessageSerializerTest.kt66
3 files changed, 198 insertions, 45 deletions
diff --git a/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/ConsulConfigurationProviderTest.kt b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/ConfigurationProviderTest.kt
index ccae3c99..21aaa129 100644
--- a/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/ConsulConfigurationProviderTest.kt
+++ b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/ConfigurationProviderTest.kt
@@ -2,7 +2,7 @@
* ============LICENSE_START=======================================================
* dcaegen2-collectors-veshv
* ================================================================================
- * Copyright (C) 2018 NOKIA
+ * Copyright (C) 2018-2019 NOKIA
* ================================================================================
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -19,6 +19,7 @@
*/
package org.onap.dcae.collectors.veshv.impl.adapters
+import com.google.gson.JsonParser
import com.nhaarman.mockitokotlin2.any
import com.nhaarman.mockitokotlin2.eq
import com.nhaarman.mockitokotlin2.mock
@@ -29,11 +30,12 @@ import org.jetbrains.spek.api.dsl.describe
import org.jetbrains.spek.api.dsl.given
import org.jetbrains.spek.api.dsl.it
import org.jetbrains.spek.api.dsl.on
-import org.mockito.Mockito
import org.onap.dcae.collectors.veshv.domain.VesEventDomain.HEARTBEAT
import org.onap.dcae.collectors.veshv.domain.VesEventDomain.FAULT
import org.onap.dcae.collectors.veshv.healthcheck.api.HealthDescription
import org.onap.dcae.collectors.veshv.healthcheck.api.HealthState
+import org.onap.dcaegen2.services.sdk.rest.services.cbs.client.api.CbsClient
+import reactor.core.publisher.Flux
import reactor.core.publisher.Mono
import reactor.retry.Retry
@@ -44,24 +46,36 @@ import java.time.Duration
* @author Jakub Dudycz <jakub.dudycz@nokia.com>
* @since May 2018
*/
-internal object ConsulConfigurationProviderTest : Spek({
+internal object ConfigurationProviderImplTest : Spek({
- describe("Consul configuration provider") {
+ describe("Configuration provider") {
- val httpAdapterMock: HttpAdapter = mock()
+ val cbsClient: CbsClient = mock()
+ val cbsClientMock: Mono<CbsClient> = Mono.just(cbsClient)
val healthStateProvider = HealthState.INSTANCE
- given("valid resource url") {
- val validUrl = "http://valid-url/"
- val consulConfigProvider = constructConsulConfigProvider(validUrl, httpAdapterMock, healthStateProvider)
+ given("configuration is never in cbs") {
+ val configProvider = constructConfigurationProvider(cbsClientMock, healthStateProvider)
- on("call to consul") {
- whenever(httpAdapterMock.get(eq(validUrl), any(), Mockito.anyMap()))
- .thenReturn(Mono.just(constructConsulResponse()))
+ on("waiting for configuration") {
+ val waitTime = Duration.ofMillis(100)
+ it("should not get it") {
+ StepVerifier.create(configProvider().take(1))
+ .expectNoEvent(waitTime)
+ }
+ }
+
+ }
+ given("valid configuration from cbs") {
+ val configProvider = constructConfigurationProvider(cbsClientMock, healthStateProvider)
+
+ on("new configuration") {
+ whenever(cbsClient.updates(any(), eq(firstRequestDelay), eq(requestInterval)))
+ .thenReturn(Flux.just(validConfiguration))
it("should use received configuration") {
- StepVerifier.create(consulConfigProvider().take(1))
+ StepVerifier.create(configProvider().take(1))
.consumeNextWith {
val route1 = it.routing.routes[0]
@@ -85,22 +99,19 @@ internal object ConsulConfigurationProviderTest : Spek({
}
}
- given("invalid resource url") {
- val invalidUrl = "http://invalid-url/"
-
+ given("invalid configuration from cbs") {
val iterationCount = 3L
- val consulConfigProvider = constructConsulConfigProvider(
- invalidUrl, httpAdapterMock, healthStateProvider, iterationCount
+ val configProvider = constructConfigurationProvider(
+ cbsClientMock, healthStateProvider, iterationCount
)
- on("call to consul") {
- whenever(httpAdapterMock.get(eq(invalidUrl), any(), Mockito.anyMap()))
- .thenReturn(Mono.error(RuntimeException("Test exception")))
+ on("new configuration") {
+ whenever(cbsClient.updates(any(), eq(firstRequestDelay), eq(requestInterval)))
+ .thenReturn(Flux.just(invalidConfiguration))
it("should interrupt the flux") {
-
- StepVerifier.create(consulConfigProvider())
- .verifyErrorMessage("Test exception")
+ StepVerifier.create(configProvider())
+ .verifyError()
}
it("should update the health state") {
@@ -115,28 +126,9 @@ internal object ConsulConfigurationProviderTest : Spek({
})
-private fun constructConsulConfigProvider(url: String,
- httpAdapter: HttpAdapter,
- healthState: HealthState,
- iterationCount: Long = 1
-): ConsulConfigurationProvider {
-
- val firstRequestDelay = Duration.ofMillis(1)
- val requestInterval = Duration.ofMillis(1)
- val retry = Retry.onlyIf<Any> { it.iteration() <= iterationCount }.fixedBackoff(Duration.ofNanos(1))
- return ConsulConfigurationProvider(
- httpAdapter,
- url,
- firstRequestDelay,
- requestInterval,
- healthState,
- retry
- )
-}
-
-fun constructConsulResponse(): String =
- """{
+private val validConfiguration = JsonParser().parse("""
+{
"whatever": "garbage",
"collector.routing": [
{
@@ -148,4 +140,34 @@ fun constructConsulResponse(): String =
"toTopic": "test-topic-2"
}
]
- }"""
+}""").asJsonObject
+
+private val invalidConfiguration = JsonParser().parse("""
+{
+ "whatever": "garbage",
+ "collector.routing": [
+ {
+ "fromDomain": "garbage",
+ "meaningful": "garbage"
+ }
+ ]
+}""").asJsonObject
+
+private val firstRequestDelay = Duration.ofMillis(1)
+private val requestInterval = Duration.ofMillis(1)
+
+private fun constructConfigurationProvider(cbsClientMono: Mono<CbsClient>,
+ healthState: HealthState,
+ iterationCount: Long = 1
+): ConfigurationProviderImpl {
+
+ val retry = Retry.onlyIf<Any> { it.iteration() <= iterationCount }.fixedBackoff(Duration.ofNanos(1))
+
+ return ConfigurationProviderImpl(
+ cbsClientMono,
+ firstRequestDelay,
+ requestInterval,
+ healthState,
+ retry
+ )
+}
diff --git a/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/ProtobufSerializerTest.kt b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/ProtobufSerializerTest.kt
new file mode 100644
index 00000000..63caaf0a
--- /dev/null
+++ b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/ProtobufSerializerTest.kt
@@ -0,0 +1,65 @@
+/*
+ * ============LICENSE_START=======================================================
+ * dcaegen2-collectors-veshv
+ * ================================================================================
+ * Copyright (C) 2019 NOKIA
+ * ================================================================================
+ * 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.
+ * ============LICENSE_END=========================================================
+ */
+package org.onap.dcae.collectors.veshv.impl.adapters.kafka
+
+import com.google.protobuf.MessageLite
+import com.nhaarman.mockitokotlin2.mock
+import com.nhaarman.mockitokotlin2.verify
+import org.jetbrains.spek.api.Spek
+import org.jetbrains.spek.api.dsl.describe
+import org.jetbrains.spek.api.dsl.it
+import org.jetbrains.spek.api.dsl.on
+import org.onap.ves.VesEventOuterClass.CommonEventHeader.*
+
+
+class ProtobufSerializerTest : Spek({
+
+ describe("ProtobufSerializerTest") {
+ val serializer = ProtobufSerializer()
+
+ on("serialize") {
+ it("should return byte array from WTP Frame paylaod") {
+ val header = getDefaultInstance()
+ val payload = header.toByteArray()
+ val msg: MessageLite = mock()
+
+ serializer.serialize("", msg)
+
+ verify(msg).toByteArray()
+ }
+ }
+
+ on("configuring") {
+ it("should do nothing") {
+ // increase code coverage
+ serializer.configure(hashMapOf<String, String>(), false)
+ }
+ }
+
+ on("closing") {
+ it("should do nothing") {
+ // increase code coverage
+ serializer.close()
+ }
+ }
+ }
+
+
+}) \ No newline at end of file
diff --git a/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/VesMessageSerializerTest.kt b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/VesMessageSerializerTest.kt
new file mode 100644
index 00000000..3a194b47
--- /dev/null
+++ b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/VesMessageSerializerTest.kt
@@ -0,0 +1,66 @@
+/*
+ * ============LICENSE_START=======================================================
+ * dcaegen2-collectors-veshv
+ * ================================================================================
+ * Copyright (C) 2019 NOKIA
+ * ================================================================================
+ * 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.
+ * ============LICENSE_END=========================================================
+ */
+package org.onap.dcae.collectors.veshv.impl.adapters.kafka
+
+import org.assertj.core.api.Assertions.assertThat
+import org.jetbrains.spek.api.Spek
+import org.jetbrains.spek.api.dsl.describe
+import org.jetbrains.spek.api.dsl.it
+import org.jetbrains.spek.api.dsl.on
+import org.onap.dcae.collectors.veshv.domain.WireFrameMessage
+import org.onap.dcae.collectors.veshv.model.VesMessage
+import org.onap.ves.VesEventOuterClass.CommonEventHeader.*
+
+
+class VesMessageSerializerTest : Spek({
+
+ describe("VesMessageSerializer") {
+ val serializer = VesMessageSerializer()
+
+ on("serialize") {
+ it("should return byte array from WTP Frame paylaod") {
+ val header = getDefaultInstance()
+ val payload = header.toByteArray()
+ val msg = VesMessage(header, WireFrameMessage(payload))
+
+ val serialized = serializer.serialize("", msg)
+
+ assertThat(serialized).isEqualTo(payload)
+ }
+ }
+
+ on("configuring") {
+ it("should do nothing") {
+ // increase code coverage
+ serializer.configure(hashMapOf<String, String>(), false)
+ }
+ }
+
+ on("closing") {
+ it("should do nothing") {
+ // increase code coverage
+ serializer.close()
+ }
+ }
+ }
+
+
+
+}) \ No newline at end of file