diff options
author | Piotr Jaszczyk <piotr.jaszczyk@nokia.com> | 2018-11-28 15:46:50 +0100 |
---|---|---|
committer | Piotr Jaszczyk <piotr.jaszczyk@nokia.com> | 2018-11-29 14:41:42 +0100 |
commit | dde383a2aa75f94c26d7949665b79cc95486a223 (patch) | |
tree | 75f3e8f564067afd0e67dbe6254183e45ca26944 /sources | |
parent | 77f896523f2065b1da1be21545155a29edea5122 (diff) |
Custom detekt rule for logger usage check
Check if logger invocations don't use unoptimal invocations, eg.
concatenation `debug("a=" + a)` instead of lambda use `debug {"a=" + a}`
Unfortunately to avoid defining dependencies in many places and having
circural dependencies it was necessarry to reorganize the maven module
structure. The goal was to have `sources` module with production code and
`build` module with build-time tooling (detekt rules among them).
Issue-ID: DCAEGEN2-1002
Change-Id: I36e677b98972aaae6905d722597cbce5e863d201
Signed-off-by: Piotr Jaszczyk <piotr.jaszczyk@nokia.com>
Diffstat (limited to 'sources')
157 files changed, 11901 insertions, 0 deletions
diff --git a/sources/hv-collector-core/pom.xml b/sources/hv-collector-core/pom.xml new file mode 100644 index 00000000..7f7922e1 --- /dev/null +++ b/sources/hv-collector-core/pom.xml @@ -0,0 +1,137 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ ============LICENSE_START======================================================= + ~ dcaegen2-collectors-veshv + ~ ================================================================================ + ~ Copyright (C) 2018 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========================================================= + --> +<project xmlns="http://maven.apache.org/POM/4.0.0" + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + + <licenses> + <license> + <name>The Apache Software License, Version 2.0</name> + <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url> + </license> + </licenses> + + <parent> + <groupId>org.onap.dcaegen2.collectors.hv-ves</groupId> + <artifactId>hv-collector-sources</artifactId> + <version>1.1.0-SNAPSHOT</version> + <relativePath>..</relativePath> + </parent> + + <artifactId>hv-collector-core</artifactId> + <description>VES HighVolume Collector :: Core</description> + + <properties> + <skipAnalysis>false</skipAnalysis> + </properties> + + <build> + <plugins> + <plugin> + <artifactId>kotlin-maven-plugin</artifactId> + <groupId>org.jetbrains.kotlin</groupId> + </plugin> + <plugin> + <artifactId>maven-surefire-plugin</artifactId> + <groupId>org.apache.maven.plugins</groupId> + </plugin> + <plugin> + <groupId>org.jacoco</groupId> + <artifactId>jacoco-maven-plugin</artifactId> + </plugin> + </plugins> + </build> + + <dependencies> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-ssl</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-utils</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-health-check</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-domain</artifactId> + <version>${project.parent.version}</version> + <scope>compile</scope> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-test-utils</artifactId> + <version>${project.parent.version}</version> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.jetbrains.kotlin</groupId> + <artifactId>kotlin-reflect</artifactId> + </dependency> + <dependency> + <groupId>io.arrow-kt</groupId> + <artifactId>arrow-effects</artifactId> + </dependency> + <dependency> + <groupId>io.arrow-kt</groupId> + <artifactId>arrow-core</artifactId> + </dependency> + <dependency> + <groupId>io.projectreactor</groupId> + <artifactId>reactor-core</artifactId> + </dependency> + <dependency> + <groupId>io.projectreactor.addons</groupId> + <artifactId>reactor-extra</artifactId> + </dependency> + <dependency> + <groupId>io.projectreactor.netty</groupId> + <artifactId>reactor-netty</artifactId> + </dependency> + <dependency> + <groupId>io.projectreactor.kafka</groupId> + <artifactId>reactor-kafka</artifactId> + </dependency> + <dependency> + <groupId>javax.json</groupId> + <artifactId>javax.json-api</artifactId> + </dependency> + <dependency> + <groupId>org.glassfish</groupId> + <artifactId>javax.json</artifactId> + <scope>runtime</scope> + </dependency> + <dependency> + <groupId>ch.qos.logback</groupId> + <artifactId>logback-classic</artifactId> + <scope>test</scope> + </dependency> + </dependencies> + +</project> diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/boundary/adapters.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/boundary/adapters.kt new file mode 100644 index 00000000..dd0111bc --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/boundary/adapters.kt @@ -0,0 +1,50 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.boundary + +import org.onap.dcae.collectors.veshv.model.CollectorConfiguration +import org.onap.dcae.collectors.veshv.model.RoutedMessage +import reactor.core.publisher.Flux + +interface Sink { + fun send(messages: Flux<RoutedMessage>): Flux<RoutedMessage> +} + +interface Metrics { + fun notifyBytesReceived(size: Int) + fun notifyMessageReceived(size: Int) + fun notifyMessageSent(topic: String) +} + +@FunctionalInterface +interface SinkProvider { + operator fun invoke(config: CollectorConfiguration): Sink + + companion object { + fun just(sink: Sink): SinkProvider = + object : SinkProvider { + override fun invoke(config: CollectorConfiguration): Sink = sink + } + } +} + +interface ConfigurationProvider { + operator fun invoke(): Flux<CollectorConfiguration> +} diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/boundary/api.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/boundary/api.kt new file mode 100644 index 00000000..3c85a9b1 --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/boundary/api.kt @@ -0,0 +1,38 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.boundary + +import arrow.core.Option +import arrow.effects.IO +import io.netty.buffer.ByteBuf +import io.netty.buffer.ByteBufAllocator +import org.onap.dcae.collectors.veshv.utils.ServerHandle +import reactor.core.publisher.Flux +import reactor.core.publisher.Mono + +interface Collector { + fun handleConnection(alloc: ByteBufAllocator, dataStream: Flux<ByteBuf>): Mono<Void> +} + +typealias CollectorProvider = () -> Option<Collector> + +interface Server { + fun start(): IO<ServerHandle> +} diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/factory/CollectorFactory.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/factory/CollectorFactory.kt new file mode 100644 index 00000000..5c96e1c5 --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/factory/CollectorFactory.kt @@ -0,0 +1,80 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.factory + +import org.onap.dcae.collectors.veshv.boundary.Collector +import org.onap.dcae.collectors.veshv.boundary.CollectorProvider +import org.onap.dcae.collectors.veshv.boundary.ConfigurationProvider +import org.onap.dcae.collectors.veshv.boundary.Metrics +import org.onap.dcae.collectors.veshv.boundary.SinkProvider +import org.onap.dcae.collectors.veshv.domain.WireFrameDecoder +import org.onap.dcae.collectors.veshv.healthcheck.api.HealthState +import org.onap.dcae.collectors.veshv.healthcheck.api.HealthDescription +import org.onap.dcae.collectors.veshv.impl.Router +import org.onap.dcae.collectors.veshv.impl.VesDecoder +import org.onap.dcae.collectors.veshv.impl.VesHvCollector +import org.onap.dcae.collectors.veshv.impl.wire.WireChunkDecoder +import org.onap.dcae.collectors.veshv.model.CollectorConfiguration +import org.onap.dcae.collectors.veshv.utils.arrow.getOption +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import java.util.concurrent.atomic.AtomicReference + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +class CollectorFactory(val configuration: ConfigurationProvider, + private val sinkProvider: SinkProvider, + private val metrics: Metrics, + private val maximumPayloadSizeBytes: Int, + private val healthState: HealthState = HealthState.INSTANCE) { + + fun createVesHvCollectorProvider(): CollectorProvider { + val collector: AtomicReference<Collector> = AtomicReference() + configuration() + .map(this::createVesHvCollector) + .doOnNext { + logger.info("Using updated configuration for new connections") + healthState.changeState(HealthDescription.HEALTHY) + } + .doOnError { + logger.error("Failed to acquire configuration from consul") + healthState.changeState(HealthDescription.CONSUL_CONFIGURATION_NOT_FOUND) + } + .subscribe(collector::set) + return collector::getOption + } + + private fun createVesHvCollector(config: CollectorConfiguration): Collector { + return VesHvCollector( + wireChunkDecoderSupplier = { alloc -> + WireChunkDecoder(WireFrameDecoder(maximumPayloadSizeBytes), alloc) + }, + protobufDecoder = VesDecoder(), + router = Router(config.routing), + sink = sinkProvider(config), + metrics = metrics) + } + + companion object { + private val logger = Logger(CollectorFactory::class) + } +} + diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/factory/ServerFactory.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/factory/ServerFactory.kt new file mode 100644 index 00000000..dce933ab --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/factory/ServerFactory.kt @@ -0,0 +1,35 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.factory + +import org.onap.dcae.collectors.veshv.boundary.CollectorProvider +import org.onap.dcae.collectors.veshv.boundary.Server +import org.onap.dcae.collectors.veshv.impl.socket.NettyTcpServer +import org.onap.dcae.collectors.veshv.model.ServerConfiguration +import org.onap.dcae.collectors.veshv.ssl.boundary.ServerSslContextFactory + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +object ServerFactory { + fun createNettyTcpServer(serverConfiguration: ServerConfiguration, collectorProvider: CollectorProvider): Server = + NettyTcpServer(serverConfiguration, ServerSslContextFactory(), collectorProvider) +} diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/MessageValidator.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/MessageValidator.kt new file mode 100644 index 00000000..fb949079 --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/MessageValidator.kt @@ -0,0 +1,38 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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 + +import org.onap.dcae.collectors.veshv.domain.headerRequiredFieldDescriptors +import org.onap.dcae.collectors.veshv.domain.vesEventListenerVersionRegex +import org.onap.dcae.collectors.veshv.model.VesMessage +import org.onap.ves.VesEventOuterClass.CommonEventHeader + +internal object MessageValidator { + + fun isValid(message: VesMessage): Boolean { + return allMandatoryFieldsArePresent(message.header) + } + + private fun allMandatoryFieldsArePresent(header: CommonEventHeader) = + headerRequiredFieldDescriptors + .all { fieldDescriptor -> header.hasField(fieldDescriptor) } + .and(vesEventListenerVersionRegex.matches(header.vesEventListenerVersion)) + +} diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/Router.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/Router.kt new file mode 100644 index 00000000..cee658b6 --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/Router.kt @@ -0,0 +1,30 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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 + +import arrow.core.Option +import org.onap.dcae.collectors.veshv.model.RoutedMessage +import org.onap.dcae.collectors.veshv.model.Routing +import org.onap.dcae.collectors.veshv.model.VesMessage + +class Router(private val routing: Routing) { + fun findDestination(message: VesMessage): Option<RoutedMessage> = + routing.routeFor(message.header).map { it(message) } +} diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/VesDecoder.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/VesDecoder.kt new file mode 100644 index 00000000..1d43588f --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/VesDecoder.kt @@ -0,0 +1,39 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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 + +import arrow.core.Try +import arrow.core.Option +import org.onap.dcae.collectors.veshv.domain.ByteData +import org.onap.dcae.collectors.veshv.model.VesMessage +import org.onap.ves.VesEventOuterClass.VesEvent + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +internal class VesDecoder { + + fun decode(bytes: ByteData): Option<VesMessage> = + Try { + val decodedHeader = VesEvent.parseFrom(bytes.unsafeAsArray()).commonEventHeader + VesMessage(decodedHeader, bytes) + }.toOption() +} diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/VesHvCollector.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/VesHvCollector.kt new file mode 100644 index 00000000..2f12e0cd --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/VesHvCollector.kt @@ -0,0 +1,90 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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 + +import arrow.core.Option +import io.netty.buffer.ByteBuf +import io.netty.buffer.ByteBufAllocator +import org.onap.dcae.collectors.veshv.boundary.Collector +import org.onap.dcae.collectors.veshv.boundary.Metrics +import org.onap.dcae.collectors.veshv.boundary.Sink +import org.onap.dcae.collectors.veshv.domain.WireFrameMessage +import org.onap.dcae.collectors.veshv.impl.wire.WireChunkDecoder +import org.onap.dcae.collectors.veshv.model.RoutedMessage +import org.onap.dcae.collectors.veshv.model.VesMessage +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import org.onap.dcae.collectors.veshv.utils.logging.handleReactiveStreamError +import reactor.core.publisher.Flux +import reactor.core.publisher.Mono + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +internal class VesHvCollector( + private val wireChunkDecoderSupplier: (ByteBufAllocator) -> WireChunkDecoder, + private val protobufDecoder: VesDecoder, + private val router: Router, + private val sink: Sink, + private val metrics: Metrics) : Collector { + + override fun handleConnection(alloc: ByteBufAllocator, dataStream: Flux<ByteBuf>): Mono<Void> = + wireChunkDecoderSupplier(alloc).let { wireDecoder -> + dataStream + .transform { decodeWireFrame(it, wireDecoder) } + .filter(WireFrameMessage::isValid) + .transform(::decodePayload) + .filter(VesMessage::isValid) + .transform(::routeMessage) + .onErrorResume { logger.handleReactiveStreamError(it) } + .doFinally { releaseBuffersMemory(wireDecoder) } + .then() + } + + private fun decodeWireFrame(flux: Flux<ByteBuf>, decoder: WireChunkDecoder): Flux<WireFrameMessage> = flux + .doOnNext { metrics.notifyBytesReceived(it.readableBytes()) } + .concatMap(decoder::decode) + .doOnNext { metrics.notifyMessageReceived(it.payloadSize) } + + private fun decodePayload(flux: Flux<WireFrameMessage>): Flux<VesMessage> = flux + .map(WireFrameMessage::payload) + .map(protobufDecoder::decode) + .flatMap { omitWhenNone(it) } + + private fun routeMessage(flux: Flux<VesMessage>): Flux<RoutedMessage> = flux + .flatMap(this::findRoute) + .compose(sink::send) + .doOnNext { metrics.notifyMessageSent(it.topic) } + + + private fun findRoute(msg: VesMessage): Mono<RoutedMessage> = omitWhenNone((router::findDestination)(msg)) + + private fun <V> omitWhenNone(it: Option<V>): Mono<V> = it.fold( + { + logger.info("ommiting the message" + 5) + Mono.empty() }, + { Mono.just(it) }) + + private fun releaseBuffersMemory(wireChunkDecoder: WireChunkDecoder) = wireChunkDecoder.release() + + companion object { + private val logger = Logger(VesHvCollector::class) + } +} diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/AdapterFactory.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/AdapterFactory.kt new file mode 100644 index 00000000..8c16736d --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/AdapterFactory.kt @@ -0,0 +1,40 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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 + +import org.onap.dcae.collectors.veshv.boundary.ConfigurationProvider +import org.onap.dcae.collectors.veshv.boundary.SinkProvider +import org.onap.dcae.collectors.veshv.impl.adapters.kafka.KafkaSinkProvider +import org.onap.dcae.collectors.veshv.model.ConfigurationProviderParams +import reactor.netty.http.client.HttpClient + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +object AdapterFactory { + fun kafkaSink(): SinkProvider = KafkaSinkProvider() + fun loggingSink(): SinkProvider = LoggingSinkProvider() + + fun consulConfigurationProvider(configurationProviderParams: ConfigurationProviderParams): ConfigurationProvider = + ConsulConfigurationProvider(httpAdapter(), configurationProviderParams) + + private fun httpAdapter(): HttpAdapter = HttpAdapter(HttpClient.create()) +} diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/ConsulConfigurationProvider.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/ConsulConfigurationProvider.kt new file mode 100644 index 00000000..ec7c60c0 --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/ConsulConfigurationProvider.kt @@ -0,0 +1,122 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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 + +import org.onap.dcae.collectors.veshv.boundary.ConfigurationProvider +import org.onap.dcae.collectors.veshv.healthcheck.api.HealthDescription +import org.onap.dcae.collectors.veshv.healthcheck.api.HealthState +import org.onap.dcae.collectors.veshv.model.CollectorConfiguration +import org.onap.dcae.collectors.veshv.model.ConfigurationProviderParams +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import reactor.core.publisher.Flux +import reactor.core.publisher.Mono +import reactor.retry.Jitter +import reactor.retry.Retry +import java.io.StringReader +import java.time.Duration +import java.util.concurrent.atomic.AtomicReference +import javax.json.Json +import javax.json.JsonObject + + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since May 2018 + */ +internal class ConsulConfigurationProvider(private val http: HttpAdapter, + private val url: String, + private val firstRequestDelay: Duration, + private val requestInterval: Duration, + private val healthState: HealthState, + retrySpec: Retry<Any> + +) : ConfigurationProvider { + + private val lastConfigurationHash: AtomicReference<Int> = AtomicReference(0) + private val retry = retrySpec + .doOnRetry { + logger.warn("Could not get fresh configuration", it.exception()) + healthState.changeState(HealthDescription.RETRYING_FOR_CONSUL_CONFIGURATION) + } + + constructor(http: HttpAdapter, + params: ConfigurationProviderParams) : this( + http, + params.configurationUrl, + params.firstRequestDelay, + params.requestInterval, + HealthState.INSTANCE, + Retry.any<Any>() + .retryMax(MAX_RETRIES) + .fixedBackoff(params.requestInterval.dividedBy(BACKOFF_INTERVAL_FACTOR)) + .jitter(Jitter.random()) + ) + + override fun invoke(): Flux<CollectorConfiguration> = + Flux.interval(firstRequestDelay, requestInterval) + .concatMap { askForConfig() } + .flatMap(::filterDifferentValues) + .map(::parseJsonResponse) + .map(::createCollectorConfiguration) + .retryWhen(retry) + + private fun askForConfig(): Mono<String> = http.get(url) + + private fun filterDifferentValues(configurationString: String) = + hashOf(configurationString).let { + if (it == lastConfigurationHash.get()) { + Mono.empty() + } else { + lastConfigurationHash.set(it) + Mono.just(configurationString) + } + } + + private fun hashOf(str: String) = str.hashCode() + + private fun parseJsonResponse(responseString: String): JsonObject = + Json.createReader(StringReader(responseString)).readObject() + + private fun createCollectorConfiguration(configuration: JsonObject): CollectorConfiguration { + logger.info { "Obtained new configuration from consul:\n${configuration}" } + val routing = configuration.getJsonArray("collector.routing") + + return CollectorConfiguration( + kafkaBootstrapServers = configuration.getString("dmaap.kafkaBootstrapServers"), + routing = org.onap.dcae.collectors.veshv.model.routing { + for (route in routing) { + val routeObj = route.asJsonObject() + defineRoute { + fromDomain(routeObj.getString("fromDomain")) + toTopic(routeObj.getString("toTopic")) + withFixedPartitioning() + } + } + }.build() + ) + } + + companion object { + private const val MAX_RETRIES = 5L + private const val BACKOFF_INTERVAL_FACTOR = 30L + private val logger = Logger(ConsulConfigurationProvider::class) + } +} + diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/HttpAdapter.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/HttpAdapter.kt new file mode 100644 index 00000000..bdce6f73 --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/HttpAdapter.kt @@ -0,0 +1,68 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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 + +import io.netty.handler.codec.http.HttpStatusClass +import org.slf4j.LoggerFactory +import reactor.core.publisher.Mono +import reactor.netty.http.client.HttpClient + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since May 2018 + */ +open class HttpAdapter(private val httpClient: HttpClient) { + + private val logger = LoggerFactory.getLogger(HttpAdapter::class.java) + + open fun get(url: String, queryParams: Map<String, Any> = emptyMap()): Mono<String> = httpClient + .get() + .uri(url + createQueryString(queryParams)) + .responseSingle { response, content -> + if (response.status().codeClass() == HttpStatusClass.SUCCESS) + content.asString() + else { + val errorMessage = "$url ${response.status().code()} ${response.status().reasonPhrase()}" + Mono.error(IllegalStateException(errorMessage)) + } + } + .doOnError { + logger.error("Failed to get resource on path: $url (${it.localizedMessage})") + logger.debug("Nested exception:", it) + } + + private fun createQueryString(params: Map<String, Any>): String { + if (params.isEmpty()) + return "" + + val builder = StringBuilder("?") + params.forEach { (key, value) -> + builder + .append(key) + .append("=") + .append(value) + .append("&") + + } + + return builder.removeSuffix("&").toString() + } + +} diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/LoggingSinkProvider.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/LoggingSinkProvider.kt new file mode 100644 index 00000000..5f4bf354 --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/LoggingSinkProvider.kt @@ -0,0 +1,62 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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 + +import org.onap.dcae.collectors.veshv.boundary.Sink +import org.onap.dcae.collectors.veshv.boundary.SinkProvider +import org.onap.dcae.collectors.veshv.model.CollectorConfiguration +import org.onap.dcae.collectors.veshv.model.RoutedMessage +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import reactor.core.publisher.Flux +import java.util.concurrent.atomic.AtomicLong + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ +internal class LoggingSinkProvider : SinkProvider { + + override fun invoke(config: CollectorConfiguration): Sink { + return object : Sink { + private val totalMessages = AtomicLong() + private val totalBytes = AtomicLong() + + override fun send(messages: Flux<RoutedMessage>): Flux<RoutedMessage> = + messages + .doOnNext(this::logMessage) + + private fun logMessage(msg: RoutedMessage) { + val msgs = totalMessages.addAndGet(1) + val bytes = totalBytes.addAndGet(msg.message.rawMessage.size().toLong()) + val logMessageSupplier = { "Message routed to ${msg.topic}. Total = $msgs ($bytes B)" } + if (msgs % INFO_LOGGING_FREQ == 0L) + logger.info(logMessageSupplier) + else + logger.trace(logMessageSupplier) + } + + } + } + + companion object { + const val INFO_LOGGING_FREQ = 100_000 + private val logger = Logger(LoggingSinkProvider::class) + } +} diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/KafkaSink.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/KafkaSink.kt new file mode 100644 index 00000000..a0c22418 --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/KafkaSink.kt @@ -0,0 +1,82 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.onap.dcae.collectors.veshv.boundary.Sink +import org.onap.dcae.collectors.veshv.model.RoutedMessage +import org.onap.dcae.collectors.veshv.model.VesMessage +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import org.onap.ves.VesEventOuterClass.CommonEventHeader +import reactor.core.publisher.Flux +import reactor.kafka.sender.KafkaSender +import reactor.kafka.sender.SenderRecord +import reactor.kafka.sender.SenderResult +import java.util.concurrent.atomic.AtomicLong + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +internal class KafkaSink(private val sender: KafkaSender<CommonEventHeader, VesMessage>) : Sink { + private val sentMessages = AtomicLong(0) + + override fun send(messages: Flux<RoutedMessage>): Flux<RoutedMessage> { + val records = messages.map(this::vesToKafkaRecord) + val result = sender.send(records) + .doOnNext(::logException) + .filter(::isSuccessful) + .map { it.correlationMetadata() } + + return if (logger.traceEnabled) { + result.doOnNext(::logSentMessage) + } else { + result + } + } + + private fun vesToKafkaRecord(msg: RoutedMessage): SenderRecord<CommonEventHeader, VesMessage, RoutedMessage> { + return SenderRecord.create( + msg.topic, + msg.partition, + System.currentTimeMillis(), + msg.message.header, + msg.message, + msg) + } + + private fun logException(senderResult: SenderResult<out Any>) { + if (senderResult.exception() != null) { + logger.warn(senderResult.exception()) { "Failed to send message to Kafka" } + } + } + + private fun logSentMessage(sentMsg: RoutedMessage) { + logger.trace { + val msgNum = sentMessages.incrementAndGet() + "Message #$msgNum has been sent to ${sentMsg.topic}:${sentMsg.partition}" + } + } + + private fun isSuccessful(senderResult: SenderResult<out Any>) = senderResult.exception() == null + + companion object { + val logger = Logger(KafkaSink::class) + } +} diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/KafkaSinkProvider.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/KafkaSinkProvider.kt new file mode 100644 index 00000000..18191952 --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/KafkaSinkProvider.kt @@ -0,0 +1,45 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.apache.kafka.clients.producer.ProducerConfig +import org.onap.dcae.collectors.veshv.boundary.Sink +import org.onap.dcae.collectors.veshv.boundary.SinkProvider +import org.onap.dcae.collectors.veshv.model.CollectorConfiguration +import org.onap.dcae.collectors.veshv.model.VesMessage +import org.onap.ves.VesEventOuterClass.CommonEventHeader +import reactor.kafka.sender.KafkaSender +import reactor.kafka.sender.SenderOptions + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ +internal class KafkaSinkProvider : SinkProvider { + override fun invoke(config: CollectorConfiguration): Sink { + return KafkaSink(KafkaSender.create(constructSenderOptions(config))) + } + + private fun constructSenderOptions(config: CollectorConfiguration) = + SenderOptions.create<CommonEventHeader, VesMessage>() + .producerProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, config.kafkaBootstrapServers) + .producerProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ProtobufSerializer::class.java) + .producerProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, VesMessageSerializer::class.java) +} diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/ProtobufSerializer.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/ProtobufSerializer.kt new file mode 100644 index 00000000..4e9932cc --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/ProtobufSerializer.kt @@ -0,0 +1,40 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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 org.apache.kafka.common.serialization.Serializer + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ +class ProtobufSerializer : Serializer<MessageLite> { + override fun configure(configs: MutableMap<String, *>?, isKey: Boolean) { + // no configuration + } + + override fun serialize(topic: String?, data: MessageLite?): ByteArray? = + data?.toByteArray() + + override fun close() { + // cleanup not needed + } +} diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/VesMessageSerializer.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/VesMessageSerializer.kt new file mode 100644 index 00000000..7a6ac7c8 --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/VesMessageSerializer.kt @@ -0,0 +1,37 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.apache.kafka.common.serialization.Serializer +import org.onap.dcae.collectors.veshv.model.VesMessage + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ +class VesMessageSerializer : Serializer<VesMessage> { + override fun configure(configs: MutableMap<String, *>?, isKey: Boolean) { + } + + override fun serialize(topic: String?, msg: VesMessage?): ByteArray? = msg?.rawMessage?.unsafeAsArray() + + override fun close() { + } +} diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/socket/NettyTcpServer.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/socket/NettyTcpServer.kt new file mode 100644 index 00000000..e535300a --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/socket/NettyTcpServer.kt @@ -0,0 +1,114 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.socket + +import arrow.core.getOrElse +import arrow.effects.IO +import org.onap.dcae.collectors.veshv.boundary.CollectorProvider +import org.onap.dcae.collectors.veshv.boundary.Server +import org.onap.dcae.collectors.veshv.model.ServerConfiguration +import org.onap.dcae.collectors.veshv.ssl.boundary.ServerSslContextFactory +import org.onap.dcae.collectors.veshv.utils.NettyServerHandle +import org.onap.dcae.collectors.veshv.utils.ServerHandle +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import reactor.core.publisher.Mono +import reactor.netty.ByteBufFlux +import reactor.netty.Connection +import reactor.netty.NettyInbound +import reactor.netty.NettyOutbound +import reactor.netty.tcp.TcpServer +import java.time.Duration + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +internal class NettyTcpServer(private val serverConfig: ServerConfiguration, + private val sslContextFactory: ServerSslContextFactory, + private val collectorProvider: CollectorProvider) : Server { + + override fun start(): IO<ServerHandle> = IO { + val tcpServer = TcpServer.create() + .addressSupplier { serverConfig.serverListenAddress } + .configureSsl() + .handle(this::handleConnection) + + NettyServerHandle(tcpServer.bindNow()) + } + + private fun TcpServer.configureSsl() = + sslContextFactory + .createSslContext(serverConfig.securityConfiguration) + .map { sslContext -> + this.secure { b -> b.sslContext(sslContext) } + }.getOrElse { this } + + private fun handleConnection(nettyInbound: NettyInbound, nettyOutbound: NettyOutbound): Mono<Void> = + collectorProvider().fold( + { + nettyInbound.withConnection { conn -> + logger.warn { "Collector not ready. Closing connection from ${conn.address()}..." } + } + Mono.empty() + }, + { + nettyInbound.withConnection { conn -> + logger.info { "Handling connection from ${conn.address()}" } + conn.configureIdleTimeout(serverConfig.idleTimeout) + .logConnectionClosed() + } + it.handleConnection(nettyOutbound.alloc(), createDataStream(nettyInbound)) + } + ) + + private fun createDataStream(nettyInbound: NettyInbound): ByteBufFlux = nettyInbound + .receive() + .retain() + + private fun Connection.configureIdleTimeout(timeout: Duration): Connection { + onReadIdle(timeout.toMillis()) { + logger.info { + "Idle timeout of ${timeout.seconds} s reached. Closing connection from ${address()}..." + } + disconnectClient() + } + return this + } + + private fun Connection.disconnectClient() { + channel().close().addListener { + if (it.isSuccess) + logger.debug { "Channel (${address()}) closed successfully." } + else + logger.warn("Channel close failed", it.cause()) + } + } + + private fun Connection.logConnectionClosed(): Connection { + onTerminate().subscribe { + logger.info("Connection from ${address()} has been closed") + } + return this + } + + companion object { + private val logger = Logger(NettyTcpServer::class) + } +} diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/wire/WireChunkDecoder.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/wire/WireChunkDecoder.kt new file mode 100644 index 00000000..4a2ef6b2 --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/wire/WireChunkDecoder.kt @@ -0,0 +1,101 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.wire + +import arrow.effects.IO +import io.netty.buffer.ByteBuf +import io.netty.buffer.ByteBufAllocator +import org.onap.dcae.collectors.veshv.domain.WireFrameMessage +import org.onap.dcae.collectors.veshv.domain.WireFrameDecoder +import org.onap.dcae.collectors.veshv.domain.InvalidWireFrame +import org.onap.dcae.collectors.veshv.domain.WireFrameDecodingError +import org.onap.dcae.collectors.veshv.domain.MissingWireFrameBytes +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import org.onap.dcae.collectors.veshv.utils.logging.handleReactiveStreamError +import reactor.core.publisher.Flux +import reactor.core.publisher.SynchronousSink + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +internal class WireChunkDecoder( + private val decoder: WireFrameDecoder, + alloc: ByteBufAllocator = ByteBufAllocator.DEFAULT) { + private val streamBuffer = alloc.compositeBuffer() + + fun release() { + streamBuffer.release() + } + + fun decode(byteBuf: ByteBuf): Flux<WireFrameMessage> = Flux.defer { + logIncomingMessage(byteBuf) + if (byteBuf.readableBytes() == 0) { + byteBuf.release() + Flux.empty() + } else { + streamBuffer.addComponent(true, byteBuf) + generateFrames() + .onErrorResume { logger.handleReactiveStreamError(it, Flux.error(it)) } + .doFinally { streamBuffer.discardReadComponents() } + } + } + + private fun generateFrames(): Flux<WireFrameMessage> = Flux.generate { next -> + decoder.decodeFirst(streamBuffer) + .fold(onError(next), onSuccess(next)) + .unsafeRunSync() + } + + private fun onError(next: SynchronousSink<WireFrameMessage>): (WireFrameDecodingError) -> IO<Unit> = { err -> + when (err) { + is InvalidWireFrame -> IO { + next.error(WireFrameException(err)) + } + is MissingWireFrameBytes -> IO { + logEndOfData() + next.complete() + } + } + } + + private fun onSuccess(next: SynchronousSink<WireFrameMessage>): (WireFrameMessage) -> IO<Unit> = { frame -> + IO { + logDecodedWireMessage(frame) + next.next(frame) + } + } + + private fun logIncomingMessage(wire: ByteBuf) { + logger.trace { "Got message with total size of ${wire.readableBytes()} B" } + } + + private fun logDecodedWireMessage(wire: WireFrameMessage) { + logger.trace { "Wire payload size: ${wire.payloadSize} B" } + } + + private fun logEndOfData() { + logger.trace { "End of data in current TCP buffer" } + } + + companion object { + val logger = Logger(WireChunkDecoder::class) + } +} diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/wire/WireFrameException.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/wire/WireFrameException.kt new file mode 100644 index 00000000..83a7cd85 --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/wire/WireFrameException.kt @@ -0,0 +1,29 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.wire + +import org.onap.dcae.collectors.veshv.domain.WireFrameDecodingError + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ +class WireFrameException(error: WireFrameDecodingError) + : Exception("${error::class.simpleName}: ${error.message}") diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/CollectorConfiguration.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/CollectorConfiguration.kt new file mode 100644 index 00000000..ec546c7d --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/CollectorConfiguration.kt @@ -0,0 +1,26 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.model + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +data class CollectorConfiguration(val kafkaBootstrapServers: String, val routing: Routing) diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/ConfigurationProviderParams.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/ConfigurationProviderParams.kt new file mode 100644 index 00000000..9de34498 --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/ConfigurationProviderParams.kt @@ -0,0 +1,30 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.model + +import java.time.Duration + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since July 2018 + */ +data class ConfigurationProviderParams(val configurationUrl: String, + val firstRequestDelay: Duration, + val requestInterval: Duration) diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/RoutedMessage.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/RoutedMessage.kt new file mode 100644 index 00000000..782877e3 --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/RoutedMessage.kt @@ -0,0 +1,22 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.model + +data class RoutedMessage(val topic: String, val partition: Int, val message: VesMessage) diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/ServerConfiguration.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/ServerConfiguration.kt new file mode 100644 index 00000000..85117684 --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/ServerConfiguration.kt @@ -0,0 +1,37 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.model + +import org.onap.dcae.collectors.veshv.domain.SecurityConfiguration +import java.net.InetSocketAddress +import java.time.Duration + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +data class ServerConfiguration( + val serverListenAddress: InetSocketAddress, + val configurationProviderParams: ConfigurationProviderParams, + val securityConfiguration: SecurityConfiguration, + val idleTimeout: Duration, + val healthCheckApiListenAddress: InetSocketAddress, + val maximumPayloadSizeBytes: Int, + val dummyMode: Boolean = false) diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/VesMessage.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/VesMessage.kt new file mode 100644 index 00000000..f5bfcce1 --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/VesMessage.kt @@ -0,0 +1,32 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.model + +import org.onap.dcae.collectors.veshv.domain.ByteData +import org.onap.dcae.collectors.veshv.impl.MessageValidator +import org.onap.ves.VesEventOuterClass.CommonEventHeader + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +data class VesMessage(val header: CommonEventHeader, val rawMessage: ByteData) { + fun isValid(): Boolean = MessageValidator.isValid(this) +} diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/routing.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/routing.kt new file mode 100644 index 00000000..bab95c57 --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/routing.kt @@ -0,0 +1,83 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.model + +import arrow.core.Option +import org.onap.ves.VesEventOuterClass.CommonEventHeader + +data class Routing(val routes: List<Route>) { + + fun routeFor(commonHeader: CommonEventHeader): Option<Route> = + Option.fromNullable(routes.find { it.applies(commonHeader) }) +} + +data class Route(val domain: String, val targetTopic: String, val partitioning: (CommonEventHeader) -> Int) { + + fun applies(commonHeader: CommonEventHeader) = commonHeader.domain == domain + + operator fun invoke(message: VesMessage): RoutedMessage = + RoutedMessage(targetTopic, partitioning(message.header), message) +} + + +/* +Configuration DSL + */ + +fun routing(init: RoutingBuilder.() -> Unit): RoutingBuilder { + val conf = RoutingBuilder() + conf.init() + return conf +} + +class RoutingBuilder { + private val routes: MutableList<RouteBuilder> = mutableListOf() + + fun defineRoute(init: RouteBuilder.() -> Unit): RouteBuilder { + val rule = RouteBuilder() + rule.init() + routes.add(rule) + return rule + } + + fun build() = Routing(routes.map { it.build() }.toList()) +} + +class RouteBuilder { + + private lateinit var domain: String + private lateinit var targetTopic: String + private lateinit var partitioning: (CommonEventHeader) -> Int + + fun fromDomain(domain: String) { + this.domain = domain + } + + fun toTopic(targetTopic: String) { + this.targetTopic = targetTopic + } + + fun withFixedPartitioning(num: Int = 0) { + partitioning = { num } + } + + fun build() = Route(domain, targetTopic, partitioning) + +} diff --git a/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/MessageValidatorTest.kt b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/MessageValidatorTest.kt new file mode 100644 index 00000000..3090042d --- /dev/null +++ b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/MessageValidatorTest.kt @@ -0,0 +1,128 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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 + +import org.assertj.core.api.Assertions.assertThat +import org.jetbrains.spek.api.Spek +import org.jetbrains.spek.api.dsl.given +import org.jetbrains.spek.api.dsl.it +import org.jetbrains.spek.api.dsl.on +import org.onap.dcae.collectors.veshv.domain.ByteData +import org.onap.dcae.collectors.veshv.domain.VesEventDomain +import org.onap.dcae.collectors.veshv.model.VesMessage +import org.onap.dcae.collectors.veshv.tests.utils.commonHeader +import org.onap.dcae.collectors.veshv.tests.utils.vesEventBytes +import org.onap.ves.VesEventOuterClass.CommonEventHeader.* + +internal object MessageValidatorTest : Spek({ + + given("Message validator") { + val cut = MessageValidator + + on("ves hv message including header with fully initialized fields") { + val commonHeader = commonHeader() + + it("should accept message with fully initialized message header") { + val vesMessage = VesMessage(commonHeader, vesEventBytes(commonHeader)) + assertThat(cut.isValid(vesMessage)).describedAs("message validation result").isTrue() + } + + VesEventDomain.values() + .forEach { domain -> + it("should accept message with $domain domain") { + val header = commonHeader(domain) + val vesMessage = VesMessage(header, vesEventBytes(header)) + assertThat(cut.isValid(vesMessage)) + .isTrue() + } + } + } + + on("ves hv message bytes") { + val vesMessage = VesMessage(getDefaultInstance(), ByteData.EMPTY) + it("should not accept message with default header") { + assertThat(cut.isValid(vesMessage)).describedAs("message validation result").isFalse() + } + } + + val priorityTestCases = mapOf( + Priority.PRIORITY_NOT_PROVIDED to false, + Priority.HIGH to true + ) + + priorityTestCases.forEach { value, expectedResult -> + on("ves hv message including header with priority $value") { + val commonEventHeader = commonHeader(priority = value) + val vesMessage = VesMessage(commonEventHeader, vesEventBytes(commonEventHeader)) + + it("should resolve validation result") { + assertThat(cut.isValid(vesMessage)).describedAs("message validation results") + .isEqualTo(expectedResult) + } + } + } + + on("ves hv message including header with not initialized fields") { + val commonHeader = newBuilder() + .setVersion("1.9") + .setEventName("Sample event name") + .setEventId("Sample event Id") + .setSourceName("Sample Source") + .build() + val rawMessageBytes = vesEventBytes(commonHeader) + + it("should not accept not fully initialized message header") { + val vesMessage = VesMessage(commonHeader, rawMessageBytes) + assertThat(cut.isValid(vesMessage)).describedAs("message validation result").isFalse() + } + } + + on("ves hv message including header.vesEventListenerVersion with non-string major part") { + val commonHeader = commonHeader(vesEventListenerVersion = "sample-version") + val rawMessageBytes = vesEventBytes(commonHeader) + + + it("should not accept message header") { + val vesMessage = VesMessage(commonHeader, rawMessageBytes) + assertThat(cut.isValid(vesMessage)).describedAs("message validation result").isFalse() + } + } + + on("ves hv message including header.vesEventListenerVersion with major part != 7") { + val commonHeader = commonHeader(vesEventListenerVersion = "1.2.3") + val rawMessageBytes = vesEventBytes(commonHeader) + + it("should not accept message header") { + val vesMessage = VesMessage(commonHeader, rawMessageBytes) + assertThat(cut.isValid(vesMessage)).describedAs("message validation result").isFalse() + } + } + + on("ves hv message including header.vesEventListenerVersion with minor part not starting with a digit") { + val commonHeader = commonHeader(vesEventListenerVersion = "7.test") + val rawMessageBytes = vesEventBytes(commonHeader) + + it("should not accept message header") { + val vesMessage = VesMessage(commonHeader, rawMessageBytes) + assertThat(cut.isValid(vesMessage)).describedAs("message validation result").isFalse() + } + } + } +}) diff --git a/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/RouterTest.kt b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/RouterTest.kt new file mode 100644 index 00000000..e8a31231 --- /dev/null +++ b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/RouterTest.kt @@ -0,0 +1,112 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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 + +import arrow.core.None +import arrow.core.Some +import org.assertj.core.api.Assertions.assertThat +import org.jetbrains.spek.api.Spek +import org.jetbrains.spek.api.dsl.given +import org.jetbrains.spek.api.dsl.it +import org.jetbrains.spek.api.dsl.on +import org.onap.dcae.collectors.veshv.domain.ByteData +import org.onap.dcae.collectors.veshv.domain.VesEventDomain.PERF3GPP +import org.onap.dcae.collectors.veshv.domain.VesEventDomain.HEARTBEAT +import org.onap.dcae.collectors.veshv.domain.VesEventDomain.SYSLOG +import org.onap.dcae.collectors.veshv.model.RoutedMessage +import org.onap.dcae.collectors.veshv.model.VesMessage +import org.onap.dcae.collectors.veshv.model.routing +import org.onap.dcae.collectors.veshv.tests.utils.commonHeader + + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +object RouterTest : Spek({ + given("sample configuration") { + val config = routing { + + defineRoute { + fromDomain(PERF3GPP.domainName) + toTopic("ves_rtpm") + withFixedPartitioning(2) + } + + defineRoute { + fromDomain(SYSLOG.domainName) + toTopic("ves_trace") + withFixedPartitioning() + } + }.build() + val cut = Router(config) + + on("message with existing route (rtpm)") { + val message = VesMessage(commonHeader(PERF3GPP), ByteData.EMPTY) + val result = cut.findDestination(message) + + it("should have route available") { + assertThat(result).isNotNull() + } + + it("should be routed to proper partition") { + assertThat(result.map(RoutedMessage::partition)).isEqualTo(Some(2)) + } + + it("should be routed to proper topic") { + assertThat(result.map(RoutedMessage::topic)).isEqualTo(Some("ves_rtpm")) + } + + it("should be routed with a given message") { + assertThat(result.map(RoutedMessage::message)).isEqualTo(Some(message)) + } + } + + on("message with existing route (trace)") { + val message = VesMessage(commonHeader(SYSLOG), ByteData.EMPTY) + val result = cut.findDestination(message) + + it("should have route available") { + assertThat(result).isNotNull() + } + + it("should be routed to proper partition") { + assertThat(result.map(RoutedMessage::partition)).isEqualTo(Some(0)) + } + + it("should be routed to proper topic") { + assertThat(result.map(RoutedMessage::topic)).isEqualTo(Some("ves_trace")) + } + + it("should be routed with a given message") { + assertThat(result.map(RoutedMessage::message)).isEqualTo(Some(message)) + } + } + + on("message with unknown route") { + val message = VesMessage(commonHeader(HEARTBEAT), ByteData.EMPTY) + val result = cut.findDestination(message) + + it("should not have route available") { + assertThat(result).isEqualTo(None) + } + } + } +})
\ No newline at end of file diff --git a/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/VesDecoderTest.kt b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/VesDecoderTest.kt new file mode 100644 index 00000000..8950a557 --- /dev/null +++ b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/VesDecoderTest.kt @@ -0,0 +1,74 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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 + +import arrow.core.Option +import com.google.protobuf.ByteString +import org.jetbrains.spek.api.Spek +import org.jetbrains.spek.api.dsl.given +import org.jetbrains.spek.api.dsl.it +import org.jetbrains.spek.api.dsl.on +import org.onap.dcae.collectors.veshv.domain.ByteData +import org.onap.dcae.collectors.veshv.domain.VesEventDomain.HEARTBEAT +import org.onap.dcae.collectors.veshv.model.VesMessage +import org.onap.dcae.collectors.veshv.tests.utils.commonHeader +import org.onap.dcae.collectors.veshv.tests.utils.vesEventBytes +import java.nio.charset.Charset +import kotlin.test.assertTrue +import kotlin.test.fail + + +internal object VesDecoderTest : Spek({ + + given("ves message decoder") { + val cut = VesDecoder() + + on("ves hv message bytes") { + val commonHeader = commonHeader(HEARTBEAT) + val rawMessageBytes = vesEventBytes(commonHeader, ByteString.copyFromUtf8("highvolume measurements")) + + it("should decode only header and pass it on along with raw message") { + val expectedMessage = VesMessage( + commonHeader, + rawMessageBytes + ) + + assertTrue { + cut.decode(rawMessageBytes).exists { + it == expectedMessage + } + } + } + } + + on("invalid ves hv message bytes") { + val rawMessageBytes = ByteData("ala ma kota".toByteArray(Charset.defaultCharset())) + + it("should throw error") { + assertFailedWithError(cut.decode(rawMessageBytes)) + } + } + } +}) + +private fun <A> assertFailedWithError(option: Option<A>) = + option.exists { + fail("Error expected") + } 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/ConsulConfigurationProviderTest.kt new file mode 100644 index 00000000..c6364f74 --- /dev/null +++ b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/ConsulConfigurationProviderTest.kt @@ -0,0 +1,157 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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 + +import com.nhaarman.mockitokotlin2.eq +import com.nhaarman.mockitokotlin2.mock +import com.nhaarman.mockitokotlin2.whenever +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.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 reactor.core.publisher.Mono +import reactor.retry.Retry +import reactor.test.StepVerifier +import java.time.Duration +import java.util.* +import kotlin.test.assertEquals + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since May 2018 + */ +internal object ConsulConfigurationProviderTest : Spek({ + + describe("Consul configuration provider") { + + val httpAdapterMock: HttpAdapter = mock() + val healthStateProvider = HealthState.INSTANCE + + given("valid resource url") { + val validUrl = "http://valid-url/" + val consulConfigProvider = constructConsulConfigProvider(validUrl, httpAdapterMock, healthStateProvider) + + on("call to consul") { + whenever(httpAdapterMock.get(eq(validUrl), Mockito.anyMap())) + .thenReturn(Mono.just(constructConsulResponse())) + + it("should use received configuration") { + + StepVerifier.create(consulConfigProvider().take(1)) + .consumeNextWith { + + assertEquals("$kafkaAddress:9093", it.kafkaBootstrapServers) + + val route1 = it.routing.routes[0] + assertThat(FAULT.domainName) + .describedAs("routed domain 1") + .isEqualTo(route1.domain) + assertThat("test-topic-1") + .describedAs("target topic 1") + .isEqualTo(route1.targetTopic) + + val route2 = it.routing.routes[1] + assertThat(HEARTBEAT.domainName) + .describedAs("routed domain 2") + .isEqualTo(route2.domain) + assertThat("test-topic-2") + .describedAs("target topic 2") + .isEqualTo(route2.targetTopic) + + }.verifyComplete() + } + } + + } + given("invalid resource url") { + val invalidUrl = "http://invalid-url/" + + val iterationCount = 3L + val consulConfigProvider = constructConsulConfigProvider( + invalidUrl, httpAdapterMock, healthStateProvider, iterationCount + ) + + on("call to consul") { + whenever(httpAdapterMock.get(eq(invalidUrl), Mockito.anyMap())) + .thenReturn(Mono.error(RuntimeException("Test exception"))) + + it("should interrupt the flux") { + + StepVerifier.create(consulConfigProvider()) + .verifyErrorMessage("Test exception") + } + + it("should update the health state") { + StepVerifier.create(healthStateProvider().take(iterationCount)) + .expectNextCount(iterationCount - 1) + .expectNext(HealthDescription.RETRYING_FOR_CONSUL_CONFIGURATION) + .verifyComplete() + } + } + } + } + +}) + +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 + ) +} + + +const val kafkaAddress = "message-router-kafka" + +fun constructConsulResponse(): String = + """{ + "dmaap.kafkaBootstrapServers": "$kafkaAddress:9093", + "collector.routing": [ + { + "fromDomain": "fault", + "toTopic": "test-topic-1" + }, + { + "fromDomain": "heartbeat", + "toTopic": "test-topic-2" + } + ] + }""" diff --git a/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/HttpAdapterTest.kt b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/HttpAdapterTest.kt new file mode 100644 index 00000000..91457faf --- /dev/null +++ b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/HttpAdapterTest.kt @@ -0,0 +1,86 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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 + +import org.jetbrains.spek.api.Spek +import org.jetbrains.spek.api.dsl.describe +import org.jetbrains.spek.api.dsl.given +import org.jetbrains.spek.api.dsl.it +import reactor.core.publisher.Mono +import reactor.netty.http.client.HttpClient +import reactor.netty.http.server.HttpServer +import reactor.test.StepVerifier +import reactor.test.test + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since May 2018 + */ +internal object HttpAdapterTest : Spek({ + describe("HttpAdapter") { + + val httpServer = HttpServer.create() + .host("127.0.0.1") + .route { routes -> + routes.get("/url") { req, resp -> + resp.sendString(Mono.just(req.uri())) + } + } + .bindNow() + val baseUrl = "http://${httpServer.host()}:${httpServer.port()}" + val httpAdapter = HttpAdapter(HttpClient.create().baseUrl(baseUrl)) + + afterGroup { + httpServer.disposeNow() + } + + given("url without query params") { + val url = "/url" + + it("should not append query string") { + httpAdapter.get(url).test() + .expectNext(url) + .verifyComplete() + } + } + + given("url with query params") { + val queryParams = mapOf(Pair("p", "the-value")) + val url = "/url" + + it("should add them as query string to the url") { + httpAdapter.get(url, queryParams).test() + .expectNext("/url?p=the-value") + .verifyComplete() + } + } + + given("invalid url") { + val invalidUrl = "/wtf" + + it("should interrupt the flux") { + StepVerifier + .create(httpAdapter.get(invalidUrl)) + .verifyError() + } + } + } + +})
\ No newline at end of file diff --git a/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/wire/WireChunkDecoderTest.kt b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/wire/WireChunkDecoderTest.kt new file mode 100644 index 00000000..f06a0dc7 --- /dev/null +++ b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/wire/WireChunkDecoderTest.kt @@ -0,0 +1,234 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.wire + +import io.netty.buffer.ByteBuf +import io.netty.buffer.Unpooled +import io.netty.buffer.UnpooledByteBufAllocator +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.given +import org.jetbrains.spek.api.dsl.it +import org.onap.dcae.collectors.veshv.domain.WireFrameEncoder +import org.onap.dcae.collectors.veshv.domain.WireFrameDecoder +import org.onap.dcae.collectors.veshv.domain.WireFrameMessage +import reactor.test.test + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk></piotr.jaszczyk>@nokia.com> + * @since May 2018 + */ +internal object WireChunkDecoderTest : Spek({ + val alloc = UnpooledByteBufAllocator.DEFAULT + val samplePayload = "konstantynopolitanczykowianeczka".toByteArray() + val anotherPayload = "ala ma kota a kot ma ale".toByteArray() + + val encoder = WireFrameEncoder(alloc) + + fun WireChunkDecoder.decode(frame: WireFrameMessage) = decode(encoder.encode(frame)) + + fun createInstance() = WireChunkDecoder(WireFrameDecoder(WireFrameMessage.DEFAULT_MAX_PAYLOAD_SIZE_BYTES), alloc) + + fun verifyMemoryReleased(vararg byteBuffers: ByteBuf) { + for (bb in byteBuffers) { + assertThat(bb.refCnt()) + .describedAs("should be released: $bb ref count") + .isEqualTo(0) + } + } + + fun verifyMemoryNotReleased(vararg byteBuffers: ByteBuf) { + for (bb in byteBuffers) { + assertThat(bb.refCnt()) + .describedAs("should not be released: $bb ref count") + .isEqualTo(1) + } + } + + describe("decoding wire protocol") { + given("empty input") { + val input = Unpooled.EMPTY_BUFFER + + it("should yield empty result") { + createInstance().decode(input).test().verifyComplete() + } + } + + given("input with no readable bytes") { + val input = Unpooled.wrappedBuffer(byteArrayOf(0x00)).readerIndex(1) + + it("should yield empty result") { + createInstance().decode(input).test().verifyComplete() + } + + it("should release memory") { + verifyMemoryReleased(input) + } + } + + given("invalid input (not starting with marker)") { + val input = Unpooled.wrappedBuffer(samplePayload) + + it("should yield error") { + createInstance().decode(input).test() + .verifyError(WireFrameException::class.java) + } + + it("should leave memory unreleased") { + verifyMemoryNotReleased(input) + } + } + + given("valid input") { + val input = WireFrameMessage(samplePayload) + + it("should yield decoded input frame") { + createInstance().decode(input).test() + .expectNextMatches { it.payloadSize == samplePayload.size } + .verifyComplete() + } + } + + given("valid input with part of next frame") { + val input = Unpooled.buffer() + .writeBytes(encoder.encode(WireFrameMessage(samplePayload))) + .writeBytes(encoder.encode(WireFrameMessage(samplePayload)).slice(0, 3)) + + it("should yield decoded input frame") { + createInstance().decode(input).test() + .expectNextMatches { it.payloadSize == samplePayload.size } + .verifyComplete() + } + + it("should leave memory unreleased") { + verifyMemoryNotReleased(input) + } + } + + given("valid input with garbage after it") { + val input = Unpooled.buffer() + .writeBytes(encoder.encode(WireFrameMessage(samplePayload))) + .writeBytes(Unpooled.wrappedBuffer(samplePayload)) + + it("should yield decoded input frame and error") { + createInstance().decode(input).test() + .expectNextMatches { it.payloadSize == samplePayload.size } + .verifyError(WireFrameException::class.java) + } + + it("should leave memory unreleased") { + verifyMemoryNotReleased(input) + } + } + + given("two inputs containing two separate messages") { + val input1 = encoder.encode(WireFrameMessage(samplePayload)) + val input2 = encoder.encode(WireFrameMessage(anotherPayload)) + + it("should yield decoded input frames") { + val cut = createInstance() + cut.decode(input1).test() + .expectNextMatches { it.payloadSize == samplePayload.size } + .verifyComplete() + cut.decode(input2).test() + .expectNextMatches { it.payloadSize == anotherPayload.size } + .verifyComplete() + } + + it("should release memory") { + verifyMemoryReleased(input1, input2) + } + } + + given("1st input containing 1st frame and 2nd input containing garbage") { + val input1 = encoder.encode(WireFrameMessage(samplePayload)) + val input2 = Unpooled.wrappedBuffer(anotherPayload) + + it("should yield decoded input frames") { + val cut = createInstance() + cut.decode(input1) + .test() + .expectNextMatches { it.payloadSize == samplePayload.size } + .verifyComplete() + cut.decode(input2).test() + .verifyError(WireFrameException::class.java) + } + + it("should release memory for 1st input") { + verifyMemoryReleased(input1) + } + + it("should leave memory unreleased for 2nd input") { + verifyMemoryNotReleased(input2) + } + } + + + given("1st input containing 1st frame + part of 2nd frame and 2nd input containing rest of 2nd frame") { + val frame1 = encoder.encode(WireFrameMessage(samplePayload)) + val frame2 = encoder.encode(WireFrameMessage(anotherPayload)) + + val input1 = Unpooled.buffer() + .writeBytes(frame1) + .writeBytes(frame2, 3) + val input2 = Unpooled.buffer().writeBytes(frame2) + + it("should yield decoded input frames") { + val cut = createInstance() + cut.decode(input1).test() + .expectNextMatches { it.payloadSize == samplePayload.size } + .verifyComplete() + cut.decode(input2).test() + .expectNextMatches { it.payloadSize == anotherPayload.size } + .verifyComplete() + } + + it("should release memory") { + verifyMemoryReleased(input1, input2) + } + } + + given("1st input containing part of 1st frame and 2nd input containing rest of 1st + 2nd frame") { + val frame1 = encoder.encode(WireFrameMessage(samplePayload)) + val frame2 = encoder.encode(WireFrameMessage(anotherPayload)) + + val input1 = Unpooled.buffer() + .writeBytes(frame1, 5) + val input2 = Unpooled.buffer() + .writeBytes(frame1) + .writeBytes(frame2) + + it("should yield decoded input frames") { + val cut = createInstance() + cut.decode(input1).test() + .verifyComplete() + cut.decode(input2).test() + .expectNextMatches { it.payloadSize == samplePayload.size } + .expectNextMatches { it.payloadSize == anotherPayload.size } + .verifyComplete() + } + + it("should release memory") { + verifyMemoryReleased(input1, input2) + } + } + } +})
\ No newline at end of file diff --git a/sources/hv-collector-core/src/test/resources/logback-test.xml b/sources/hv-collector-core/src/test/resources/logback-test.xml new file mode 100644 index 00000000..9a4eacfe --- /dev/null +++ b/sources/hv-collector-core/src/test/resources/logback-test.xml @@ -0,0 +1,35 @@ +<?xml version="1.0" encoding="UTF-8"?> +<configuration> + <property name="LOG_FILE" + value="${LOG_FILE:-${LOG_PATH:-${LOG_TEMP:-${java.io.tmpdir:-/tmp}}/}ves-hv.log}"/> + <property name="FILE_LOG_PATTERN" value="%d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %-5level [%-40.40logger{10}] - %msg%n"/> + + <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> + <encoder> + <pattern> + %d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %highlight(%-5level) [%-40.40logger{10}] - %msg%n + </pattern> + </encoder> + </appender> + + <appender name="ROLLING-FILE" + class="ch.qos.logback.core.rolling.RollingFileAppender"> + <encoder> + <pattern>${FILE_LOG_PATTERN}</pattern> + </encoder> + <file>${LOG_FILE}</file> + <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy"> + <fileNamePattern>${LOG_FILE}.%d{yyyy-MM-dd}.log</fileNamePattern> + <maxFileSize>50MB</maxFileSize> + <maxHistory>30</maxHistory> + <totalSizeCap>10GB</totalSizeCap> + </rollingPolicy> + </appender> + + <logger name="org.onap.dcae.collectors.veshv" level="TRACE"/> + + <root level="INFO"> + <appender-ref ref="CONSOLE"/> + <appender-ref ref="ROLLING-FILE"/> + </root> +</configuration> diff --git a/sources/hv-collector-ct/pom.xml b/sources/hv-collector-ct/pom.xml new file mode 100644 index 00000000..61ac426b --- /dev/null +++ b/sources/hv-collector-ct/pom.xml @@ -0,0 +1,115 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ ============LICENSE_START======================================================= + ~ dcaegen2-collectors-veshv + ~ ================================================================================ + ~ Copyright (C) 2018 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========================================================= + --> +<project xmlns="http://maven.apache.org/POM/4.0.0" + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + + <licenses> + <license> + <name>The Apache Software License, Version 2.0</name> + <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url> + </license> + </licenses> + + <parent> + <groupId>org.onap.dcaegen2.collectors.hv-ves</groupId> + <artifactId>hv-collector-sources</artifactId> + <version>1.1.0-SNAPSHOT</version> + <relativePath>..</relativePath> + </parent> + + <artifactId>hv-collector-ct</artifactId> + <description>VES HighVolume Collector :: Component tests</description> + + <properties> + <failIfMissingUnitTests>false</failIfMissingUnitTests> + <failIfMissingComponentTests>true</failIfMissingComponentTests> + </properties> + + <build> + <plugins> + <plugin> + <artifactId>kotlin-maven-plugin</artifactId> + <groupId>org.jetbrains.kotlin</groupId> + </plugin> + <plugin> + <artifactId>maven-surefire-plugin</artifactId> + <groupId>org.apache.maven.plugins</groupId> + </plugin> + </plugins> + </build> + + <dependencies> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-core</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-xnf-simulator</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-test-utils</artifactId> + <version>${project.parent.version}</version> + </dependency> + + + <dependency> + <groupId>io.arrow-kt</groupId> + <artifactId>arrow-syntax</artifactId> + </dependency> + <dependency> + <groupId>org.slf4j</groupId> + <artifactId>slf4j-api</artifactId> + </dependency> + <dependency> + <groupId>ch.qos.logback</groupId> + <artifactId>logback-classic</artifactId> + </dependency> + + <dependency> + <groupId>org.assertj</groupId> + <artifactId>assertj-core</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.kotlin</groupId> + <artifactId>kotlin-test</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.spek</groupId> + <artifactId>spek-api</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.spek</groupId> + <artifactId>spek-junit-platform-engine</artifactId> + </dependency> + <dependency> + <groupId>io.projectreactor</groupId> + <artifactId>reactor-test</artifactId> + </dependency> + </dependencies> + + +</project>
\ No newline at end of file diff --git a/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/component/PerformanceSpecification.kt b/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/component/PerformanceSpecification.kt new file mode 100644 index 00000000..0897e910 --- /dev/null +++ b/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/component/PerformanceSpecification.kt @@ -0,0 +1,203 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.tests.component + +import arrow.syntax.function.partially1 +import io.netty.buffer.ByteBuf +import io.netty.buffer.ByteBufAllocator +import io.netty.buffer.CompositeByteBuf +import io.netty.buffer.Unpooled +import io.netty.buffer.UnpooledByteBufAllocator +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.onap.dcae.collectors.veshv.domain.VesEventDomain.PERF3GPP +import org.onap.dcae.collectors.veshv.domain.WireFrameEncoder +import org.onap.dcae.collectors.veshv.tests.fakes.CountingSink +import org.onap.dcae.collectors.veshv.tests.fakes.basicConfiguration +import org.onap.dcae.collectors.veshv.tests.utils.commonHeader +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageParameters +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageType.VALID +import org.onap.dcae.collectors.veshv.ves.message.generator.factory.MessageGeneratorFactory +import reactor.core.publisher.Flux +import reactor.math.sum +import java.security.MessageDigest +import java.time.Duration +import java.util.* +import kotlin.system.measureTimeMillis + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +object PerformanceSpecification : Spek({ + debugRx(false) + + describe("VES High Volume Collector performance") { + it("should handle multiple clients in reasonable time") { + val sink = CountingSink() + val sut = Sut(sink) + sut.configurationProvider.updateConfiguration(basicConfiguration) + + val numMessages: Long = 300_000 + val runs = 4 + val timeout = Duration.ofMinutes((1 + (runs / 2)).toLong()) + + val params = MessageParameters( + commonEventHeader = commonHeader(PERF3GPP), + messageType = VALID, + amount = numMessages + ) + + val fluxes = (1.rangeTo(runs)).map { + sut.collector.handleConnection(sut.alloc, generateDataStream(sut.alloc, params)) + } + val durationMs = measureTimeMillis { + Flux.merge(fluxes).then().block(timeout) + } + + val durationSec = durationMs / 1000.0 + val throughput = sink.count / durationSec + logger.info("Processed $runs connections each containing $numMessages msgs.") + logger.info("Forwarded ${sink.count / ONE_MILION} Mmsgs in $durationSec seconds, that is $throughput msgs/s") + assertThat(sink.count) + .describedAs("should send all events") + .isEqualTo(runs * numMessages) + } + + it("should disconnect on transmission errors") { + val sink = CountingSink() + val sut = Sut(sink) + sut.configurationProvider.updateConfiguration(basicConfiguration) + + val numMessages: Long = 100_000 + val timeout = Duration.ofSeconds(30) + + val params = MessageParameters( + commonEventHeader = commonHeader(PERF3GPP), + messageType = VALID, + amount = numMessages + ) + + val dataStream = generateDataStream(sut.alloc, params) + .transform(::dropWhenIndex.partially1 { it % 101 == 0L }) + sut.collector.handleConnection(sut.alloc, dataStream) + .timeout(timeout) + .block() + + logger.info("Forwarded ${sink.count} msgs") + assertThat(sink.count) + .describedAs("should send up to number of events") + .isLessThan(numMessages) + } + } + + describe("test infrastructure") { + val digest = MessageDigest.getInstance("MD5") + + fun collectDigest(bb: ByteBuf) { + bb.markReaderIndex() + while (bb.isReadable) { + digest.update(bb.readByte()) + } + bb.resetReaderIndex() + } + + fun calculateDigest(arrays: List<ByteArray>): ByteArray { + for (array in arrays) { + digest.update(array) + } + return digest.digest() + } + + it("should yield same bytes as in the input") { + val numberOfBuffers = 10 + val singleBufferSize = 1000 + val arrays = (1.rangeTo(numberOfBuffers)).map { randomByteArray(singleBufferSize) } + val inputDigest = calculateDigest(arrays) + + val actualTotalSize = Flux.fromIterable(arrays) + .map { Unpooled.wrappedBuffer(it) } + .transform { simulateRemoteTcp(UnpooledByteBufAllocator.DEFAULT, 4, it) } + .doOnNext(::collectDigest) + .map { + val size = it.readableBytes() + it.release() + size + } + .sum() + .map(Long::toInt) + .block() + + val outputDigest = digest.digest() + + assertThat(actualTotalSize!!).isEqualTo(numberOfBuffers * singleBufferSize) + assertThat(outputDigest).isEqualTo(inputDigest) + + } + } +}) + + +private const val ONE_MILION = 1_000_000.0 + +private val rand = Random() +private fun randomByteArray(size: Int): ByteArray { + val bytes = ByteArray(size) + rand.nextBytes(bytes) + return bytes +} + +fun dropWhenIndex(predicate: (Long) -> Boolean, stream: Flux<ByteBuf>): Flux<ByteBuf> = + stream.index() + .filter { predicate(it.t1) } + .map { it.t2 } + +private fun generateDataStream(alloc: ByteBufAllocator, params: MessageParameters): Flux<ByteBuf> = + WireFrameEncoder(alloc).let { encoder -> + MessageGeneratorFactory.create(Sut.MAX_PAYLOAD_SIZE_BYTES) + .createMessageFlux(listOf(params)) + .map(encoder::encode) + .transform { simulateRemoteTcp(alloc, 1000, it) } + } + +private fun simulateRemoteTcp(alloc: ByteBufAllocator, maxSize: Int, byteBuffers: Flux<ByteBuf>) = + byteBuffers + .bufferTimeout(maxSize, Duration.ofMillis(250)) + .map { joinBuffers(alloc, it) } + .concatMap { randomlySplitTcpFrames(it) } + +private fun joinBuffers(alloc: ByteBufAllocator, it: List<ByteBuf>?) = + alloc.compositeBuffer().addComponents(true, it) + +private fun randomlySplitTcpFrames(bb: CompositeByteBuf): Flux<ByteBuf> { + val targetFrameSize = Math.max(4, (bb.readableBytes() * Math.random()).toInt()) + return Flux.create<ByteBuf> { sink -> + while (bb.isReadable) { + val frameSize = Math.min(targetFrameSize, bb.readableBytes()) + sink.next(bb.retainedSlice(bb.readerIndex(), frameSize)) + bb.readerIndex(bb.readerIndex() + frameSize) + } + bb.release() + sink.complete() + } +} + diff --git a/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/component/Sut.kt b/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/component/Sut.kt new file mode 100644 index 00000000..0495ced5 --- /dev/null +++ b/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/component/Sut.kt @@ -0,0 +1,68 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.tests.component + +import arrow.core.getOrElse +import io.netty.buffer.ByteBuf +import io.netty.buffer.ByteBufAllocator +import io.netty.buffer.UnpooledByteBufAllocator +import org.onap.dcae.collectors.veshv.boundary.Collector +import org.onap.dcae.collectors.veshv.boundary.Sink +import org.onap.dcae.collectors.veshv.boundary.SinkProvider +import org.onap.dcae.collectors.veshv.factory.CollectorFactory +import org.onap.dcae.collectors.veshv.model.RoutedMessage +import org.onap.dcae.collectors.veshv.tests.fakes.FakeConfigurationProvider +import org.onap.dcae.collectors.veshv.tests.fakes.FakeHealthState +import org.onap.dcae.collectors.veshv.tests.fakes.FakeMetrics +import org.onap.dcae.collectors.veshv.tests.fakes.StoringSink +import reactor.core.publisher.Flux +import java.time.Duration + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +class Sut(sink: Sink = StoringSink()) { + val configurationProvider = FakeConfigurationProvider() + val healthStateProvider = FakeHealthState() + + val alloc: ByteBufAllocator = UnpooledByteBufAllocator.DEFAULT + private val metrics = FakeMetrics() + private val collectorFactory = CollectorFactory( + configurationProvider, + SinkProvider.just(sink), + metrics, + MAX_PAYLOAD_SIZE_BYTES, + healthStateProvider) + private val collectorProvider = collectorFactory.createVesHvCollectorProvider() + + val collector: Collector + get() = collectorProvider().getOrElse{ throw IllegalStateException("Collector not available.") } + + companion object { + const val MAX_PAYLOAD_SIZE_BYTES = 1024 + } + +} + +fun Sut.handleConnection(sink: StoringSink, vararg packets: ByteBuf): List<RoutedMessage> { + collector.handleConnection(alloc, Flux.fromArray(packets)).block(Duration.ofSeconds(10)) + return sink.sentMessages +} diff --git a/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/component/VesHvSpecification.kt b/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/component/VesHvSpecification.kt new file mode 100644 index 00000000..2d81c671 --- /dev/null +++ b/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/component/VesHvSpecification.kt @@ -0,0 +1,347 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.tests.component + +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.given +import org.jetbrains.spek.api.dsl.it +import org.onap.dcae.collectors.veshv.domain.VesEventDomain.OTHER +import org.onap.dcae.collectors.veshv.domain.VesEventDomain.PERF3GPP +import org.onap.dcae.collectors.veshv.domain.VesEventDomain.HEARTBEAT +import org.onap.dcae.collectors.veshv.domain.VesEventDomain.MEASUREMENT +import org.onap.dcae.collectors.veshv.healthcheck.api.HealthDescription +import org.onap.dcae.collectors.veshv.tests.fakes.ALTERNATE_PERF3GPP_TOPIC +import org.onap.dcae.collectors.veshv.tests.fakes.PERF3GPP_TOPIC +import org.onap.dcae.collectors.veshv.tests.fakes.MEASUREMENTS_FOR_VF_SCALING_TOPIC +import org.onap.dcae.collectors.veshv.tests.fakes.StoringSink +import org.onap.dcae.collectors.veshv.tests.fakes.basicConfiguration +import org.onap.dcae.collectors.veshv.tests.fakes.configurationWithDifferentRouting +import org.onap.dcae.collectors.veshv.tests.fakes.configurationWithoutRouting +import org.onap.dcae.collectors.veshv.tests.fakes.twoDomainsToOneTopicConfiguration +import org.onap.dcae.collectors.veshv.tests.utils.garbageFrame +import org.onap.dcae.collectors.veshv.tests.utils.invalidWireFrame +import org.onap.dcae.collectors.veshv.tests.utils.vesMessageWithPayloadOfSize +import org.onap.dcae.collectors.veshv.tests.utils.vesWireFrameMessage +import org.onap.dcae.collectors.veshv.tests.utils.wireFrameMessageWithInvalidPayload + +import reactor.core.publisher.Flux +import java.time.Duration + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +object VesHvSpecification : Spek({ + debugRx(false) + + describe("VES High Volume Collector") { + it("should handle multiple HV RAN events") { + val (sut, sink) = vesHvWithStoringSink() + val messages = sut.handleConnection(sink, + vesWireFrameMessage(PERF3GPP), + vesWireFrameMessage(PERF3GPP) + ) + + assertThat(messages) + .describedAs("should send all events") + .hasSize(2) + } + } + + describe("Memory management") { + it("should release memory for each handled and dropped message") { + val (sut, sink) = vesHvWithStoringSink() + val validMessage = vesWireFrameMessage(PERF3GPP) + val msgWithInvalidFrame = invalidWireFrame() + val msgWithTooBigPayload = vesMessageWithPayloadOfSize(Sut.MAX_PAYLOAD_SIZE_BYTES + 1, PERF3GPP) + val expectedRefCnt = 0 + + val handledEvents = sut.handleConnection( + sink, validMessage, msgWithInvalidFrame, msgWithTooBigPayload) + + assertThat(handledEvents).hasSize(1) + + assertThat(validMessage.refCnt()) + .describedAs("handled message should be released") + .isEqualTo(expectedRefCnt) + assertThat(msgWithInvalidFrame.refCnt()) + .describedAs("message with invalid frame should be released") + .isEqualTo(expectedRefCnt) + assertThat(msgWithTooBigPayload.refCnt()) + .describedAs("message with payload exceeding 1MiB should be released") + .isEqualTo(expectedRefCnt) + } + + it("should release memory for each message with invalid payload") { + val (sut, sink) = vesHvWithStoringSink() + val validMessage = vesWireFrameMessage(PERF3GPP) + val msgWithInvalidPayload = wireFrameMessageWithInvalidPayload() + val expectedRefCnt = 0 + + val handledEvents = sut.handleConnection(sink, validMessage, msgWithInvalidPayload) + + assertThat(handledEvents).hasSize(1) + + assertThat(validMessage.refCnt()) + .describedAs("handled message should be released") + .isEqualTo(expectedRefCnt) + assertThat(msgWithInvalidPayload.refCnt()) + .describedAs("message with invalid payload should be released") + .isEqualTo(expectedRefCnt) + + } + + it("should release memory for each message with garbage frame") { + val (sut, sink) = vesHvWithStoringSink() + val validMessage = vesWireFrameMessage(PERF3GPP) + val msgWithGarbageFrame = garbageFrame() + val expectedRefCnt = 0 + + val handledEvents = sut.handleConnection(sink, validMessage, msgWithGarbageFrame) + + assertThat(handledEvents).hasSize(1) + + assertThat(validMessage.refCnt()) + .describedAs("handled message should be released") + .isEqualTo(expectedRefCnt) + assertThat(msgWithGarbageFrame.refCnt()) + .describedAs("message with garbage frame should be released") + .isEqualTo(expectedRefCnt) + + } + } + + describe("message routing") { + it("should direct message to a topic by means of routing configuration") { + val (sut, sink) = vesHvWithStoringSink() + + val messages = sut.handleConnection(sink, vesWireFrameMessage(PERF3GPP)) + assertThat(messages).describedAs("number of routed messages").hasSize(1) + + val msg = messages[0] + assertThat(msg.topic).describedAs("routed message topic").isEqualTo(PERF3GPP_TOPIC) + assertThat(msg.partition).describedAs("routed message partition").isEqualTo(0) + } + + it("should be able to direct 2 messages from different domains to one topic") { + val (sut, sink) = vesHvWithStoringSink() + + sut.configurationProvider.updateConfiguration(twoDomainsToOneTopicConfiguration) + + val messages = sut.handleConnection(sink, + vesWireFrameMessage(PERF3GPP), + vesWireFrameMessage(HEARTBEAT), + vesWireFrameMessage(MEASUREMENT)) + + assertThat(messages).describedAs("number of routed messages").hasSize(3) + + assertThat(messages[0].topic).describedAs("first message topic") + .isEqualTo(PERF3GPP_TOPIC) + + assertThat(messages[1].topic).describedAs("second message topic") + .isEqualTo(PERF3GPP_TOPIC) + + assertThat(messages[2].topic).describedAs("last message topic") + .isEqualTo(MEASUREMENTS_FOR_VF_SCALING_TOPIC) + } + + it("should drop message if route was not found") { + val (sut, sink) = vesHvWithStoringSink() + val messages = sut.handleConnection(sink, + vesWireFrameMessage(OTHER, "first"), + vesWireFrameMessage(PERF3GPP, "second"), + vesWireFrameMessage(HEARTBEAT, "third")) + + assertThat(messages).describedAs("number of routed messages").hasSize(1) + + val msg = messages[0] + assertThat(msg.topic).describedAs("routed message topic").isEqualTo(PERF3GPP_TOPIC) + assertThat(msg.message.header.eventId).describedAs("routed message eventId").isEqualTo("second") + } + } + + describe("configuration update") { + + val defaultTimeout = Duration.ofSeconds(10) + + given("successful configuration change") { + + lateinit var sut: Sut + lateinit var sink: StoringSink + + beforeEachTest { + vesHvWithStoringSink().run { + sut = first + sink = second + } + } + + it("should update collector") { + val firstCollector = sut.collector + + sut.configurationProvider.updateConfiguration(configurationWithDifferentRouting) + val collectorAfterUpdate = sut.collector + + assertThat(collectorAfterUpdate).isNotSameAs(firstCollector) + } + + it("should start routing messages") { + + sut.configurationProvider.updateConfiguration(configurationWithoutRouting) + + val messages = sut.handleConnection(sink, vesWireFrameMessage(PERF3GPP)) + assertThat(messages).isEmpty() + + sut.configurationProvider.updateConfiguration(basicConfiguration) + + val messagesAfterUpdate = sut.handleConnection(sink, vesWireFrameMessage(PERF3GPP)) + assertThat(messagesAfterUpdate).hasSize(1) + val message = messagesAfterUpdate[0] + + assertThat(message.topic).describedAs("routed message topic after configuration's change") + .isEqualTo(PERF3GPP_TOPIC) + assertThat(message.partition).describedAs("routed message partition") + .isEqualTo(0) + } + + it("should change domain routing") { + + val messages = sut.handleConnection(sink, vesWireFrameMessage(PERF3GPP)) + assertThat(messages).hasSize(1) + val firstMessage = messages[0] + + assertThat(firstMessage.topic).describedAs("routed message topic on initial configuration") + .isEqualTo(PERF3GPP_TOPIC) + assertThat(firstMessage.partition).describedAs("routed message partition") + .isEqualTo(0) + + + sut.configurationProvider.updateConfiguration(configurationWithDifferentRouting) + + val messagesAfterUpdate = sut.handleConnection(sink, vesWireFrameMessage(PERF3GPP)) + assertThat(messagesAfterUpdate).hasSize(2) + val secondMessage = messagesAfterUpdate[1] + + assertThat(secondMessage.topic).describedAs("routed message topic after configuration's change") + .isEqualTo(ALTERNATE_PERF3GPP_TOPIC) + assertThat(secondMessage.partition).describedAs("routed message partition") + .isEqualTo(0) + } + + it("should update routing for each client sending one message") { + + val messagesAmount = 10 + val messagesForEachTopic = 5 + + Flux.range(0, messagesAmount).doOnNext { + if (it == messagesForEachTopic) { + sut.configurationProvider.updateConfiguration(configurationWithDifferentRouting) + } + }.doOnNext { + sut.handleConnection(sink, vesWireFrameMessage(PERF3GPP)) + }.then().block(defaultTimeout) + + + val messages = sink.sentMessages + val firstTopicMessagesCount = messages.count { it.topic == PERF3GPP_TOPIC } + val secondTopicMessagesCount = messages.count { it.topic == ALTERNATE_PERF3GPP_TOPIC } + + assertThat(messages.size).isEqualTo(messagesAmount) + assertThat(messagesForEachTopic) + .describedAs("amount of messages routed to each topic") + .isEqualTo(firstTopicMessagesCount) + .isEqualTo(secondTopicMessagesCount) + } + + it("should not update routing for client sending continuous stream of messages") { + + val messageStreamSize = 10 + val pivot = 5 + + val incomingMessages = Flux.range(0, messageStreamSize) + .doOnNext { + if (it == pivot) { + sut.configurationProvider.updateConfiguration(configurationWithDifferentRouting) + println("config changed") + } + } + .map { vesWireFrameMessage(PERF3GPP) } + + + sut.collector.handleConnection(sut.alloc, incomingMessages).block(defaultTimeout) + + val messages = sink.sentMessages + val firstTopicMessagesCount = messages.count { it.topic == PERF3GPP_TOPIC } + val secondTopicMessagesCount = messages.count { it.topic == ALTERNATE_PERF3GPP_TOPIC } + + assertThat(messages.size).isEqualTo(messageStreamSize) + assertThat(firstTopicMessagesCount) + .describedAs("amount of messages routed to first topic") + .isEqualTo(messageStreamSize) + + assertThat(secondTopicMessagesCount) + .describedAs("amount of messages routed to second topic") + .isEqualTo(0) + } + + it("should mark the application healthy") { + assertThat(sut.healthStateProvider.currentHealth) + .describedAs("application health state") + .isEqualTo(HealthDescription.HEALTHY) + } + } + + given("failed configuration change") { + val (sut, _) = vesHvWithStoringSink() + sut.configurationProvider.shouldThrowExceptionOnConfigUpdate(true) + sut.configurationProvider.updateConfiguration(basicConfiguration) + + it("should mark the application unhealthy ") { + assertThat(sut.healthStateProvider.currentHealth) + .describedAs("application health state") + .isEqualTo(HealthDescription.CONSUL_CONFIGURATION_NOT_FOUND) + } + } + } + + describe("request validation") { + it("should reject message with payload greater than 1 MiB and all subsequent messages") { + val (sut, sink) = vesHvWithStoringSink() + + val handledMessages = sut.handleConnection(sink, + vesWireFrameMessage(PERF3GPP, "first"), + vesMessageWithPayloadOfSize(Sut.MAX_PAYLOAD_SIZE_BYTES + 1, PERF3GPP), + vesWireFrameMessage(PERF3GPP)) + + assertThat(handledMessages).hasSize(1) + assertThat(handledMessages.first().message.header.eventId).isEqualTo("first") + } + } + +}) + +private fun vesHvWithStoringSink(): Pair<Sut, StoringSink> { + val sink = StoringSink() + val sut = Sut(sink) + sut.configurationProvider.updateConfiguration(basicConfiguration) + return Pair(sut, sink) +} diff --git a/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/component/spekUtils.kt b/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/component/spekUtils.kt new file mode 100644 index 00000000..29df8c70 --- /dev/null +++ b/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/component/spekUtils.kt @@ -0,0 +1,38 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.tests.component + +import org.jetbrains.spek.api.dsl.SpecBody +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import reactor.core.publisher.Hooks + +fun SpecBody.debugRx(debug: Boolean = true) { + if (debug) { + beforeGroup { + Hooks.onOperatorDebug() + } + + afterGroup { + Hooks.resetOnOperatorDebug() + } + } +} + +val logger = Logger("org.onap.dcae.collectors.veshv.tests.component") diff --git a/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/fakes/FakeHealthState.kt b/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/fakes/FakeHealthState.kt new file mode 100644 index 00000000..c25771b7 --- /dev/null +++ b/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/fakes/FakeHealthState.kt @@ -0,0 +1,37 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.tests.fakes + +import org.onap.dcae.collectors.veshv.healthcheck.api.HealthDescription +import org.onap.dcae.collectors.veshv.healthcheck.api.HealthState +import reactor.core.publisher.Flux + +class FakeHealthState : HealthState { + + lateinit var currentHealth: HealthDescription + + override fun changeState(healthDescription: HealthDescription) { + currentHealth = healthDescription + } + + override fun invoke(): Flux<HealthDescription> { + throw NotImplementedError() + } +} diff --git a/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/fakes/configuration.kt b/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/fakes/configuration.kt new file mode 100644 index 00000000..3770913a --- /dev/null +++ b/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/fakes/configuration.kt @@ -0,0 +1,106 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.tests.fakes + +import org.onap.dcae.collectors.veshv.boundary.ConfigurationProvider +import org.onap.dcae.collectors.veshv.domain.VesEventDomain.PERF3GPP +import org.onap.dcae.collectors.veshv.domain.VesEventDomain.HEARTBEAT +import org.onap.dcae.collectors.veshv.domain.VesEventDomain.MEASUREMENT +import org.onap.dcae.collectors.veshv.model.CollectorConfiguration +import org.onap.dcae.collectors.veshv.model.routing + +import reactor.core.publisher.FluxProcessor +import reactor.core.publisher.UnicastProcessor +import reactor.retry.RetryExhaustedException + + +const val PERF3GPP_TOPIC = "HV_VES_PERF3GPP" +const val MEASUREMENTS_FOR_VF_SCALING_TOPIC = "HV_VES_MEAS_FOR_VF_SCALING" +const val ALTERNATE_PERF3GPP_TOPIC = "HV_VES_PERF3GPP_ALTERNATIVE" + +val basicConfiguration: CollectorConfiguration = CollectorConfiguration( + kafkaBootstrapServers = "localhost:9969", + routing = routing { + defineRoute { + fromDomain(PERF3GPP.domainName) + toTopic(PERF3GPP_TOPIC) + withFixedPartitioning() + } + }.build() +) + +val twoDomainsToOneTopicConfiguration: CollectorConfiguration = CollectorConfiguration( + kafkaBootstrapServers = "localhost:9969", + routing = routing { + defineRoute { + fromDomain(PERF3GPP.domainName) + toTopic(PERF3GPP_TOPIC) + withFixedPartitioning() + } + defineRoute { + fromDomain(HEARTBEAT.domainName) + toTopic(PERF3GPP_TOPIC) + withFixedPartitioning() + } + defineRoute { + fromDomain(MEASUREMENT.domainName) + toTopic(MEASUREMENTS_FOR_VF_SCALING_TOPIC) + withFixedPartitioning() + } + }.build() +) + + +val configurationWithDifferentRouting: CollectorConfiguration = CollectorConfiguration( + kafkaBootstrapServers = "localhost:9969", + routing = routing { + defineRoute { + fromDomain(PERF3GPP.domainName) + toTopic(ALTERNATE_PERF3GPP_TOPIC) + withFixedPartitioning() + } + }.build() +) + + +val configurationWithoutRouting: CollectorConfiguration = CollectorConfiguration( + kafkaBootstrapServers = "localhost:9969", + routing = routing { + }.build() +) + +class FakeConfigurationProvider : ConfigurationProvider { + private var shouldThrowException = false + private val configStream: FluxProcessor<CollectorConfiguration, CollectorConfiguration> = UnicastProcessor.create() + + fun updateConfiguration(collectorConfiguration: CollectorConfiguration) = + if (shouldThrowException) { + configStream.onError(RetryExhaustedException("I'm so tired")) + } else { + configStream.onNext(collectorConfiguration) + } + + + fun shouldThrowExceptionOnConfigUpdate(shouldThrowException: Boolean) { + this.shouldThrowException = shouldThrowException + } + + override fun invoke() = configStream +} diff --git a/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/fakes/metrics.kt b/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/fakes/metrics.kt new file mode 100644 index 00000000..aa3fdc39 --- /dev/null +++ b/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/fakes/metrics.kt @@ -0,0 +1,37 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.tests.fakes + +import org.onap.dcae.collectors.veshv.boundary.Metrics + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ +class FakeMetrics: Metrics { + override fun notifyBytesReceived(size: Int) { + } + + override fun notifyMessageReceived(size: Int) { + } + + override fun notifyMessageSent(topic: String) { + } +}
\ No newline at end of file diff --git a/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/fakes/sink.kt b/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/fakes/sink.kt new file mode 100644 index 00000000..a5fd546a --- /dev/null +++ b/sources/hv-collector-ct/src/test/kotlin/org/onap/dcae/collectors/veshv/tests/fakes/sink.kt @@ -0,0 +1,59 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.tests.fakes + +import org.onap.dcae.collectors.veshv.boundary.Sink +import org.onap.dcae.collectors.veshv.model.RoutedMessage +import reactor.core.publisher.Flux +import java.util.* +import java.util.concurrent.ConcurrentLinkedDeque +import java.util.concurrent.atomic.AtomicLong + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +class StoringSink : Sink { + private val sent: Deque<RoutedMessage> = ConcurrentLinkedDeque() + + val sentMessages: List<RoutedMessage> + get() = sent.toList() + + override fun send(messages: Flux<RoutedMessage>): Flux<RoutedMessage> { + return messages.doOnNext(sent::addLast) + } +} + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +class CountingSink : Sink { + private val atomicCount = AtomicLong(0) + + val count: Long + get() = atomicCount.get() + + override fun send(messages: Flux<RoutedMessage>): Flux<RoutedMessage> { + return messages.doOnNext { + atomicCount.incrementAndGet() + } + } +} diff --git a/sources/hv-collector-ct/src/test/resources/logback-test.xml b/sources/hv-collector-ct/src/test/resources/logback-test.xml new file mode 100644 index 00000000..93f22771 --- /dev/null +++ b/sources/hv-collector-ct/src/test/resources/logback-test.xml @@ -0,0 +1,35 @@ +<?xml version="1.0" encoding="UTF-8"?> +<configuration> + <property name="LOG_FILE" + value="${LOG_FILE:-${LOG_PATH:-${LOG_TEMP:-${java.io.tmpdir:-/tmp}}/}ves-hv.log}"/> + <property name="FILE_LOG_PATTERN" value="%d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %-5level [%-40.40logger{10}] - %msg%n"/> + + <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> + <encoder> + <pattern> + %d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %highlight(%-5level) [%-40.40logger{10}] - %msg%n + </pattern> + </encoder> + </appender> + + <appender name="ROLLING-FILE" + class="ch.qos.logback.core.rolling.RollingFileAppender"> + <encoder> + <pattern>${FILE_LOG_PATTERN}</pattern> + </encoder> + <file>${LOG_FILE}</file> + <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy"> + <fileNamePattern>${LOG_FILE}.%d{yyyy-MM-dd}.log</fileNamePattern> + <maxFileSize>50MB</maxFileSize> + <maxHistory>30</maxHistory> + <totalSizeCap>10GB</totalSizeCap> + </rollingPolicy> + </appender> + + <logger name="org.onap.dcae.collectors.veshv" level="DEBUG"/> + + <root level="INFO"> + <appender-ref ref="CONSOLE"/> + <appender-ref ref="ROLLING-FILE"/> + </root> +</configuration> diff --git a/sources/hv-collector-dcae-app-simulator/Dockerfile b/sources/hv-collector-dcae-app-simulator/Dockerfile new file mode 100644 index 00000000..a561fff7 --- /dev/null +++ b/sources/hv-collector-dcae-app-simulator/Dockerfile @@ -0,0 +1,18 @@ +FROM docker.io/openjdk:11-jre-slim + +LABEL copyright="Copyright (C) 2018 NOKIA" +LABEL license.name="The Apache Software License, Version 2.0" +LABEL license.url="http://www.apache.org/licenses/LICENSE-2.0" +LABEL maintainer="Nokia Wroclaw ONAP Team" + +RUN apt-get update \ + && apt-get install -y --no-install-recommends curl \ + && apt-get clean + +WORKDIR /opt/ves-hv-dcae-app-simulator + +ENTRYPOINT ["java", "-cp", "*:", "org.onap.dcae.collectors.veshv.simulators.dcaeapp.MainKt"] + +COPY target/libs/external/* ./ +COPY target/libs/internal/* ./ +COPY target/hv-collector-dcae-app-simulator-*.jar ./ diff --git a/sources/hv-collector-dcae-app-simulator/pom.xml b/sources/hv-collector-dcae-app-simulator/pom.xml new file mode 100644 index 00000000..c34e885d --- /dev/null +++ b/sources/hv-collector-dcae-app-simulator/pom.xml @@ -0,0 +1,155 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ ============LICENSE_START======================================================= + ~ dcaegen2-collectors-veshv + ~ ================================================================================ + ~ Copyright (C) 2018 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========================================================= + --> +<project xmlns="http://maven.apache.org/POM/4.0.0" + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + + <licenses> + <license> + <name>The Apache Software License, Version 2.0</name> + <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url> + </license> + </licenses> + + <parent> + <groupId>org.onap.dcaegen2.collectors.hv-ves</groupId> + <artifactId>hv-collector-sources</artifactId> + <version>1.1.0-SNAPSHOT</version> + <relativePath>..</relativePath> + </parent> + + <artifactId>hv-collector-dcae-app-simulator</artifactId> + <description>VES HighVolume Collector :: Dcae app simulator</description> + + <properties> + <skipAnalysis>false</skipAnalysis> + </properties> + + <build> + <plugins> + <plugin> + <artifactId>kotlin-maven-plugin</artifactId> + <groupId>org.jetbrains.kotlin</groupId> + </plugin> + <plugin> + <artifactId>maven-surefire-plugin</artifactId> + <groupId>org.apache.maven.plugins</groupId> + </plugin> + </plugins> + </build> + <profiles> + <profile> + <id>docker</id> + <activation> + <property> + <name>!skipDocker</name> + </property> + </activation> + <build> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-dependency-plugin</artifactId> + </plugin> + <plugin> + <groupId>io.fabric8</groupId> + <artifactId>docker-maven-plugin</artifactId> + </plugin> + </plugins> + </build> + </profile> + </profiles> + <dependencies> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-domain</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-utils</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-ves-message-generator</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-test-utils</artifactId> + <version>${project.parent.version}</version> + <scope>test</scope> + </dependency> + <dependency> + <groupId>io.arrow-kt</groupId> + <artifactId>arrow-effects</artifactId> + </dependency> + <dependency> + <groupId>io.arrow-kt</groupId> + <artifactId>arrow-effects-instances</artifactId> + </dependency> + <dependency> + <groupId>io.arrow-kt</groupId> + <artifactId>arrow-effects-reactor</artifactId> + </dependency> + <dependency> + <groupId>io.arrow-kt</groupId> + <artifactId>arrow-syntax</artifactId> + </dependency> + <dependency> + <groupId>io.ratpack</groupId> + <artifactId>ratpack-core</artifactId> + </dependency> + <dependency> + <groupId>io.projectreactor.kafka</groupId> + <artifactId>reactor-kafka</artifactId> + </dependency> + <dependency> + <groupId>com.google.protobuf</groupId> + <artifactId>protobuf-java-util</artifactId> + </dependency> + <dependency> + <groupId>commons-cli</groupId> + <artifactId>commons-cli</artifactId> + </dependency> + <dependency> + <groupId>org.slf4j</groupId> + <artifactId>slf4j-api</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.kotlin</groupId> + <artifactId>kotlin-stdlib-jdk8</artifactId> + </dependency> + <dependency> + <groupId>org.assertj</groupId> + <artifactId>assertj-core</artifactId> + </dependency> + <dependency> + <groupId>ch.qos.logback</groupId> + <artifactId>logback-classic</artifactId> + <scope>runtime</scope> + </dependency> + </dependencies> + + +</project>
\ No newline at end of file diff --git a/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/DcaeAppSimulator.kt b/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/DcaeAppSimulator.kt new file mode 100644 index 00000000..490cde4a --- /dev/null +++ b/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/DcaeAppSimulator.kt @@ -0,0 +1,74 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.dcaeapp.impl + +import arrow.core.getOrElse +import arrow.effects.IO +import arrow.effects.fix +import arrow.effects.instances.io.monadError.monadError +import arrow.typeclasses.bindingCatch +import org.onap.dcae.collectors.veshv.utils.arrow.getOption +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import java.io.InputStream +import java.util.concurrent.atomic.AtomicReference + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since August 2018 + */ +class DcaeAppSimulator(private val consumerFactory: ConsumerFactory, + private val messageStreamValidation: MessageStreamValidation) { + private val consumerState: AtomicReference<ConsumerStateProvider> = AtomicReference() + + fun listenToTopics(topicsString: String) = listenToTopics(extractTopics(topicsString)) + + fun listenToTopics(topics: Set<String>): IO<Unit> = IO.monadError().bindingCatch { + if (topics.any { it.isBlank() }) + throw IllegalArgumentException("Topic list cannot contain empty elements") + if (topics.isEmpty()) + throw IllegalArgumentException("Topic list cannot be empty") + + logger.info("Received new configuration. Creating consumer for topics: $topics") + consumerState.set(consumerFactory.createConsumerForTopics(topics).bind()) + }.fix() + + fun state() = consumerState.getOption().map { it.currentState() } + + fun resetState(): IO<Unit> = consumerState.getOption().fold( + { IO.unit }, + { it.reset() } + ) + + fun validate(jsonDescription: InputStream) = messageStreamValidation.validate(jsonDescription, currentMessages()) + + private fun currentMessages(): List<ByteArray> = + consumerState.getOption() + .map { it.currentState().consumedMessages } + .getOrElse(::emptyList) + + private fun extractTopics(topicsString: String): Set<String> = + topicsString.substringAfter("=") + .split(",") + .toSet() + + companion object { + private val logger = Logger(DcaeAppSimulator::class) + } +} diff --git a/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/MessageStreamValidation.kt b/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/MessageStreamValidation.kt new file mode 100644 index 00000000..e423191d --- /dev/null +++ b/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/MessageStreamValidation.kt @@ -0,0 +1,88 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.dcaeapp.impl + +import arrow.effects.IO +import arrow.effects.fix +import arrow.effects.instances.io.monadError.monadError +import arrow.typeclasses.bindingCatch +import org.onap.dcae.collectors.veshv.domain.ByteData +import org.onap.dcae.collectors.veshv.domain.WireFrameMessage +import org.onap.dcae.collectors.veshv.utils.arrow.asIo +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageGenerator +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageParameters +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageParametersParser +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageType +import org.onap.ves.VesEventOuterClass +import java.io.InputStream +import javax.json.Json + +class MessageStreamValidation( + private val messageGenerator: MessageGenerator, + private val messageParametersParser: MessageParametersParser = MessageParametersParser.INSTANCE) { + + fun validate(jsonDescription: InputStream, consumedMessages: List<ByteArray>): IO<Boolean> = + IO.monadError().bindingCatch { + val messageParams = parseMessageParams(jsonDescription) + val expectedEvents = generateEvents(messageParams).bind() + val actualEvents = decodeConsumedEvents(consumedMessages) + if (shouldValidatePayloads(messageParams)) { + expectedEvents == actualEvents + } else { + validateHeaders(actualEvents, expectedEvents) + } + }.fix() + + private fun parseMessageParams(input: InputStream): List<MessageParameters> { + val expectations = Json.createReader(input).readArray() + val messageParams = messageParametersParser.parse(expectations) + + return messageParams.fold( + { throw IllegalArgumentException("Parsing error: " + it.message) }, + { + if (it.isEmpty()) + throw IllegalArgumentException("Message param list cannot be empty") + it + } + ) + } + + private fun shouldValidatePayloads(parameters: List<MessageParameters>) = + parameters.all { it.messageType == MessageType.FIXED_PAYLOAD } + + private fun validateHeaders(actual: List<VesEventOuterClass.VesEvent>, + expected: List<VesEventOuterClass.VesEvent>): Boolean { + val consumedHeaders = actual.map { it.commonEventHeader } + val generatedHeaders = expected.map { it.commonEventHeader } + return generatedHeaders == consumedHeaders + } + + private fun generateEvents(parameters: List<MessageParameters>): IO<List<VesEventOuterClass.VesEvent>> = + messageGenerator.createMessageFlux(parameters) + .map(WireFrameMessage::payload) + .map(ByteData::unsafeAsArray) + .map(VesEventOuterClass.VesEvent::parseFrom) + .collectList() + .asIo() + + private fun decodeConsumedEvents(consumedMessages: List<ByteArray>) = + consumedMessages.map(VesEventOuterClass.VesEvent::parseFrom) + +} diff --git a/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/adapters/DcaeAppApiServer.kt b/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/adapters/DcaeAppApiServer.kt new file mode 100644 index 00000000..1eca9317 --- /dev/null +++ b/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/adapters/DcaeAppApiServer.kt @@ -0,0 +1,101 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.dcaeapp.impl.adapters + +import arrow.effects.IO +import org.onap.dcae.collectors.veshv.simulators.dcaeapp.impl.DcaeAppSimulator +import org.onap.dcae.collectors.veshv.utils.http.HttpConstants +import org.onap.dcae.collectors.veshv.utils.http.HttpStatus +import org.onap.dcae.collectors.veshv.utils.http.Responses +import org.onap.dcae.collectors.veshv.utils.http.sendAndHandleErrors +import org.onap.dcae.collectors.veshv.utils.http.sendOrError +import ratpack.handling.Chain +import ratpack.server.RatpackServer +import ratpack.server.ServerConfig + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +class DcaeAppApiServer(private val simulator: DcaeAppSimulator) { + private val responseValid by lazy { + Responses.statusResponse( + name = "valid", + message = "validation succeeded" + ) + } + + private val responseInvalid by lazy { + Responses.statusResponse( + name = "invalid", + message = "validation failed", + httpStatus = HttpStatus.BAD_REQUEST + ) + } + + + fun start(port: Int, kafkaTopics: Set<String>): IO<RatpackServer> = + simulator.listenToTopics(kafkaTopics).map { + RatpackServer.start { server -> + server.serverConfig(ServerConfig.embedded().port(port)) + .handlers(::setupHandlers) + } + } + + private fun setupHandlers(chain: Chain) { + chain + .put("configuration/topics") { ctx -> + ctx.request.body.then { body -> + val operation = simulator.listenToTopics(body.text) + ctx.response.sendOrError(operation) + } + + } + .delete("messages") { ctx -> + ctx.response.contentType(CONTENT_TEXT) + ctx.response.sendOrError(simulator.resetState()) + } + .get("messages/all/count") { ctx -> + simulator.state().fold( + { ctx.response.status(HttpConstants.STATUS_NOT_FOUND) }, + { + ctx.response + .contentType(CONTENT_TEXT) + .send(it.messagesCount.toString()) + }) + } + .post("messages/all/validate") { ctx -> + ctx.request.body.then { body -> + val response = simulator.validate(body.inputStream) + .map { isValid -> + if (isValid) responseValid else responseInvalid + } + ctx.response.sendAndHandleErrors(response) + } + } + .get("healthcheck") { ctx -> + ctx.response.status(HttpConstants.STATUS_OK).send() + } + } + + companion object { + private const val CONTENT_TEXT = "text/plain" + } +} diff --git a/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/adapters/KafkaSource.kt b/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/adapters/KafkaSource.kt new file mode 100644 index 00000000..10dedbdf --- /dev/null +++ b/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/adapters/KafkaSource.kt @@ -0,0 +1,66 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.dcaeapp.impl.adapters + +import arrow.effects.IO +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.common.serialization.ByteArrayDeserializer +import org.onap.dcae.collectors.veshv.simulators.dcaeapp.impl.Consumer +import org.onap.dcae.collectors.veshv.utils.arrow.evaluateIo +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import reactor.kafka.receiver.KafkaReceiver +import reactor.kafka.receiver.ReceiverOptions + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +class KafkaSource(private val receiver: KafkaReceiver<ByteArray, ByteArray>) { + + fun start(): IO<Consumer> = IO { + val consumer = Consumer() + receiver.receive().map(consumer::update).evaluateIo().subscribe() + consumer + } + + companion object { + private val logger = Logger(KafkaSource::class) + + fun create(bootstrapServers: String, topics: Set<String>): KafkaSource { + return KafkaSource(KafkaReceiver.create(createReceiverOptions(bootstrapServers, topics))) + } + + fun createReceiverOptions(bootstrapServers: String, + topics: Set<String>): ReceiverOptions<ByteArray, ByteArray>? { + val props = mapOf<String, Any>( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG to bootstrapServers, + ConsumerConfig.CLIENT_ID_CONFIG to "hv-collector-dcae-app-simulator", + ConsumerConfig.GROUP_ID_CONFIG to "hv-collector-simulators", + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG to ByteArrayDeserializer::class.java, + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG to ByteArrayDeserializer::class.java, + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG to "earliest" + ) + return ReceiverOptions.create<ByteArray, ByteArray>(props) + .addAssignListener { partitions -> logger.debug { "Partitions assigned $partitions" } } + .addRevokeListener { partitions -> logger.debug { "Partitions revoked $partitions" } } + .subscription(topics) + } + } +} diff --git a/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/config/ArgDcaeAppSimConfiguration.kt b/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/config/ArgDcaeAppSimConfiguration.kt new file mode 100644 index 00000000..17eeb5b1 --- /dev/null +++ b/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/config/ArgDcaeAppSimConfiguration.kt @@ -0,0 +1,67 @@ +/* +* ============LICENSE_START======================================================= +* dcaegen2-collectors-veshv +* ================================================================================ +* Copyright (C) 2018 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.simulators.dcaeapp.impl.config + +import arrow.core.Option +import arrow.core.fix +import arrow.instances.option.monad.monad +import arrow.typeclasses.binding +import org.apache.commons.cli.CommandLine +import org.apache.commons.cli.DefaultParser +import org.onap.dcae.collectors.veshv.domain.WireFrameMessage +import org.onap.dcae.collectors.veshv.utils.commandline.ArgBasedConfiguration +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.KAFKA_SERVERS +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.KAFKA_TOPICS +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.LISTEN_PORT +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.MAXIMUM_PAYLOAD_SIZE_BYTES +import org.onap.dcae.collectors.veshv.utils.commandline.intValue +import org.onap.dcae.collectors.veshv.utils.commandline.stringValue + +class ArgDcaeAppSimConfiguration : ArgBasedConfiguration<DcaeAppSimConfiguration>(DefaultParser()) { + override val cmdLineOptionsList: List<CommandLineOption> = listOf( + LISTEN_PORT, + MAXIMUM_PAYLOAD_SIZE_BYTES, + KAFKA_SERVERS, + KAFKA_TOPICS + ) + + override fun getConfiguration(cmdLine: CommandLine): Option<DcaeAppSimConfiguration> = + Option.monad().binding { + val listenPort = cmdLine + .intValue(LISTEN_PORT) + .bind() + val maxPayloadSizeBytes = cmdLine + .intValue(MAXIMUM_PAYLOAD_SIZE_BYTES, WireFrameMessage.DEFAULT_MAX_PAYLOAD_SIZE_BYTES) + val kafkaBootstrapServers = cmdLine + .stringValue(KAFKA_SERVERS) + .bind() + val kafkaTopics = cmdLine + .stringValue(KAFKA_TOPICS) + .map { it.split(",").toSet() } + .bind() + + DcaeAppSimConfiguration( + listenPort, + maxPayloadSizeBytes, + kafkaBootstrapServers, + kafkaTopics) + }.fix() +} diff --git a/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/config/DcaeAppSimConfiguration.kt b/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/config/DcaeAppSimConfiguration.kt new file mode 100644 index 00000000..a6fc8053 --- /dev/null +++ b/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/config/DcaeAppSimConfiguration.kt @@ -0,0 +1,27 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.dcaeapp.impl.config + +data class DcaeAppSimConfiguration( + val apiPort: Int, + val maxPayloadSizeBytes: Int, + val kafkaBootstrapServers: String, + val kafkaTopics: Set<String> +) diff --git a/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/consumer.kt b/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/consumer.kt new file mode 100644 index 00000000..1eefdbdb --- /dev/null +++ b/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/consumer.kt @@ -0,0 +1,70 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.dcaeapp.impl + +import arrow.effects.IO +import org.onap.dcae.collectors.veshv.simulators.dcaeapp.impl.adapters.KafkaSource +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import reactor.kafka.receiver.ReceiverRecord +import java.util.concurrent.ConcurrentLinkedQueue + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ +class ConsumerState(private val messages: ConcurrentLinkedQueue<ByteArray>) { + val messagesCount: Int by lazy { + messages.size + } + + val consumedMessages: List<ByteArray> by lazy { + messages.toList() + } +} + +interface ConsumerStateProvider { + fun currentState(): ConsumerState + fun reset(): IO<Unit> +} + +class Consumer : ConsumerStateProvider { + + private var consumedMessages: ConcurrentLinkedQueue<ByteArray> = ConcurrentLinkedQueue() + + override fun currentState(): ConsumerState = ConsumerState(consumedMessages) + + override fun reset(): IO<Unit> = IO { + consumedMessages.clear() + } + + fun update(record: ReceiverRecord<ByteArray, ByteArray>) = IO<Unit> { + logger.trace { "Updating stats for message from ${record.topic()}:${record.partition()}" } + consumedMessages.add(record.value()) + } + + companion object { + private val logger = Logger(Consumer::class) + } +} + +class ConsumerFactory(private val kafkaBootstrapServers: String) { + fun createConsumerForTopics(kafkaTopics: Set<String>): IO<Consumer> = + KafkaSource.create(kafkaBootstrapServers, kafkaTopics.toSet()).start() +} diff --git a/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/main.kt b/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/main.kt new file mode 100644 index 00000000..06ff4d59 --- /dev/null +++ b/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/main.kt @@ -0,0 +1,62 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.dcaeapp + +import arrow.effects.IO +import org.onap.dcae.collectors.veshv.simulators.dcaeapp.impl.config.ArgDcaeAppSimConfiguration +import org.onap.dcae.collectors.veshv.simulators.dcaeapp.impl.config.DcaeAppSimConfiguration +import org.onap.dcae.collectors.veshv.simulators.dcaeapp.impl.ConsumerFactory +import org.onap.dcae.collectors.veshv.simulators.dcaeapp.impl.DcaeAppSimulator +import org.onap.dcae.collectors.veshv.simulators.dcaeapp.impl.MessageStreamValidation +import org.onap.dcae.collectors.veshv.simulators.dcaeapp.impl.adapters.DcaeAppApiServer +import org.onap.dcae.collectors.veshv.utils.arrow.ExitFailure +import org.onap.dcae.collectors.veshv.utils.arrow.unsafeRunEitherSync +import org.onap.dcae.collectors.veshv.utils.arrow.unit +import org.onap.dcae.collectors.veshv.utils.commandline.handleWrongArgumentErrorCurried +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import org.onap.dcae.collectors.veshv.ves.message.generator.factory.MessageGeneratorFactory + +private const val PACKAGE_NAME = "org.onap.dcae.collectors.veshv.simulators.dcaeapp" +private val logger = Logger(PACKAGE_NAME) +const val PROGRAM_NAME = "java $PACKAGE_NAME.MainKt" + +fun main(args: Array<String>) = + ArgDcaeAppSimConfiguration().parse(args) + .mapLeft(handleWrongArgumentErrorCurried(PROGRAM_NAME)) + .map(::startApp) + .unsafeRunEitherSync( + { ex -> + logger.error("Failed to start a server", ex) + ExitFailure(1) + }, + { + logger.info("Started DCAE-APP Simulator API server") + } + ) + + +private fun startApp(config: DcaeAppSimConfiguration): IO<Unit> { + logger.info("Using configuration: $config") + val consumerFactory = ConsumerFactory(config.kafkaBootstrapServers) + val messageStreamValidation = MessageStreamValidation(MessageGeneratorFactory.create(config.maxPayloadSizeBytes)) + return DcaeAppApiServer(DcaeAppSimulator(consumerFactory, messageStreamValidation)) + .start(config.apiPort, config.kafkaTopics) + .unit() +} diff --git a/sources/hv-collector-dcae-app-simulator/src/main/resources/logback.xml b/sources/hv-collector-dcae-app-simulator/src/main/resources/logback.xml new file mode 100644 index 00000000..48da3b18 --- /dev/null +++ b/sources/hv-collector-dcae-app-simulator/src/main/resources/logback.xml @@ -0,0 +1,36 @@ +<?xml version="1.0" encoding="UTF-8"?> +<configuration> + <property name="LOG_FILE" + value="${LOG_FILE:-${LOG_PATH:-${LOG_TEMP:-${java.io.tmpdir:-/tmp}}/}ves-hv.log}"/> + <property name="FILE_LOG_PATTERN" value="%d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %-5level [%-40.40logger{10}] - %msg%n"/> + + <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> + <encoder> + <pattern> + %d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %highlight(%-5level) [%-40.40logger{10}] - %msg%n + </pattern> + </encoder> + </appender> + + <appender name="ROLLING-FILE" + class="ch.qos.logback.core.rolling.RollingFileAppender"> + <encoder> + <pattern>${FILE_LOG_PATTERN}</pattern> + </encoder> + <file>${LOG_FILE}</file> + <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy"> + <fileNamePattern>${LOG_FILE}.%d{yyyy-MM-dd}.log</fileNamePattern> + <maxFileSize>50MB</maxFileSize> + <maxHistory>30</maxHistory> + <totalSizeCap>10GB</totalSizeCap> + </rollingPolicy> + </appender> + + <logger name="org.onap.dcae.collectors.veshv" level="INFO"/> + <!--<logger name="reactor.ipc.netty" level="DEBUG"/>--> + + <root level="INFO"> + <appender-ref ref="CONSOLE"/> + <appender-ref ref="ROLLING-FILE"/> + </root> +</configuration>
\ No newline at end of file diff --git a/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/ConsumerTest.kt b/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/ConsumerTest.kt new file mode 100644 index 00000000..08558d76 --- /dev/null +++ b/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/ConsumerTest.kt @@ -0,0 +1,82 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.dcaeapp.impl + +import org.apache.kafka.clients.consumer.ConsumerRecord +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 reactor.kafka.receiver.ReceiverRecord + + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since August 2018 + */ +internal class ConsumerTest : Spek({ + + lateinit var cut: Consumer + + beforeEachTest { + cut = Consumer() + } + + describe("Consumer which holds the state of received Kafka records") { + it("should contain empty state in the beginning") { + assertEmptyState(cut) + } + + describe("update") { + val value = byteArrayOf(2) + + beforeEachTest { + cut.update(receiverRecord( + topic = "topic", + key = byteArrayOf(1), + value = value + )).unsafeRunSync() + } + + it("should contain one message if it was updated once") { + assertState(cut, value) + } + + it("should contain empty state message if it was reset after update") { + cut.reset().unsafeRunSync() + assertEmptyState(cut) + } + } + } +}) + +fun assertEmptyState(cut: Consumer) { + assertState(cut) +} + +fun assertState(cut: Consumer, vararg values: ByteArray) { + assertThat(cut.currentState().consumedMessages) + .containsOnly(*values) + assertThat(cut.currentState().messagesCount) + .isEqualTo(values.size) +} + +fun receiverRecord(topic: String, key: ByteArray, value: ByteArray) = + ReceiverRecord(ConsumerRecord(topic, 1, 100L, key, value), null) diff --git a/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/DcaeAppSimulatorTest.kt b/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/DcaeAppSimulatorTest.kt new file mode 100644 index 00000000..e1641cbb --- /dev/null +++ b/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/DcaeAppSimulatorTest.kt @@ -0,0 +1,183 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.dcaeapp.impl + +import arrow.core.Left +import arrow.core.None +import arrow.core.Some +import arrow.effects.IO +import com.google.protobuf.ByteString +import com.nhaarman.mockitokotlin2.any +import com.nhaarman.mockitokotlin2.eq +import com.nhaarman.mockitokotlin2.mock +import com.nhaarman.mockitokotlin2.never +import com.nhaarman.mockitokotlin2.verify +import com.nhaarman.mockitokotlin2.whenever +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.mockito.ArgumentMatchers.anySet +import org.onap.ves.VesEventOuterClass.CommonEventHeader +import org.onap.ves.VesEventOuterClass.VesEvent +import java.util.concurrent.ConcurrentLinkedQueue + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since August 2018 + */ +internal class DcaeAppSimulatorTest : Spek({ + lateinit var consumerFactory: ConsumerFactory + lateinit var messageStreamValidation: MessageStreamValidation + lateinit var consumer: Consumer + lateinit var cut: DcaeAppSimulator + + beforeEachTest { + consumerFactory = mock() + messageStreamValidation = mock() + consumer = mock() + cut = DcaeAppSimulator(consumerFactory, messageStreamValidation) + + whenever(consumerFactory.createConsumerForTopics(anySet())).thenReturn(IO.just(consumer)) + } + + fun consumerState(vararg messages: ByteArray) = ConsumerState(ConcurrentLinkedQueue(messages.toList())) + + describe("listenToTopics") { + val topics = setOf("perf3gpp", "faults") + + it("should fail when topic list is empty") { + val result = cut.listenToTopics(setOf()).attempt().unsafeRunSync() + assertThat(result.isLeft()).isTrue() + } + + it("should fail when topic list contains empty strings") { + val result = cut.listenToTopics(setOf("perf3gpp", " ", "faults")).attempt().unsafeRunSync() + assertThat(result.isLeft()).isTrue() + } + + it("should subscribe to given topics") { + cut.listenToTopics(topics).unsafeRunSync() + verify(consumerFactory).createConsumerForTopics(topics) + } + + it("should subscribe to given topics when called with comma separated list") { + cut.listenToTopics("perf3gpp,faults").unsafeRunSync() + verify(consumerFactory).createConsumerForTopics(topics) + } + + it("should handle errors") { + // given + val error = RuntimeException("WTF") + whenever(consumerFactory.createConsumerForTopics(anySet())) + .thenReturn(IO.raiseError(error)) + + // when + val result = cut.listenToTopics("perf3gpp").attempt().unsafeRunSync() + + // then + assertThat(result).isEqualTo(Left(error)) + } + } + + describe("state") { + + it("should return None when topics hasn't been initialized") { + assertThat(cut.state()).isEqualTo(None) + } + + describe("when topics are initialized") { + beforeEachTest { + cut.listenToTopics("perf3gpp").unsafeRunSync() + } + + it("should return some state when it has been set") { + val state = consumerState() + whenever(consumer.currentState()).thenReturn(state) + + assertThat(cut.state()).isEqualTo(Some(state)) + } + } + } + + describe("resetState") { + it("should do nothing when topics hasn't been initialized") { + cut.resetState().unsafeRunSync() + verify(consumer, never()).reset() + } + + describe("when topics are initialized") { + beforeEachTest { + cut.listenToTopics("perf3gpp").unsafeRunSync() + } + + it("should reset the state") { + // given + whenever(consumer.reset()).thenReturn(IO.unit) + + // when + cut.resetState().unsafeRunSync() + + // then + verify(consumer).reset() + } + } + } + + describe("validate") { + beforeEachTest { + whenever(messageStreamValidation.validate(any(), any())).thenReturn(IO.just(true)) + } + + it("should use empty list when consumer is unavailable") { + // when + val result = cut.validate("['The JSON']".byteInputStream()).unsafeRunSync() + + // then + verify(messageStreamValidation).validate(any(), eq(emptyList())) + assertThat(result).isTrue() + } + + it("should delegate to MessageStreamValidation") { + // given + cut.listenToTopics("perf3gpp").unsafeRunSync() + whenever(consumer.currentState()).thenReturn(consumerState(vesEvent().toByteArray())) + + // when + val result = cut.validate("['The JSON']".byteInputStream()).unsafeRunSync() + + // then + verify(messageStreamValidation).validate(any(), any()) + assertThat(result).isTrue() + } + } +}) + + +private const val DUMMY_EVENT_ID = "aaa" +private const val DUMMY_PAYLOAD = "payload" + +private fun vesEvent(eventId: String = DUMMY_EVENT_ID, payload: String = DUMMY_PAYLOAD): VesEvent { + return VesEvent.newBuilder() + .setCommonEventHeader(CommonEventHeader.newBuilder() + .setEventId(eventId)) + .setEventFields(ByteString.copyFrom(payload.toByteArray())) + .build() +} diff --git a/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/MessageStreamValidationTest.kt b/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/MessageStreamValidationTest.kt new file mode 100644 index 00000000..a631be76 --- /dev/null +++ b/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/MessageStreamValidationTest.kt @@ -0,0 +1,214 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.dcaeapp.impl + +import arrow.core.Either +import arrow.core.Right +import com.google.protobuf.ByteString +import com.nhaarman.mockitokotlin2.any +import com.nhaarman.mockitokotlin2.mock +import com.nhaarman.mockitokotlin2.whenever +import org.assertj.core.api.Assertions.assertThat +import org.assertj.core.api.Assertions.fail +import org.jetbrains.spek.api.Spek +import org.jetbrains.spek.api.dsl.describe +import org.jetbrains.spek.api.dsl.it +import org.mockito.ArgumentMatchers.anyList +import org.onap.dcae.collectors.veshv.domain.WireFrameMessage +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageGenerator +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageParameters +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageParametersParser +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageType +import org.onap.ves.VesEventOuterClass.CommonEventHeader +import org.onap.ves.VesEventOuterClass.VesEvent +import reactor.core.publisher.Flux +import javax.json.stream.JsonParsingException + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since August 2018 + */ +internal class MessageStreamValidationTest : Spek({ + lateinit var messageParametersParser: MessageParametersParser + lateinit var messageGenerator: MessageGenerator + lateinit var cut: MessageStreamValidation + + beforeEachTest { + messageParametersParser = mock() + messageGenerator = mock() + cut = MessageStreamValidation(messageGenerator, messageParametersParser) + } + + fun givenParsedMessageParameters(vararg params: MessageParameters) { + whenever(messageParametersParser.parse(any())).thenReturn(Right(params.toList())) + } + + describe("validate") { + + it("should return error when JSON is invalid") { + // when + val result = cut.validate("[{invalid json}]".byteInputStream(), listOf()).attempt().unsafeRunSync() + + // then + when(result) { + is Either.Left -> assertThat(result.a).isInstanceOf(JsonParsingException::class.java) + else -> fail("validation should fail") + } + } + + it("should return error when message param list is empty") { + // given + givenParsedMessageParameters() + + // when + val result = cut.validate(sampleJsonAsStream(), listOf()).attempt().unsafeRunSync() + + // then + assertThat(result.isLeft()).isTrue() + } + + describe("when validating headers only") { + it("should return true when messages are the same") { + // given + val jsonAsStream = sampleJsonAsStream() + val event = vesEvent() + val generatedWireProtocolFrame = WireFrameMessage(event.toByteArray()) + val receivedMessageBytes = event.toByteArray() + + givenParsedMessageParameters(MessageParameters(event.commonEventHeader, MessageType.VALID, 1)) + whenever(messageGenerator.createMessageFlux(anyList())).thenReturn(Flux.just(generatedWireProtocolFrame)) + + // when + val result = cut.validate(jsonAsStream, listOf(receivedMessageBytes)).unsafeRunSync() + + // then + assertThat(result).isTrue() + } + + it("should return true when messages differ with payload only") { + // given + val jsonAsStream = sampleJsonAsStream() + val generatedEvent = vesEvent(payload = "payload A") + val receivedEvent = vesEvent(payload = "payload B") + val generatedWireProtocolFrame = WireFrameMessage(generatedEvent.toByteArray()) + val receivedMessageBytes = receivedEvent.toByteArray() + + givenParsedMessageParameters(MessageParameters(generatedEvent.commonEventHeader, MessageType.VALID, 1)) + whenever(messageGenerator.createMessageFlux(anyList())).thenReturn(Flux.just(generatedWireProtocolFrame)) + + // when + val result = cut.validate(jsonAsStream, listOf(receivedMessageBytes)).unsafeRunSync() + + // then + assertThat(result).isTrue() + } + + it("should return false when messages are different") { + // given + val jsonAsStream = sampleJsonAsStream() + val generatedEvent = vesEvent() + val receivedEvent = vesEvent(eventId = "bbb") + val generatedWireProtocolFrame = WireFrameMessage(generatedEvent.toByteArray()) + val receivedMessageBytes = receivedEvent.toByteArray() + + givenParsedMessageParameters(MessageParameters(generatedEvent.commonEventHeader, MessageType.VALID, 1)) + whenever(messageGenerator.createMessageFlux(anyList())).thenReturn(Flux.just(generatedWireProtocolFrame)) + + // when + val result = cut.validate(jsonAsStream, listOf(receivedMessageBytes)).unsafeRunSync() + + // then + assertThat(result).isFalse() + } + } + + describe("when validating whole messages") { + it("should return true when messages are the same") { + // given + val jsonAsStream = sampleJsonAsStream() + val event = vesEvent() + val generatedWireProtocolFrame = WireFrameMessage(event.toByteArray()) + val receivedMessageBytes = event.toByteArray() + + givenParsedMessageParameters(MessageParameters(event.commonEventHeader, MessageType.FIXED_PAYLOAD, 1)) + whenever(messageGenerator.createMessageFlux(anyList())).thenReturn(Flux.just(generatedWireProtocolFrame)) + + // when + val result = cut.validate(jsonAsStream, listOf(receivedMessageBytes)).unsafeRunSync() + + // then + assertThat(result).isTrue() + } + + it("should return false when messages differ with payload only") { + // given + val jsonAsStream = sampleJsonAsStream() + val generatedEvent = vesEvent(payload = "payload A") + val receivedEvent = vesEvent(payload = "payload B") + val generatedWireProtocolFrame = WireFrameMessage(generatedEvent.toByteArray()) + val receivedMessageBytes = receivedEvent.toByteArray() + + givenParsedMessageParameters(MessageParameters(generatedEvent.commonEventHeader, MessageType.FIXED_PAYLOAD, 1)) + whenever(messageGenerator.createMessageFlux(anyList())).thenReturn(Flux.just(generatedWireProtocolFrame)) + + // when + val result = cut.validate(jsonAsStream, listOf(receivedMessageBytes)).unsafeRunSync() + + // then + assertThat(result).isFalse() + } + + it("should return false when messages are different") { + // given + val jsonAsStream = sampleJsonAsStream() + val generatedEvent = vesEvent() + val receivedEvent = vesEvent("bbb") + val generatedWireProtocolFrame = WireFrameMessage(generatedEvent.toByteArray()) + val receivedMessageBytes = receivedEvent.toByteArray() + + givenParsedMessageParameters(MessageParameters(generatedEvent.commonEventHeader, MessageType.FIXED_PAYLOAD, 1)) + whenever(messageGenerator.createMessageFlux(anyList())).thenReturn(Flux.just(generatedWireProtocolFrame)) + + // when + val result = cut.validate(jsonAsStream, listOf(receivedMessageBytes)).unsafeRunSync() + + // then + assertThat(result).isFalse() + } + } + } +}) + + + +private const val DUMMY_EVENT_ID = "aaa" +private const val DUMMY_PAYLOAD = "payload" + +private fun vesEvent(eventId: String = DUMMY_EVENT_ID, payload: String = DUMMY_PAYLOAD): VesEvent { + return VesEvent.newBuilder() + .setCommonEventHeader(CommonEventHeader.newBuilder() + .setEventId(eventId)) + .setEventFields(ByteString.copyFrom(payload.toByteArray())) + .build() +} + +private const val sampleJsonArray = """["headersOnly"]""" + +private fun sampleJsonAsStream() = sampleJsonArray.byteInputStream() diff --git a/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/adapters/KafkaSourceTest.kt b/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/adapters/KafkaSourceTest.kt new file mode 100644 index 00000000..de74f628 --- /dev/null +++ b/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/adapters/KafkaSourceTest.kt @@ -0,0 +1,54 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.dcaeapp.impl.adapters + +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.common.serialization.ByteArrayDeserializer +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 + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk></piotr.jaszczyk>@nokia.com> + * @since August 2018 + */ +internal class KafkaSourceTest : Spek({ + val servers = "kafka1:9080,kafka2:9080" + val topics = setOf("topic1", "topic2") + + describe("receiver options") { + val options = KafkaSource.createReceiverOptions(servers, topics)!!.toImmutable() + + fun verifyProperty(key: String, expectedValue: Any) { + it("should have $key option set") { + assertThat(options.consumerProperty(key)) + .isEqualTo(expectedValue) + } + } + + verifyProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, servers) + verifyProperty(ConsumerConfig.CLIENT_ID_CONFIG, "hv-collector-dcae-app-simulator") + verifyProperty(ConsumerConfig.GROUP_ID_CONFIG, "hv-collector-simulators") + verifyProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer::class.java) + verifyProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer::class.java) + verifyProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + } +})
\ No newline at end of file diff --git a/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/config/ArgDcaeAppSimConfigurationTest.kt b/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/config/ArgDcaeAppSimConfigurationTest.kt new file mode 100644 index 00000000..7137fe12 --- /dev/null +++ b/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/config/ArgDcaeAppSimConfigurationTest.kt @@ -0,0 +1,125 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.dcaeapp.impl.config + +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.given +import org.jetbrains.spek.api.dsl.it +import org.onap.dcae.collectors.veshv.tests.utils.parseExpectingFailure +import org.onap.dcae.collectors.veshv.tests.utils.parseExpectingSuccess +import org.onap.dcae.collectors.veshv.utils.commandline.WrongArgumentError + + +internal class ArgDcaeAppSimConfigurationTest : Spek({ + + lateinit var cut: ArgDcaeAppSimConfiguration + val listenPort = "1234" + val kafkaBootstrapServers = "localhosting:123,localhostinger:12345" + val kafkaTopics = "top1,top2" + + beforeEachTest { + cut = ArgDcaeAppSimConfiguration() + } + + describe("parsing arguments") { + lateinit var result: DcaeAppSimConfiguration + + given("all parameters are present in the long form") { + + beforeEachTest { + result = cut.parseExpectingSuccess( + "--listen-port", listenPort, + "--kafka-bootstrap-servers", kafkaBootstrapServers, + "--kafka-topics", kafkaTopics + ) + } + + it("should set proper port") { + assertThat(result.apiPort).isEqualTo(listenPort.toInt()) + } + + + it("should set proper kafka bootstrap servers") { + assertThat(result.kafkaBootstrapServers).isEqualTo(kafkaBootstrapServers) + } + + it("should set proper kafka topics") { + assertThat(result.kafkaTopics).isEqualTo( + setOf("top1", "top2") + ) + } + } + + given("some parameters are present in the short form") { + + beforeEachTest { + result = cut.parseExpectingSuccess( + "-p", listenPort, + "--kafka-bootstrap-servers", kafkaBootstrapServers, + "-f", kafkaTopics) + } + + it("should set proper port") { + assertThat(result.apiPort).isEqualTo(listenPort.toInt()) + } + + it("should set proper kafka bootstrap servers") { + assertThat(result.kafkaBootstrapServers).isEqualTo(kafkaBootstrapServers) + } + + it("should set proper kafka topics") { + assertThat(result.kafkaTopics).isEqualTo( + setOf("top1", "top2") + ) + } + } + + describe("required parameter is absent") { + given("kafka topics are missing") { + it("should throw exception") { + assertThat(cut.parseExpectingFailure( + "-p", listenPort, + "-s", kafkaBootstrapServers + )).isInstanceOf(WrongArgumentError::class.java) + } + } + + given("kafka bootstrap servers is missing") { + it("should throw exception") { + assertThat(cut.parseExpectingFailure( + "-p", listenPort, + "-f", kafkaTopics + )).isInstanceOf(WrongArgumentError::class.java) + } + } + + given("listen port is missing") { + it("should throw exception") { + assertThat(cut.parseExpectingFailure( + "-p", listenPort, + "-s", kafkaBootstrapServers + )).isInstanceOf(WrongArgumentError::class.java) + } + } + } + } +})
\ No newline at end of file diff --git a/sources/hv-collector-domain/pom.xml b/sources/hv-collector-domain/pom.xml new file mode 100644 index 00000000..f60cb604 --- /dev/null +++ b/sources/hv-collector-domain/pom.xml @@ -0,0 +1,131 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ ============LICENSE_START======================================================= + ~ dcaegen2-collectors-veshv + ~ ================================================================================ + ~ Copyright (C) 2018 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========================================================= + --> +<project xmlns="http://maven.apache.org/POM/4.0.0" + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + + <licenses> + <license> + <name>The Apache Software License, Version 2.0</name> + <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url> + </license> + </licenses> + + <parent> + <groupId>org.onap.dcaegen2.collectors.hv-ves</groupId> + <artifactId>hv-collector-sources</artifactId> + <version>1.1.0-SNAPSHOT</version> + <relativePath>..</relativePath> + </parent> + + <artifactId>hv-collector-domain</artifactId> + <description>VES HighVolume Collector :: Domain</description> + + <properties> + <skipAnalysis>false</skipAnalysis> + </properties> + + <build> + <plugins> + <plugin> + <artifactId>kotlin-maven-plugin</artifactId> + <groupId>org.jetbrains.kotlin</groupId> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-compiler-plugin</artifactId> + </plugin> + <plugin> + <artifactId>maven-surefire-plugin</artifactId> + <groupId>org.apache.maven.plugins</groupId> + </plugin> + <plugin> + <groupId>org.codehaus.mojo</groupId> + <artifactId>build-helper-maven-plugin</artifactId> + </plugin> + <plugin> + <groupId>com.github.os72</groupId> + <artifactId>protoc-jar-maven-plugin</artifactId> + <executions> + <execution> + <phase>generate-sources</phase> + <goals> + <goal>run</goal> + </goals> + <configuration> + <protocArtifact>com.google.protobuf:protoc:${protobuf.version}</protocArtifact> + <inputDirectories> + <include>${project.basedir}/src/main/proto/event</include> + </inputDirectories> + <outputTargets> + <outputTarget> + <type>java</type> + <addSources>none</addSources> + <outputDirectory>${protobuf-generated-files.directory}</outputDirectory> + </outputTarget> + </outputTargets> + </configuration> + </execution> + </executions> + </plugin> + </plugins> + </build> + + <dependencies> + <dependency> + <groupId>org.jetbrains.kotlin</groupId> + <artifactId>kotlin-stdlib-jdk8</artifactId> + </dependency> + <dependency> + <groupId>io.projectreactor.netty</groupId> + <artifactId>reactor-netty</artifactId> + </dependency> + <dependency> + <groupId>com.google.protobuf</groupId> + <artifactId>protobuf-java</artifactId> + </dependency> + <dependency> + <groupId>io.arrow-kt</groupId> + <artifactId>arrow-core</artifactId> + </dependency> + + <dependency> + <groupId>org.assertj</groupId> + <artifactId>assertj-core</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.kotlin</groupId> + <artifactId>kotlin-test</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.jetbrains.spek</groupId> + <artifactId>spek-api</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.spek</groupId> + <artifactId>spek-junit-platform-engine</artifactId> + </dependency> + </dependencies> + + +</project> diff --git a/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/ByteData.kt b/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/ByteData.kt new file mode 100644 index 00000000..a1ebba37 --- /dev/null +++ b/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/ByteData.kt @@ -0,0 +1,56 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.domain + +import io.netty.buffer.ByteBuf +import java.nio.charset.Charset + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ +class ByteData(private val data: ByteArray) { + + fun size() = data.size + + /** + * This will expose mutable state of the data. + * + * @return wrapped data buffer (NOT a copy) + */ + fun unsafeAsArray() = data + + fun writeTo(byteBuf: ByteBuf) { + byteBuf.writeBytes(data) + } + + fun asString(charset: Charset = Charset.defaultCharset()) = String(data, charset) + + companion object { + val EMPTY = ByteData(byteArrayOf()) + + fun readFrom(byteBuf: ByteBuf, length: Int): ByteData { + val dataArray = ByteArray(length) + byteBuf.readBytes(dataArray) + return ByteData(dataArray) + } + } +} + diff --git a/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/PayloadContentType.kt b/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/PayloadContentType.kt new file mode 100644 index 00000000..7cbf3530 --- /dev/null +++ b/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/PayloadContentType.kt @@ -0,0 +1,34 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.domain + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ +enum class PayloadContentType(val hexValue: Int) { + GOOGLE_PROTOCOL_BUFFER(0x0001); + + companion object { + private val hexValues = PayloadContentType.values().map { it.hexValue } + + fun isValidHexValue(hex: Int) = hexValues.contains(hex) + } +} diff --git a/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/SecurityConfiguration.kt b/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/SecurityConfiguration.kt new file mode 100644 index 00000000..7f566a6d --- /dev/null +++ b/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/SecurityConfiguration.kt @@ -0,0 +1,50 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.domain + +import arrow.core.Option +import java.io.InputStream +import java.nio.file.Path + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +data class SecurityConfiguration( + val sslDisable: Boolean = false, + val keys: Option<SslKeys>) + +sealed class SslKeys + +data class OpenSslKeys(val privateKey: Path, + val cert: Path, + val trustedCert: Path) : SslKeys() + +data class JdkKeys(val keyStore: StreamProvider, + val keyStorePassword: CharArray, + val trustStore: StreamProvider, + val trustStorePassword: CharArray) : SslKeys() { + fun forgetPasswords() { + keyStorePassword.fill('x') + trustStorePassword.fill('x') + } +} + +typealias StreamProvider = () -> InputStream diff --git a/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/VesEventDomain.kt b/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/VesEventDomain.kt new file mode 100644 index 00000000..0b18337d --- /dev/null +++ b/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/VesEventDomain.kt @@ -0,0 +1,35 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.domain + +enum class VesEventDomain(val domainName: String) { + FAULT("fault"), + HEARTBEAT("heartbeat"), + MEASUREMENT("measurement"), + MOBILE_FLOW("mobileFlow"), + OTHER("other"), + PNF_REGISTRATION("pnfRegistration"), + SIP_SIGNALING("sipSignaling"), + STATE_CHANGE("stateChange"), + SYSLOG("syslog"), + THRESHOLD_CROSSING_ALERT("thresholdCrossingAlert"), + VOICE_QUALITY("voiceQuality"), + PERF3GPP("perf3gpp"); +} diff --git a/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/codec.kt b/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/codec.kt new file mode 100644 index 00000000..7fabdf14 --- /dev/null +++ b/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/codec.kt @@ -0,0 +1,102 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.domain + +import arrow.core.Either +import arrow.core.Left +import arrow.core.Right +import io.netty.buffer.ByteBuf +import io.netty.buffer.ByteBufAllocator +import org.onap.dcae.collectors.veshv.domain.WireFrameMessage.Companion.RESERVED_BYTE_COUNT + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ +class WireFrameEncoder(private val allocator: ByteBufAllocator = ByteBufAllocator.DEFAULT) { + + fun encode(frame: WireFrameMessage): ByteBuf = allocator + .buffer(WireFrameMessage.HEADER_SIZE + frame.payload.size()) + .run { + writeByte(WireFrameMessage.MARKER_BYTE.toInt()) + writeByte(frame.versionMajor.toInt()) + writeByte(frame.versionMinor.toInt()) + writeZero(RESERVED_BYTE_COUNT) + writeShort(frame.payloadType) + writeInt(frame.payloadSize) + } + .also { + frame.payload.writeTo(it) + } +} + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ +class WireFrameDecoder(private val maxPayloadSizeBytes: Int) { + + fun decodeFirst(byteBuf: ByteBuf): Either<WireFrameDecodingError, WireFrameMessage> = + when { + isEmpty(byteBuf) -> Left(EmptyWireFrame) + headerDoesNotFit(byteBuf) -> Left(MissingWireFrameHeaderBytes) + else -> parseWireFrame(byteBuf) + } + + private fun isEmpty(byteBuf: ByteBuf) = byteBuf.readableBytes() < 1 + + private fun headerDoesNotFit(byteBuf: ByteBuf) = byteBuf.readableBytes() < WireFrameMessage.HEADER_SIZE + + private fun parseWireFrame(byteBuf: ByteBuf): Either<WireFrameDecodingError, WireFrameMessage> { + byteBuf.markReaderIndex() + + val mark = byteBuf.readUnsignedByte() + return when (mark) { + WireFrameMessage.MARKER_BYTE -> parsePayloadFrame(byteBuf) + else -> { + byteBuf.resetReaderIndex() + Left(InvalidWireFrameMarker(mark)) + } + } + } + + @Suppress("ReturnCount") + private fun parsePayloadFrame(byteBuf: ByteBuf): Either<WireFrameDecodingError, WireFrameMessage> { + val versionMajor = byteBuf.readUnsignedByte() + val versionMinor = byteBuf.readUnsignedByte() + byteBuf.skipBytes(RESERVED_BYTE_COUNT) + val payloadTypeRaw = byteBuf.readUnsignedShort() + val payloadSize = byteBuf.readInt() + + if (payloadSize > maxPayloadSizeBytes) { + byteBuf.resetReaderIndex() + return Left(PayloadSizeExceeded(maxPayloadSizeBytes)) + } + + if (byteBuf.readableBytes() < payloadSize) { + byteBuf.resetReaderIndex() + return Left(MissingWireFramePayloadBytes) + } + + val payload = ByteData.readFrom(byteBuf, payloadSize) + + return Right(WireFrameMessage(payload, versionMajor, versionMinor, payloadTypeRaw, payloadSize)) + } +} diff --git a/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/errors.kt b/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/errors.kt new file mode 100644 index 00000000..0d55cebb --- /dev/null +++ b/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/errors.kt @@ -0,0 +1,48 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.domain + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ + +sealed class WireFrameDecodingError(val message: String) + + +// Invalid frame errors + +sealed class InvalidWireFrame(msg: String) : WireFrameDecodingError(msg) + +class InvalidWireFrameMarker(actualMarker: Short) : InvalidWireFrame( + "Invalid start of frame. Expected 0x%02X, but was 0x%02X" + .format(WireFrameMessage.MARKER_BYTE, actualMarker) +) + +class PayloadSizeExceeded(maxPayloadSizeBytes: Int) : + InvalidWireFrame("payload size exceeds the limit ($maxPayloadSizeBytes bytes)") + +// Missing bytes errors + +sealed class MissingWireFrameBytes(msg: String) : WireFrameDecodingError(msg) + +object MissingWireFrameHeaderBytes : MissingWireFrameBytes("readable bytes < header size") +object MissingWireFramePayloadBytes : MissingWireFrameBytes("readable bytes < payload size") +object EmptyWireFrame : MissingWireFrameBytes("empty wire frame") diff --git a/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/validation.kt b/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/validation.kt new file mode 100644 index 00000000..1eb6239f --- /dev/null +++ b/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/validation.kt @@ -0,0 +1,38 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.domain + +import org.onap.ves.VesEventOuterClass + +val headerRequiredFieldDescriptors = listOf( + "version", + "domain", + /* field "sequence" has been removed from validation, since default value "0" is acceptable */ + "priority", + "eventId", + "eventName", + "lastEpochMicrosec", + "startEpochMicrosec", + "reportingEntityName", + "sourceName", + "vesEventListenerVersion") + .map { fieldName -> VesEventOuterClass.CommonEventHeader.getDescriptor().findFieldByName(fieldName) } + +val vesEventListenerVersionRegex = """7\.[0-9].*""".toRegex() diff --git a/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/wire_frame.kt b/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/wire_frame.kt new file mode 100644 index 00000000..de37b140 --- /dev/null +++ b/sources/hv-collector-domain/src/main/kotlin/org/onap/dcae/collectors/veshv/domain/wire_frame.kt @@ -0,0 +1,86 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.domain + +/** + * Wire frame structure is presented bellow using ASN.1 notation. Please note that official supported specification + * should be available on + * [RTD documentation](https://onap.readthedocs.io/en/latest/submodules/dcaegen2.git/docs/sections/apis/ves-hv.html). + * + * ``` + * -- Wire Transfer Protocol (binary, defined using ASN.1 notation) + * -- Encoding: use "direct encoding" to the number of octets indicated in the comment [n], using network byte order. + * + * WTP DEFINITIONS ::= BEGIN + * + * -- Used to sent data from the data provider + * WtpData ::= SEQUENCE { + * magic INTEGER (0..255), -- [1] always 0xAA + * versionMajor INTEGER (0..255), -- [1] major interface version, forward incompatible + * -- with previous major version, current value: 1 + * versionMinor INTEGER (0..255), -- [1] minor interface version, forward compatible + * -- with previous minor version, current value: 0 + * reserved OCTET STRING (SIZE (3)), -- [3] reserved for future use (ignored, but use 0) + * payloadId INTEGER (0..65535), -- [2] payload type: 0x0000=undefined, 0x0001=ONAP VesEvent (protobuf) + * payloadLength INTEGER (0..4294967295). -- [4] payload length in octets + * payload OCTET STRING -- [length as per payloadLength] + * } + * + * END + * ``` + * + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +data class WireFrameMessage(val payload: ByteData, + val versionMajor: Short, + val versionMinor: Short, + val payloadType: Int, + val payloadSize: Int +) { + constructor(payload: ByteArray) : this( + ByteData(payload), + SUPPORTED_VERSION_MAJOR, + SUPPORTED_VERSION_MINOR, + PayloadContentType.GOOGLE_PROTOCOL_BUFFER.hexValue, + payload.size) + + fun isValid(): Boolean = + versionMajor == SUPPORTED_VERSION_MAJOR + && PayloadContentType.isValidHexValue(payloadType) + && payload.size() == payloadSize + + companion object { + const val MARKER_BYTE: Short = 0xAA + const val RESERVED_BYTE_COUNT: Int = 3 + + const val SUPPORTED_VERSION_MAJOR: Short = 1 + const val SUPPORTED_VERSION_MINOR: Short = 0 + + const val HEADER_SIZE = + 1 * java.lang.Byte.BYTES + // marker + 2 * java.lang.Byte.BYTES + // single byte fields + 1 * java.lang.Short.BYTES + // double byte fields + RESERVED_BYTE_COUNT * java.lang.Byte.BYTES + // reserved bytes + 1 * java.lang.Integer.BYTES // payload length + + const val DEFAULT_MAX_PAYLOAD_SIZE_BYTES = 1024 * 1024 + } +} diff --git a/sources/hv-collector-domain/src/main/proto/event/VesEvent.proto b/sources/hv-collector-domain/src/main/proto/event/VesEvent.proto new file mode 100644 index 00000000..6d4c2307 --- /dev/null +++ b/sources/hv-collector-domain/src/main/proto/event/VesEvent.proto @@ -0,0 +1,75 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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========================================================= + */ +syntax = "proto3"; +package org.onap.ves; + +message VesEvent // top-level message, currently the maximum event size supported by the HV-VES Collector is 1 MiB +{ + CommonEventHeader commonEventHeader=1; // required + + bytes eventFields=2; // required, payload + // this field contains a domain-specific GPB message + // the field being opaque (bytes), the decoding of the payload occurs in a separate step + // the name of the GPB message for domain XYZ is XyzFields + // e.g. for domain==perf3gpp, the GPB message is Perf3gppFields +} + +// VES CommonEventHeader adapted to GPB (Google Protocol Buffers) + +message CommonEventHeader +{ + string version = 1; // required, "version of the gpb common event header", current value "1.0" + string domain = 2; // required, "the eventing domain associated with the event", allowed values: + // fault, heartbeat, measurement, mobileFlow, other, pnfRegistration, sipSignaling, + // stateChange, syslog, thresholdCrossingAlert, voiceQuality, perf3gpp + + uint32 sequence = 3; // required, "ordering of events communicated by an event source instance or 0 if not needed" + + enum Priority + { + PRIORITY_NOT_PROVIDED = 0; + HIGH = 1; + MEDIUM = 2; + NORMAL = 3; + LOW = 4; + } + Priority priority = 4; // required, "processing priority" + + string eventId = 5; // required, "event key that is unique to the event source" + string eventName = 6; // required, "unique event name" + string eventType = 7; // "for example - guest05, platform" + + uint64 lastEpochMicrosec = 8; // required, "the latest unix time aka epoch time associated with the event from any component--as microseconds elapsed since 1 Jan 1970 not including leap seconds" + uint64 startEpochMicrosec = 9; // required, "the earliest unix time aka epoch time associated with the event from any component--as microseconds elapsed since 1 Jan 1970 not including leap seconds" + + string nfNamingCode = 10; // "4 character network function type, aligned with vnf naming standards" + string nfcNamingCode = 11; // "3 character network function component type, aligned with vfc naming standards" + string nfVendorName = 12; // " Vendor Name providing the nf " + + bytes reportingEntityId = 13; // "UUID identifying the entity reporting the event, for example an OAM VM; must be populated by the ATT enrichment process" + string reportingEntityName = 14; // required, "name of the entity reporting the event, for example, an EMS name; may be the same as sourceName should match A&AI entry" + bytes sourceId = 15; // "UUID identifying the entity experiencing the event issue; must be populated by the ATT enrichment process" + string sourceName = 16; // required, "name of the entity experiencing the event issued use A&AI entry" + string timeZoneOffset = 17; // "Offset to GMT to indicate local time zone for the device" + string vesEventListenerVersion = 18; // required, "Version of the VesEvent Listener", current value "7.2" + + reserved "InternalHeaderFields"; // "enrichment fields for internal VES Event Listener service use only, not supplied by event sources" + reserved 100; +} diff --git a/sources/hv-collector-domain/src/main/proto/measurements/README.md b/sources/hv-collector-domain/src/main/proto/measurements/README.md new file mode 100644 index 00000000..eb69eb4a --- /dev/null +++ b/sources/hv-collector-domain/src/main/proto/measurements/README.md @@ -0,0 +1 @@ +Measurements data (data placed in VesEvent.eventFields) description should be available in [RTD documentation](https://onap.readthedocs.io/en/latest/submodules/dcaegen2.git/docs/sections/apis/ves-hv.html).
\ No newline at end of file diff --git a/sources/hv-collector-domain/src/test/kotlin/org/onap/dcae/collectors/veshv/domain/WireFrameCodecsTest.kt b/sources/hv-collector-domain/src/test/kotlin/org/onap/dcae/collectors/veshv/domain/WireFrameCodecsTest.kt new file mode 100644 index 00000000..f17a79ba --- /dev/null +++ b/sources/hv-collector-domain/src/test/kotlin/org/onap/dcae/collectors/veshv/domain/WireFrameCodecsTest.kt @@ -0,0 +1,282 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.domain + +import arrow.core.Either +import io.netty.buffer.ByteBuf +import io.netty.buffer.Unpooled +import org.assertj.core.api.Assertions.assertThat +import org.assertj.core.api.ObjectAssert +import org.jetbrains.spek.api.Spek +import org.jetbrains.spek.api.dsl.describe +import org.jetbrains.spek.api.dsl.given +import org.jetbrains.spek.api.dsl.it +import java.nio.charset.Charset +import kotlin.test.assertTrue +import kotlin.test.fail + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ +object WireFrameCodecsTest : Spek({ + val payloadAsString = "coffeebabe" + val maxPayloadSizeBytes = 1024 + val encoder = WireFrameEncoder() + val decoder = WireFrameDecoder(maxPayloadSizeBytes) + + fun createSampleFrame() = WireFrameMessage(payloadAsString.toByteArray(Charset.defaultCharset())) + + fun encodeSampleFrame() = + createSampleFrame().let { + encoder.encode(it) + } + + describe("Wire Frame invariants") { + + given("input with unsupported major version") { + val input = WireFrameMessage( + payload = ByteData.EMPTY, + versionMajor = 100, + versionMinor = 0, + payloadType = PayloadContentType.GOOGLE_PROTOCOL_BUFFER.hexValue, + payloadSize = 0) + + it("should fail validation") { + assertThat(input.isValid()).isFalse() + } + } + + given("input with unsupported minor version") { + val input = WireFrameMessage( + payload = ByteData.EMPTY, + versionMajor = 1, + versionMinor = 6, + payloadType = PayloadContentType.GOOGLE_PROTOCOL_BUFFER.hexValue, + payloadSize = 0) + + it("should pass validation") { + assertThat(input.isValid()).isTrue() + } + } + + given("input with unsupported payload type") { + val input = WireFrameMessage( + payload = ByteData.EMPTY, + versionMajor = 1, + versionMinor = 0, + payloadType = 0x69, + payloadSize = 0) + + it("should fail validation") { + assertThat(input.isValid()).isFalse() + } + } + + given("input with too small payload size") { + val input = WireFrameMessage( + payload = ByteData(byteArrayOf(1, 2, 3)), + versionMajor = 1, + versionMinor = 0, + payloadType = PayloadContentType.GOOGLE_PROTOCOL_BUFFER.hexValue, + payloadSize = 1) + + it("should fail validation") { + assertThat(input.isValid()).isFalse() + } + } + + given("input with too big payload size") { + val input = WireFrameMessage( + payload = ByteData(byteArrayOf(1, 2, 3)), + versionMajor = 1, + versionMinor = 0, + payloadType = PayloadContentType.GOOGLE_PROTOCOL_BUFFER.hexValue, + payloadSize = 8) + + it("should fail validation") { + assertThat(input.isValid()).isFalse() + } + } + + given("valid input") { + val payload = byteArrayOf(6, 9, 8, 6) + val input = WireFrameMessage( + payload = ByteData(payload), + versionMajor = 1, + versionMinor = 0, + payloadType = PayloadContentType.GOOGLE_PROTOCOL_BUFFER.hexValue, + payloadSize = payload.size) + + it("should pass validation") { + assertThat(input.isValid()).isTrue() + } + } + + + } + + describe("Wire Frame codec") { + + describe("encode-decode methods' compatibility") { + val frame = createSampleFrame() + val encoded = encodeSampleFrame() + val decoded = decoder.decodeFirst(encoded).getMessageOrFail() + + it("should decode major version") { + assertThat(decoded.versionMajor).isEqualTo(frame.versionMajor) + } + + it("should decode minor version") { + assertThat(decoded.versionMinor).isEqualTo(frame.versionMinor) + } + + it("should decode payload type") { + assertThat(decoded.payloadType).isEqualTo(frame.payloadType) + } + + it("should decode payload size") { + assertThat(decoded.payloadSize).isEqualTo(frame.payloadSize) + } + + it("should decode payload") { + assertThat(decoded.payload.asString()) + .isEqualTo(payloadAsString) + } + } + + + describe("TCP framing") { + // see "Dealing with a Stream-based Transport" on http://netty.io/wiki/user-guide-for-4.x.html#wiki-h3-11 + + it("should return error when buffer is empty") { + val buff = Unpooled.buffer() + + decoder.decodeFirst(buff).assertFailedWithError { it.isInstanceOf(EmptyWireFrame::class.java) } + assertBufferIntact(buff) + } + + it("should return error when given any single byte other than marker byte") { + val buff = Unpooled.buffer() + .writeByte(0xEE) + + decoder.decodeFirst(buff).assertFailedWithError { it.isInstanceOf(MissingWireFrameHeaderBytes::class.java) } + assertBufferIntact(buff) + } + + it("should return error when payload message header does not fit") { + val buff = Unpooled.buffer() + .writeByte(0xAA) + .writeBytes("MOMOM".toByteArray()) + + decoder.decodeFirst(buff).assertFailedWithError { it.isInstanceOf(MissingWireFrameHeaderBytes::class.java) } + assertBufferIntact(buff) + } + + it("should return error when length looks ok but first byte is not 0xAA") { + val buff = Unpooled.buffer() + .writeByte(0xFF) + .writeBytes("some garbage".toByteArray()) + + decoder.decodeFirst(buff).assertFailedWithError { it.isInstanceOf(InvalidWireFrameMarker::class.java) } + assertBufferIntact(buff) + } + + it("should return error when payload doesn't fit") { + val buff = Unpooled.buffer() + .writeBytes(encodeSampleFrame()) + buff.writerIndex(buff.writerIndex() - 2) + + decoder.decodeFirst(buff).assertFailedWithError { it.isInstanceOf(MissingWireFramePayloadBytes::class.java) } + assertBufferIntact(buff) + } + + it("should decode payload message leaving rest unread") { + val buff = Unpooled.buffer() + .writeBytes(encodeSampleFrame()) + .writeByte(0xAB) + val decoded = decoder.decodeFirst(buff).getMessageOrFail() + + assertThat(decoded.isValid()).describedAs("should be valid").isTrue() + assertThat(buff.readableBytes()).isEqualTo(1) + } + } + + describe("payload size limit") { + + it("should decode successfully when payload size is equal 1 MiB") { + + val payload = ByteArray(maxPayloadSizeBytes) + val input = WireFrameMessage( + payload = ByteData(payload), + versionMajor = 1, + versionMinor = 0, + payloadType = PayloadContentType.GOOGLE_PROTOCOL_BUFFER.hexValue, + payloadSize = payload.size) + + + assertTrue(decoder.decodeFirst(encoder.encode(input)).isRight()) + } + + it("should return error when payload exceeds 1 MiB") { + + val payload = ByteArray(maxPayloadSizeBytes + 1) + val input = WireFrameMessage( + payload = ByteData(payload), + versionMajor = 1, + versionMinor = 0, + payloadType = PayloadContentType.GOOGLE_PROTOCOL_BUFFER.hexValue, + payloadSize = payload.size) + val buff = encoder.encode(input) + + decoder.decodeFirst(buff) + .assertFailedWithError { it.isInstanceOf(PayloadSizeExceeded::class.java) } + assertBufferIntact(buff) + } + + it("should validate only first message") { + + val payload = ByteArray(maxPayloadSizeBytes) + val input = WireFrameMessage( + payload = ByteData(payload), + versionMajor = 1, + versionMinor = 0, + payloadType = PayloadContentType.GOOGLE_PROTOCOL_BUFFER.hexValue, + payloadSize = payload.size) + + + assertTrue(decoder.decodeFirst(encoder.encode(input).writeByte(0xAA)).isRight()) + } + } + } +}) + +private fun assertBufferIntact(buff: ByteBuf) { + assertThat(buff.refCnt()).describedAs("buffer should not be released").isEqualTo(1) + assertThat(buff.readerIndex()).describedAs("buffer reader index should be intact").isEqualTo(0) +} + +private fun <A, B> Either<A, B>.assertFailedWithError(assertj: (ObjectAssert<A>) -> Unit) { + fold({ assertj(assertThat(it)) }, { fail("Error expected") }) +} + +private fun Either<WireFrameDecodingError, WireFrameMessage>.getMessageOrFail(): WireFrameMessage = + fold({ fail(it.message) }, { it }) + diff --git a/sources/hv-collector-domain/src/test/resources/logback.xml b/sources/hv-collector-domain/src/test/resources/logback.xml new file mode 100644 index 00000000..0bf2cb02 --- /dev/null +++ b/sources/hv-collector-domain/src/test/resources/logback.xml @@ -0,0 +1,54 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ ============LICENSE_START======================================================= + ~ dcaegen2-collectors-veshv + ~ ================================================================================ + ~ Copyright (C) 2018 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========================================================= + --> +<configuration> + <property name="LOG_FILE" + value="${LOG_FILE:-${LOG_PATH:-${LOG_TEMP:-${java.io.tmpdir:-/tmp}}/}ves-hv.log}"/> + <property name="FILE_LOG_PATTERN" value="%d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %-5level [%-40.40logger{10}] - %msg%n"/> + + <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> + <encoder> + <pattern> + %d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %highlight(%-5level) [%-40.40logger{10}] - %msg%n + </pattern> + </encoder> + </appender> + + <appender name="ROLLING-FILE" + class="ch.qos.logback.core.rolling.RollingFileAppender"> + <encoder> + <pattern>${FILE_LOG_PATTERN}</pattern> + </encoder> + <file>${LOG_FILE}</file> + <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy"> + <fileNamePattern>${LOG_FILE}.%d{yyyy-MM-dd}.log</fileNamePattern> + <maxFileSize>50MB</maxFileSize> + <maxHistory>30</maxHistory> + <totalSizeCap>10GB</totalSizeCap> + </rollingPolicy> + </appender> + + <logger name="org.onap.dcae.collectors.veshv" level="DEBUG"/> + + <root level="INFO"> + <appender-ref ref="CONSOLE"/> + <appender-ref ref="ROLLING-FILE"/> + </root> +</configuration>
\ No newline at end of file diff --git a/sources/hv-collector-health-check/pom.xml b/sources/hv-collector-health-check/pom.xml new file mode 100644 index 00000000..3e5c6aa0 --- /dev/null +++ b/sources/hv-collector-health-check/pom.xml @@ -0,0 +1,81 @@ +<?xml version="1.0" encoding="UTF-8"?> +<project xmlns="http://maven.apache.org/POM/4.0.0" + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + + <modelVersion>4.0.0</modelVersion> + + <licenses> + <license> + <name>The Apache Software License, Version 2.0</name> + <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url> + </license> + </licenses> + + <properties> + <skipAnalysis>false</skipAnalysis> + </properties> + + <parent> + <groupId>org.onap.dcaegen2.collectors.hv-ves</groupId> + <artifactId>hv-collector-sources</artifactId> + <version>1.1.0-SNAPSHOT</version> + <relativePath>..</relativePath> + </parent> + + <artifactId>hv-collector-health-check</artifactId> + <description>VES HighVolume Collector :: Health check</description> + + <build> + <plugins> + <plugin> + <artifactId>kotlin-maven-plugin</artifactId> + <groupId>org.jetbrains.kotlin</groupId> + </plugin> + <plugin> + <artifactId>maven-surefire-plugin</artifactId> + <groupId>org.apache.maven.plugins</groupId> + </plugin> + </plugins> + </build> + + <dependencies> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-utils</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>org.jetbrains.kotlin</groupId> + <artifactId>kotlin-stdlib-jdk8</artifactId> + </dependency> + <dependency> + <groupId>io.projectreactor.netty</groupId> + <artifactId>reactor-netty</artifactId> + </dependency> + <dependency> + <groupId>io.arrow-kt</groupId> + <artifactId>arrow-effects</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.kotlin</groupId> + <artifactId>kotlin-test</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.jetbrains.spek</groupId> + <artifactId>spek-api</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.jetbrains.spek</groupId> + <artifactId>spek-junit-platform-engine</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>io.projectreactor</groupId> + <artifactId>reactor-test</artifactId> + <scope>test</scope> + </dependency> + </dependencies> +</project>
\ No newline at end of file diff --git a/sources/hv-collector-health-check/src/main/kotlin/org/onap/dcae/collectors/veshv/healthcheck/api/HealthDescription.kt b/sources/hv-collector-health-check/src/main/kotlin/org/onap/dcae/collectors/veshv/healthcheck/api/HealthDescription.kt new file mode 100644 index 00000000..8c69406c --- /dev/null +++ b/sources/hv-collector-health-check/src/main/kotlin/org/onap/dcae/collectors/veshv/healthcheck/api/HealthDescription.kt @@ -0,0 +1,32 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.healthcheck.api + + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since August 2018 + */ +enum class HealthDescription(val message: String, val status: HealthStatus) { + HEALTHY("Healthy", HealthStatus.UP), + STARTING("Collector is starting", HealthStatus.OUT_OF_SERVICE), + RETRYING_FOR_CONSUL_CONFIGURATION("Consul configuration not available. Retrying.", HealthStatus.OUT_OF_SERVICE), + CONSUL_CONFIGURATION_NOT_FOUND("Consul configuration not found", HealthStatus.DOWN) +} diff --git a/sources/hv-collector-health-check/src/main/kotlin/org/onap/dcae/collectors/veshv/healthcheck/api/HealthState.kt b/sources/hv-collector-health-check/src/main/kotlin/org/onap/dcae/collectors/veshv/healthcheck/api/HealthState.kt new file mode 100644 index 00000000..853cc00f --- /dev/null +++ b/sources/hv-collector-health-check/src/main/kotlin/org/onap/dcae/collectors/veshv/healthcheck/api/HealthState.kt @@ -0,0 +1,39 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.healthcheck.api + +import org.onap.dcae.collectors.veshv.healthcheck.impl.HealthStateImpl +import reactor.core.publisher.Flux + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since August 2018 + */ +interface HealthState { + + operator fun invoke(): Flux<HealthDescription> + fun changeState(healthDescription: HealthDescription) + + companion object { + val INSTANCE: HealthState by lazy { + HealthStateImpl() + } + } +} diff --git a/sources/hv-collector-health-check/src/main/kotlin/org/onap/dcae/collectors/veshv/healthcheck/api/HealthStatus.kt b/sources/hv-collector-health-check/src/main/kotlin/org/onap/dcae/collectors/veshv/healthcheck/api/HealthStatus.kt new file mode 100644 index 00000000..1adf0cad --- /dev/null +++ b/sources/hv-collector-health-check/src/main/kotlin/org/onap/dcae/collectors/veshv/healthcheck/api/HealthStatus.kt @@ -0,0 +1,33 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.healthcheck.api + +import org.onap.dcae.collectors.veshv.utils.http.HttpStatus + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since August 2018 + */ +enum class HealthStatus(val httpResponseStatus: HttpStatus) { + UP(HttpStatus.OK), + DOWN(HttpStatus.SERVICE_UNAVAILABLE), + OUT_OF_SERVICE(HttpStatus.SERVICE_UNAVAILABLE), + UNKNOWN(HttpStatus.SERVICE_UNAVAILABLE) +} diff --git a/sources/hv-collector-health-check/src/main/kotlin/org/onap/dcae/collectors/veshv/healthcheck/factory/HealthCheckApiServer.kt b/sources/hv-collector-health-check/src/main/kotlin/org/onap/dcae/collectors/veshv/healthcheck/factory/HealthCheckApiServer.kt new file mode 100644 index 00000000..f12d9acf --- /dev/null +++ b/sources/hv-collector-health-check/src/main/kotlin/org/onap/dcae/collectors/veshv/healthcheck/factory/HealthCheckApiServer.kt @@ -0,0 +1,63 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.healthcheck.factory + +import arrow.effects.IO +import io.netty.handler.codec.http.HttpResponseStatus +import org.onap.dcae.collectors.veshv.healthcheck.api.HealthDescription +import org.onap.dcae.collectors.veshv.healthcheck.api.HealthState +import org.onap.dcae.collectors.veshv.utils.NettyServerHandle +import org.onap.dcae.collectors.veshv.utils.ServerHandle +import reactor.core.publisher.Flux +import reactor.core.publisher.Mono +import reactor.netty.http.server.HttpServer +import reactor.netty.http.server.HttpServerRequest +import reactor.netty.http.server.HttpServerResponse +import java.net.SocketAddress +import java.util.concurrent.atomic.AtomicReference + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since August 2018 + */ +class HealthCheckApiServer(private val healthState: HealthState, + private val listenAddress: SocketAddress) { + + private val healthDescription: AtomicReference<HealthDescription> = AtomicReference(HealthDescription.STARTING) + + fun start(): IO<ServerHandle> = IO { + healthState().subscribe(healthDescription::set) + val ctx = HttpServer.create() + .tcpConfiguration { it.addressSupplier { listenAddress } } + .route { routes -> + routes.get("/health/ready", ::readinessHandler) + routes.get("/health/alive", ::livenessHandler) + } + NettyServerHandle(ctx.bindNow()) + } + + private fun readinessHandler(_req: HttpServerRequest, resp: HttpServerResponse) = + healthDescription.get().run { + resp.status(status.httpResponseStatus.number).sendString(Flux.just(status.toString(), "\n", message)) + } + + private fun livenessHandler(_req: HttpServerRequest, resp: HttpServerResponse) = + resp.status(HttpResponseStatus.NOT_IMPLEMENTED).sendString(Mono.just("Not implemented yet")) +} diff --git a/sources/hv-collector-health-check/src/main/kotlin/org/onap/dcae/collectors/veshv/healthcheck/impl/HealthStateImpl.kt b/sources/hv-collector-health-check/src/main/kotlin/org/onap/dcae/collectors/veshv/healthcheck/impl/HealthStateImpl.kt new file mode 100644 index 00000000..c273f0a0 --- /dev/null +++ b/sources/hv-collector-health-check/src/main/kotlin/org/onap/dcae/collectors/veshv/healthcheck/impl/HealthStateImpl.kt @@ -0,0 +1,39 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.healthcheck.impl + +import org.onap.dcae.collectors.veshv.healthcheck.api.HealthState +import org.onap.dcae.collectors.veshv.healthcheck.api.HealthDescription +import reactor.core.publisher.Flux +import reactor.core.publisher.FluxProcessor +import reactor.core.publisher.UnicastProcessor + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since August 2018 + */ +internal class HealthStateImpl : HealthState { + + private val healthDescriptionStream: FluxProcessor<HealthDescription, HealthDescription> = UnicastProcessor.create() + + override fun invoke(): Flux<HealthDescription> = healthDescriptionStream + + override fun changeState(healthDescription: HealthDescription) = healthDescriptionStream.onNext(healthDescription) +} diff --git a/sources/hv-collector-health-check/src/test/kotlin/org/onap/dcae/collectors/veshv/healthcheck/impl/HealthStateProviderImplTest.kt b/sources/hv-collector-health-check/src/test/kotlin/org/onap/dcae/collectors/veshv/healthcheck/impl/HealthStateProviderImplTest.kt new file mode 100644 index 00000000..e3fced2d --- /dev/null +++ b/sources/hv-collector-health-check/src/test/kotlin/org/onap/dcae/collectors/veshv/healthcheck/impl/HealthStateProviderImplTest.kt @@ -0,0 +1,53 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.healthcheck.impl + +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.healthcheck.api.HealthDescription +import reactor.test.StepVerifier + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since August 2018 + */ +object HealthStateProviderImplTest : Spek({ + describe("Health state provider") { + val healthStateProviderImpl = HealthStateImpl() + on("health state update") { + healthStateProviderImpl.changeState(HealthDescription.HEALTHY) + healthStateProviderImpl.changeState(HealthDescription.RETRYING_FOR_CONSUL_CONFIGURATION) + healthStateProviderImpl.changeState(HealthDescription.RETRYING_FOR_CONSUL_CONFIGURATION) + healthStateProviderImpl.changeState(HealthDescription.CONSUL_CONFIGURATION_NOT_FOUND) + + it("should push new health state to the subscriber") { + StepVerifier + .create(healthStateProviderImpl().take(4)) + .expectNext(HealthDescription.HEALTHY) + .expectNext(HealthDescription.RETRYING_FOR_CONSUL_CONFIGURATION) + .expectNext(HealthDescription.RETRYING_FOR_CONSUL_CONFIGURATION) + .expectNext(HealthDescription.CONSUL_CONFIGURATION_NOT_FOUND) + .verifyComplete() + } + } + } +})
\ No newline at end of file diff --git a/sources/hv-collector-main/Dockerfile b/sources/hv-collector-main/Dockerfile new file mode 100644 index 00000000..0170b645 --- /dev/null +++ b/sources/hv-collector-main/Dockerfile @@ -0,0 +1,19 @@ +FROM docker.io/openjdk:11-jre-slim + +LABEL copyright="Copyright (C) 2018 NOKIA" +LABEL license.name="The Apache Software License, Version 2.0" +LABEL license.url="http://www.apache.org/licenses/LICENSE-2.0" +LABEL maintainer="Nokia Wroclaw ONAP Team" + +RUN apt-get update \ + && apt-get install -y --no-install-recommends curl netcat \ + && apt-get clean + +WORKDIR /opt/ves-hv-collector + +ENTRYPOINT ["java", "-cp", "*:", "org.onap.dcae.collectors.veshv.main.MainKt"] + +COPY target/libs/external/* ./ +COPY target/libs/internal/* ./ +COPY src/main/scripts/healthcheck.sh ./ +COPY target/hv-collector-main-*.jar ./ diff --git a/sources/hv-collector-main/pom.xml b/sources/hv-collector-main/pom.xml new file mode 100644 index 00000000..a94d6346 --- /dev/null +++ b/sources/hv-collector-main/pom.xml @@ -0,0 +1,143 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ ============LICENSE_START======================================================= + ~ dcaegen2-collectors-veshv + ~ ================================================================================ + ~ Copyright (C) 2018 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========================================================= + --> +<project xmlns="http://maven.apache.org/POM/4.0.0" + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + + <licenses> + <license> + <name>The Apache Software License, Version 2.0</name> + <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url> + </license> + </licenses> + + <parent> + <groupId>org.onap.dcaegen2.collectors.hv-ves</groupId> + <artifactId>hv-collector-sources</artifactId> + <version>1.1.0-SNAPSHOT</version> + <relativePath>..</relativePath> + </parent> + + <artifactId>hv-collector-main</artifactId> + <description>VES HighVolume Collector :: Main</description> + + <properties> + <skipAnalysis>false</skipAnalysis> + </properties> + + <build> + <plugins> + <plugin> + <artifactId>kotlin-maven-plugin</artifactId> + <groupId>org.jetbrains.kotlin</groupId> + </plugin> + <plugin> + <artifactId>maven-surefire-plugin</artifactId> + <groupId>org.apache.maven.plugins</groupId> + </plugin> + </plugins> + </build> + + <profiles> + <profile> + <id>docker</id> + <activation> + <property> + <name>!skipDocker</name> + </property> + </activation> + <build> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-dependency-plugin</artifactId> + </plugin> + <plugin> + <groupId>io.fabric8</groupId> + <artifactId>docker-maven-plugin</artifactId> + </plugin> + </plugins> + </build> + </profile> + </profiles> + + <dependencies> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-core</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-health-check</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-test-utils</artifactId> + <version>${project.parent.version}</version> + <scope>test</scope> + </dependency> + <dependency> + <groupId>io.arrow-kt</groupId> + <artifactId>arrow-core</artifactId> + </dependency> + <dependency> + <groupId>io.arrow-kt</groupId> + <artifactId>arrow-effects-instances</artifactId> + </dependency> + <dependency> + <groupId>io.arrow-kt</groupId> + <artifactId>arrow-syntax</artifactId> + </dependency> + <dependency> + <groupId>org.slf4j</groupId> + <artifactId>slf4j-api</artifactId> + </dependency> + <dependency> + <groupId>ch.qos.logback</groupId> + <artifactId>logback-classic</artifactId> + </dependency> + <dependency> + <groupId>commons-cli</groupId> + <artifactId>commons-cli</artifactId> + </dependency> + <!-- See comment in main pom + <dependency> + <groupId>io.netty</groupId> + <artifactId>netty-transport-native-epoll</artifactId> + <classifier>${os.detected.classifier}</classifier> + </dependency> + <dependency> + <groupId>io.netty</groupId> + <artifactId>netty-tcnative-boringssl-static</artifactId> + <classifier>${os.detected.classifier}</classifier> + </dependency> + --> + <dependency> + <groupId>io.micrometer</groupId> + <artifactId>micrometer-registry-jmx</artifactId> + </dependency> + </dependencies> + + +</project> diff --git a/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/ArgVesHvConfiguration.kt b/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/ArgVesHvConfiguration.kt new file mode 100644 index 00000000..9b985f6f --- /dev/null +++ b/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/ArgVesHvConfiguration.kt @@ -0,0 +1,118 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.main + +import arrow.core.Option +import arrow.core.fix +import arrow.instances.option.monad.monad +import arrow.typeclasses.binding +import org.apache.commons.cli.CommandLine +import org.apache.commons.cli.DefaultParser +import org.onap.dcae.collectors.veshv.domain.WireFrameMessage +import org.onap.dcae.collectors.veshv.model.ConfigurationProviderParams +import org.onap.dcae.collectors.veshv.model.ServerConfiguration +import org.onap.dcae.collectors.veshv.ssl.boundary.createSecurityConfiguration +import org.onap.dcae.collectors.veshv.utils.commandline.ArgBasedConfiguration +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.CONSUL_CONFIG_URL +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.CONSUL_FIRST_REQUEST_DELAY +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.CONSUL_REQUEST_INTERVAL +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.DUMMY_MODE +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.HEALTH_CHECK_API_PORT +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.IDLE_TIMEOUT_SEC +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.KEY_STORE_FILE +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.KEY_STORE_PASSWORD +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.LISTEN_PORT +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.MAXIMUM_PAYLOAD_SIZE_BYTES +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.SSL_DISABLE +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.TRUST_STORE_FILE +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.TRUST_STORE_PASSWORD +import org.onap.dcae.collectors.veshv.utils.commandline.hasOption +import org.onap.dcae.collectors.veshv.utils.commandline.intValue +import org.onap.dcae.collectors.veshv.utils.commandline.longValue +import org.onap.dcae.collectors.veshv.utils.commandline.stringValue +import java.net.InetSocketAddress +import java.time.Duration + +internal class ArgVesHvConfiguration : ArgBasedConfiguration<ServerConfiguration>(DefaultParser()) { + override val cmdLineOptionsList = listOf( + HEALTH_CHECK_API_PORT, + LISTEN_PORT, + CONSUL_CONFIG_URL, + CONSUL_FIRST_REQUEST_DELAY, + CONSUL_REQUEST_INTERVAL, + SSL_DISABLE, + KEY_STORE_FILE, + KEY_STORE_PASSWORD, + TRUST_STORE_FILE, + TRUST_STORE_PASSWORD, + IDLE_TIMEOUT_SEC, + MAXIMUM_PAYLOAD_SIZE_BYTES, + DUMMY_MODE + ) + + override fun getConfiguration(cmdLine: CommandLine): Option<ServerConfiguration> = + Option.monad().binding { + val healthCheckApiPort = cmdLine.intValue( + HEALTH_CHECK_API_PORT, + DefaultValues.HEALTH_CHECK_API_PORT + ) + val listenPort = cmdLine.intValue(LISTEN_PORT).bind() + val idleTimeoutSec = cmdLine.longValue(IDLE_TIMEOUT_SEC, DefaultValues.IDLE_TIMEOUT_SEC) + val maxPayloadSizeBytes = cmdLine.intValue(MAXIMUM_PAYLOAD_SIZE_BYTES, + DefaultValues.MAX_PAYLOAD_SIZE_BYTES) + val dummyMode = cmdLine.hasOption(DUMMY_MODE) + val security = createSecurityConfiguration(cmdLine).bind() + val configurationProviderParams = createConfigurationProviderParams(cmdLine).bind() + ServerConfiguration( + serverListenAddress = InetSocketAddress(listenPort), + healthCheckApiListenAddress = InetSocketAddress(healthCheckApiPort), + configurationProviderParams = configurationProviderParams, + securityConfiguration = security, + idleTimeout = Duration.ofSeconds(idleTimeoutSec), + maximumPayloadSizeBytes = maxPayloadSizeBytes, + dummyMode = dummyMode) + }.fix() + + private fun createConfigurationProviderParams(cmdLine: CommandLine): Option<ConfigurationProviderParams> = + Option.monad().binding { + val configUrl = cmdLine.stringValue(CONSUL_CONFIG_URL).bind() + val firstRequestDelay = cmdLine.longValue( + CONSUL_FIRST_REQUEST_DELAY, + DefaultValues.CONSUL_FIRST_REQUEST_DELAY + ) + val requestInterval = cmdLine.longValue( + CONSUL_REQUEST_INTERVAL, + DefaultValues.CONSUL_REQUEST_INTERVAL + ) + ConfigurationProviderParams( + configUrl, + Duration.ofSeconds(firstRequestDelay), + Duration.ofSeconds(requestInterval) + ) + }.fix() + + internal object DefaultValues { + const val HEALTH_CHECK_API_PORT = 6060 + const val CONSUL_FIRST_REQUEST_DELAY = 10L + const val CONSUL_REQUEST_INTERVAL = 5L + const val IDLE_TIMEOUT_SEC = 60L + const val MAX_PAYLOAD_SIZE_BYTES = WireFrameMessage.DEFAULT_MAX_PAYLOAD_SIZE_BYTES + } +} diff --git a/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/MicrometerMetrics.kt b/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/MicrometerMetrics.kt new file mode 100644 index 00000000..8a8b6d39 --- /dev/null +++ b/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/MicrometerMetrics.kt @@ -0,0 +1,67 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.main + +import arrow.syntax.function.memoize +import io.micrometer.core.instrument.Clock +import io.micrometer.core.instrument.Counter +import io.micrometer.core.instrument.MeterRegistry +import io.micrometer.jmx.JmxConfig +import io.micrometer.jmx.JmxMeterRegistry +import org.onap.dcae.collectors.veshv.boundary.Metrics + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ +class MicrometerMetrics( + private val registry: MeterRegistry = JmxMeterRegistry(JmxConfig.DEFAULT, Clock.SYSTEM) +) : Metrics { + + private val receivedBytes = registry.counter("data.received.bytes") + private val receivedMsgCount = registry.counter("messages.received.count") + private val receivedMsgBytes = registry.counter("messages.received.bytes") + private val sentCountTotal = registry.counter("messages.sent.count") + + init { + registry.gauge("messages.processing.count", this) { + (receivedMsgCount.count() - sentCountTotal.count()).coerceAtLeast(0.0) + } + } + + private val sentCount = { topic: String -> + registry.counter("messages.sent.count", "topic", topic) + }.memoize<String, Counter>() + + + override fun notifyBytesReceived(size: Int) { + receivedBytes.increment(size.toDouble()) + } + + override fun notifyMessageReceived(size: Int) { + receivedMsgCount.increment() + receivedMsgBytes.increment(size.toDouble()) + } + + override fun notifyMessageSent(topic: String) { + sentCountTotal.increment() + sentCount(topic).increment() + } +} diff --git a/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/main.kt b/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/main.kt new file mode 100644 index 00000000..899f51fb --- /dev/null +++ b/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/main.kt @@ -0,0 +1,55 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.main + +import arrow.effects.IO +import arrow.effects.fix +import arrow.effects.instances.io.monad.monad +import arrow.typeclasses.binding +import org.onap.dcae.collectors.veshv.main.servers.HealthCheckServer +import org.onap.dcae.collectors.veshv.main.servers.VesServer +import org.onap.dcae.collectors.veshv.model.ServerConfiguration +import org.onap.dcae.collectors.veshv.utils.arrow.ExitFailure +import org.onap.dcae.collectors.veshv.utils.arrow.unsafeRunEitherSync +import org.onap.dcae.collectors.veshv.utils.commandline.handleWrongArgumentErrorCurried +import org.onap.dcae.collectors.veshv.utils.logging.Logger + +private val logger = Logger("org.onap.dcae.collectors.veshv.main") +private const val PROGRAM_NAME = "java org.onap.dcae.collectors.veshv.main.MainKt" + +fun main(args: Array<String>) = + ArgVesHvConfiguration().parse(args) + .mapLeft(handleWrongArgumentErrorCurried(PROGRAM_NAME)) + .map(::startAndAwaitServers) + .unsafeRunEitherSync( + { ex -> + logger.error("Failed to start a server", ex) + ExitFailure(1) + }, + { logger.info("Gentle shutdown") } + ) + +private fun startAndAwaitServers(config: ServerConfiguration) = + IO.monad().binding { + logger.info("Using configuration: $config") + HealthCheckServer.start(config).bind() + VesServer.start(config).bind() + .await().bind() + }.fix() diff --git a/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/servers/HealthCheckServer.kt b/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/servers/HealthCheckServer.kt new file mode 100644 index 00000000..ae59da69 --- /dev/null +++ b/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/servers/HealthCheckServer.kt @@ -0,0 +1,41 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.main.servers + +import org.onap.dcae.collectors.veshv.healthcheck.api.HealthState +import org.onap.dcae.collectors.veshv.healthcheck.factory.HealthCheckApiServer +import org.onap.dcae.collectors.veshv.model.ServerConfiguration +import org.onap.dcae.collectors.veshv.utils.ServerHandle + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since August 2018 + */ +object HealthCheckServer : ServerStarter() { + override fun startServer(config: ServerConfiguration) = createHealthCheckServer(config).start() + + private fun createHealthCheckServer(config: ServerConfiguration) = + HealthCheckApiServer( + HealthState.INSTANCE, + config.healthCheckApiListenAddress) + + override fun serverStartedMessage(handle: ServerHandle) = + "Health check server is up and listening on ${handle.host}:${handle.port}" +} diff --git a/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/servers/ServerStarter.kt b/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/servers/ServerStarter.kt new file mode 100644 index 00000000..5c6f1277 --- /dev/null +++ b/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/servers/ServerStarter.kt @@ -0,0 +1,42 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.main.servers + +import arrow.effects.IO +import org.onap.dcae.collectors.veshv.model.ServerConfiguration +import org.onap.dcae.collectors.veshv.utils.ServerHandle +import org.onap.dcae.collectors.veshv.utils.logging.Logger + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since August 2018 + */ +abstract class ServerStarter { + fun start(config: ServerConfiguration): IO<ServerHandle> = + startServer(config) + .map { logger.info(serverStartedMessage(it)); it } + + protected abstract fun startServer(config: ServerConfiguration): IO<ServerHandle> + protected abstract fun serverStartedMessage(handle: ServerHandle): String + + companion object { + private val logger = Logger(ServerStarter::class) + } +} diff --git a/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/servers/VesServer.kt b/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/servers/VesServer.kt new file mode 100644 index 00000000..d788c164 --- /dev/null +++ b/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/servers/VesServer.kt @@ -0,0 +1,52 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.main.servers + +import arrow.effects.IO +import org.onap.dcae.collectors.veshv.boundary.Server +import org.onap.dcae.collectors.veshv.factory.CollectorFactory +import org.onap.dcae.collectors.veshv.factory.ServerFactory +import org.onap.dcae.collectors.veshv.impl.adapters.AdapterFactory +import org.onap.dcae.collectors.veshv.main.MicrometerMetrics +import org.onap.dcae.collectors.veshv.model.ServerConfiguration +import org.onap.dcae.collectors.veshv.utils.ServerHandle + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since August 2018 + */ +object VesServer : ServerStarter() { + override fun startServer(config: ServerConfiguration): IO<ServerHandle> = createVesServer(config).start() + + private fun createVesServer(config: ServerConfiguration): Server { + val sink = if (config.dummyMode) AdapterFactory.loggingSink() else AdapterFactory.kafkaSink() + val collectorProvider = CollectorFactory( + AdapterFactory.consulConfigurationProvider(config.configurationProviderParams), + sink, + MicrometerMetrics(), + config.maximumPayloadSizeBytes + ).createVesHvCollectorProvider() + + return ServerFactory.createNettyTcpServer(config, collectorProvider) + } + + override fun serverStartedMessage(handle: ServerHandle) = + "HighVolume VES Collector is up and listening on ${handle.host}:${handle.port}" +} diff --git a/sources/hv-collector-main/src/main/resources/logback.xml b/sources/hv-collector-main/src/main/resources/logback.xml new file mode 100644 index 00000000..c76ff21a --- /dev/null +++ b/sources/hv-collector-main/src/main/resources/logback.xml @@ -0,0 +1,49 @@ +<?xml version="1.0" encoding="UTF-8"?> +<configuration> + <property name="COMPONENT_NAME" + value="dcae-hv-ves-collector"/> + <property name="COMPONENT_SHORT_NAME" + value="hv-ves"/> + + <property name="LOG_FILENAME" value="${COMPONENT_SHORT_NAME}"/> + <property name="LOG_PATH" value="/var/log/ONAP/${COMPONENT_NAME}"/> + <property name="ARCHIVE" value="${LOG_PATH}/archive"/> + <property name="FILE_LOG_PATTERN" value=" +%nopexception%50.50logger +| %date{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} +| %highlight(%-5level) +| %msg +| %rootException +| %thread%n"/> + + <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> + <encoder> + <pattern>${FILE_LOG_PATTERN}</pattern> + </encoder> + </appender> + + <appender name="ROLLING-FILE" + class="ch.qos.logback.core.rolling.RollingFileAppender"> + <encoder> + <pattern>${FILE_LOG_PATTERN}</pattern> + </encoder> + <file>${LOG_PATH}/${LOG_FILENAME}.log</file> + <rollingPolicy class="ch.qos.logback.core.rolling.SizeAndTimeBasedRollingPolicy"> + <FileNamePattern>${ARCHIVE}/${LOG_FILENAME}.%d{yyyy-MM-dd}.%i.log.gz</FileNamePattern> + <maxFileSize>50MB</maxFileSize> + <maxHistory>30</maxHistory> + <totalSizeCap>10GB</totalSizeCap> + </rollingPolicy> + </appender> + + <logger name="org.onap.dcae.collectors.veshv" level="DEBUG"/> + <logger name="org.onap.dcae.collectors.veshv.impl.wire" level="DEBUG"/> + <logger name="org.onap.dcae.collectors.veshv.impl.adapters.kafka.KafkaSink" level="DEBUG"/> + <logger name="org.onap.dcae.collectors.veshv.impl.adapters.LoggingSinkProvider" level="DEBUG"/> + <!--<logger name="reactor.ipc.netty" level="DEBUG"/>--> + + <root level="INFO"> + <appender-ref ref="CONSOLE"/> + <appender-ref ref="ROLLING-FILE"/> + </root> +</configuration>
\ No newline at end of file diff --git a/sources/hv-collector-main/src/main/scripts/healthcheck.sh b/sources/hv-collector-main/src/main/scripts/healthcheck.sh new file mode 100755 index 00000000..db62eece --- /dev/null +++ b/sources/hv-collector-main/src/main/scripts/healthcheck.sh @@ -0,0 +1,4 @@ +#!/usr/bin/env bash + +curl -f http://localhost:${VESHV_HEALTH_CHECK_API_PORT:-6060}/health/ready || exit 1 +nc -vz localhost ${VESHV_LISTEN_PORT:-6061} || exit 2 diff --git a/sources/hv-collector-main/src/test/kotlin/org/onap/dcae/collectors/veshv/main/ArgVesHvConfigurationTest.kt b/sources/hv-collector-main/src/test/kotlin/org/onap/dcae/collectors/veshv/main/ArgVesHvConfigurationTest.kt new file mode 100644 index 00000000..1aac6a09 --- /dev/null +++ b/sources/hv-collector-main/src/test/kotlin/org/onap/dcae/collectors/veshv/main/ArgVesHvConfigurationTest.kt @@ -0,0 +1,138 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.main + +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.given +import org.jetbrains.spek.api.dsl.it +import org.jetbrains.spek.api.dsl.on +import org.onap.dcae.collectors.veshv.domain.JdkKeys +import org.onap.dcae.collectors.veshv.model.ServerConfiguration +import org.onap.dcae.collectors.veshv.tests.utils.parseExpectingFailure +import org.onap.dcae.collectors.veshv.tests.utils.parseExpectingSuccess +import org.onap.dcae.collectors.veshv.utils.commandline.WrongArgumentError +import java.time.Duration +import kotlin.test.assertNotNull + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +object ArgVesHvConfigurationTest : Spek({ + lateinit var cut: ArgVesHvConfiguration + val healthCheckApiPort = "6070" + val configurationUrl = "http://test-address/test" + val firstRequestDelay = "10" + val requestInterval = "5" + val listenPort = "6969" + val keyStorePassword = "kspass" + val trustStorePassword = "tspass" + + beforeEachTest { + cut = ArgVesHvConfiguration() + } + + describe("parsing arguments") { + given("all parameters are present in the long form") { + lateinit var result: ServerConfiguration + + beforeEachTest { + result = cut.parseExpectingSuccess( + "--health-check-api-port", healthCheckApiPort, + "--listen-port", listenPort, + "--config-url", configurationUrl, + "--first-request-delay", firstRequestDelay, + "--request-interval", requestInterval, + "--key-store", "/tmp/keys.p12", + "--trust-store", "/tmp/trust.p12", + "--key-store-password", keyStorePassword, + "--trust-store-password", trustStorePassword + ) + } + + it("should set proper listen port") { + assertThat(result.serverListenAddress.port).isEqualTo(listenPort.toInt()) + } + + + it("should set default listen address") { + assertThat(result.serverListenAddress.address.hostAddress).isEqualTo("0.0.0.0") + } + + it("should set proper health check api port") { + assertThat(result.healthCheckApiListenAddress.port).isEqualTo(healthCheckApiPort.toInt()) + } + + it("should set default health check api address") { + assertThat(result.healthCheckApiListenAddress.address.hostAddress).isEqualTo("0.0.0.0") + } + + it("should set proper first consul request delay") { + assertThat(result.configurationProviderParams.firstRequestDelay) + .isEqualTo(Duration.ofSeconds(firstRequestDelay.toLong())) + } + + it("should set proper consul request interval") { + assertThat(result.configurationProviderParams.requestInterval) + .isEqualTo(Duration.ofSeconds(requestInterval.toLong())) + } + + it("should set proper config url") { + assertThat(result.configurationProviderParams.configurationUrl) + .isEqualTo(configurationUrl) + } + + it("should set proper security configuration") { + assertThat(result.securityConfiguration.sslDisable).isFalse() + + val keys = result.securityConfiguration.keys.orNull() as JdkKeys + assertNotNull(keys.keyStore) + assertNotNull(keys.trustStore) + assertThat(keys.keyStorePassword).isEqualTo(keyStorePassword.toCharArray()) + assertThat(keys.trustStorePassword).isEqualTo(trustStorePassword.toCharArray()) + } + } + + describe("required parameter is absent") { + on("missing listen port") { + it("should throw exception") { + assertThat(cut.parseExpectingFailure( + "--config-url", configurationUrl, + "--ssl-disable", + "--first-request-delay", firstRequestDelay, + "--request-interval", requestInterval) + ).isInstanceOf(WrongArgumentError::class.java) + } + } + on("missing configuration url") { + it("should throw exception") { + assertThat(cut.parseExpectingFailure( + "--listen-port", listenPort, + "--ssl-disable", + "--first-request-delay", firstRequestDelay, + "--request-interval", requestInterval) + ).isInstanceOf(WrongArgumentError::class.java) + } + } + } + } +})
\ No newline at end of file diff --git a/sources/hv-collector-main/src/test/kotlin/org/onap/dcae/collectors/veshv/main/MicrometerMetricsTest.kt b/sources/hv-collector-main/src/test/kotlin/org/onap/dcae/collectors/veshv/main/MicrometerMetricsTest.kt new file mode 100644 index 00000000..a379933e --- /dev/null +++ b/sources/hv-collector-main/src/test/kotlin/org/onap/dcae/collectors/veshv/main/MicrometerMetricsTest.kt @@ -0,0 +1,191 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.main + +import arrow.core.Try +import io.micrometer.core.instrument.Counter +import io.micrometer.core.instrument.Gauge +import io.micrometer.core.instrument.search.RequiredSearch +import io.micrometer.core.instrument.simple.SimpleMeterRegistry +import org.assertj.core.api.Assertions.assertThat +import org.assertj.core.data.Percentage +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 + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ +object MicrometerMetricsTest : Spek({ + val doublePrecision = Percentage.withPercentage(0.5) + lateinit var registry: SimpleMeterRegistry + lateinit var cut: MicrometerMetrics + + beforeEachTest { + registry = SimpleMeterRegistry() + cut = MicrometerMetrics(registry) + } + + fun registrySearch() = RequiredSearch.`in`(registry) + + fun <M, T> verifyMeter(search: RequiredSearch, map: (RequiredSearch) -> M, verifier: (M) -> T) = + Try { + map(search) + }.fold( + { ex -> assertThat(ex).doesNotThrowAnyException() }, + verifier + ) + + fun <T> verifyGauge(name: String, verifier: (Gauge) -> T) = + verifyMeter(registrySearch().name(name), RequiredSearch::gauge, verifier) + + fun <T> verifyCounter(search: RequiredSearch, verifier: (Counter) -> T) = + verifyMeter(search, RequiredSearch::counter, verifier) + + fun <T> verifyCounter(name: String, verifier: (Counter) -> T) = + verifyCounter(registrySearch().name(name), verifier) + + fun verifyAllCountersAreUnchangedBut(vararg changedCounters: String) { + registry.meters + .filter { it is Counter } + .filterNot { it.id.name in changedCounters } + .forEach { assertThat((it as Counter).count()).isCloseTo(0.0, doublePrecision) } + } + + describe("notifyBytesReceived") { + + on("data.received.bytes counter") { + val counterName = "data.received.bytes" + + it("should increment counter") { + val bytes = 128 + cut.notifyBytesReceived(bytes) + + verifyCounter(counterName) { counter -> + assertThat(counter.count()).isCloseTo(bytes.toDouble(), doublePrecision) + } + } + + it("should leave all other counters unchanged") { + cut.notifyBytesReceived(128) + verifyAllCountersAreUnchangedBut(counterName) + } + } + } + + describe("notifyMessageReceived") { + on("messages.received.count counter") { + val counterName = "messages.received.count" + + it("should increment counter") { + cut.notifyMessageReceived(777) + + verifyCounter(counterName) { counter -> + assertThat(counter.count()).isCloseTo(1.0, doublePrecision) + } + } + } + + on("messages.received.bytes counter") { + val counterName = "messages.received.bytes" + + it("should increment counter") { + val bytes = 888 + cut.notifyMessageReceived(bytes) + + verifyCounter(counterName) { counter -> + assertThat(counter.count()).isCloseTo(bytes.toDouble(), doublePrecision) + } + } + } + + it("should leave all other counters unchanged") { + cut.notifyMessageReceived(128) + verifyAllCountersAreUnchangedBut("messages.received.count", "messages.received.bytes") + } + } + + describe("notifyMessageSent") { + val topicName = "dmaap_topic_name" + val counterName = "messages.sent.count" + + on("$counterName counter") { + + it("should increment counter") { + cut.notifyMessageSent(topicName) + + verifyCounter(counterName) { counter -> + assertThat(counter.count()).isCloseTo(1.0, doublePrecision) + } + } + } + + on("$counterName[topic=$topicName] counter") { + + it("should increment counter") { + cut.notifyMessageSent(topicName) + + verifyCounter(registrySearch().name(counterName).tag("topic", topicName)) { counter -> + assertThat(counter.count()).isCloseTo(1.0, doublePrecision) + } + } + } + + it("should leave all other counters unchanged") { + cut.notifyMessageSent(topicName) + verifyAllCountersAreUnchangedBut(counterName) + } + } + + describe("processing gauge") { + it("should show difference between sent and received messages") { + + on("positive difference") { + cut.notifyMessageReceived(128) + cut.notifyMessageReceived(256) + cut.notifyMessageReceived(256) + cut.notifyMessageSent("perf3gpp") + verifyGauge("messages.processing.count") { gauge -> + assertThat(gauge.value()).isCloseTo(2.0, doublePrecision) + } + } + + on("zero difference") { + cut.notifyMessageReceived(128) + cut.notifyMessageSent("perf3gpp") + verifyGauge("messages.processing.count") { gauge -> + assertThat(gauge.value()).isCloseTo(0.0, doublePrecision) + } + } + + on("negative difference") { + cut.notifyMessageReceived(128) + cut.notifyMessageSent("fault") + cut.notifyMessageSent("perf3gpp") + verifyGauge("messages.processing.count") { gauge -> + assertThat(gauge.value()).isCloseTo(0.0, doublePrecision) + } + } + } + } + +}) diff --git a/sources/hv-collector-main/src/test/kotlin/org/onap/dcae/collectors/veshv/main/NioBuffersTest.kt b/sources/hv-collector-main/src/test/kotlin/org/onap/dcae/collectors/veshv/main/NioBuffersTest.kt new file mode 100644 index 00000000..4eef28bb --- /dev/null +++ b/sources/hv-collector-main/src/test/kotlin/org/onap/dcae/collectors/veshv/main/NioBuffersTest.kt @@ -0,0 +1,88 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.main + +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.xdescribe +import java.nio.ByteBuffer + +object NioBuffersTest : Spek({ + + fun Int.toKibibytes(): Int = this * 1024 + fun Int.toMebibytes(): Int = this * 1024 * 1024 + + val BUFFER_SIZES = listOf(128.toKibibytes(), 512.toKibibytes(), 1.toMebibytes(), 2.toMebibytes()) + val NUMBER_OF_ITERATIONS = 100 + + fun measureCopyTimeInNanos(bb1: ByteBuffer, bb2: ByteBuffer): Double { + bb1.clear() + bb2.clear() + val start = System.nanoTime() + while (bb2.remaining() > 0) + bb2.putInt(bb1.getInt()) + val time = System.nanoTime() - start + val operations = bb1.capacity() / Integer.BYTES + return time.toDouble() / operations + } + + fun measureAverageCopyTimeInNanos(bb1: ByteBuffer, bb2: ByteBuffer): Double = + (0..NUMBER_OF_ITERATIONS).map { measureCopyTimeInNanos(bb1, bb2) }.average() + + fun measureAndPrintAverageCopyTime(message: String, bb1: ByteBuffer, bb2: ByteBuffer) { + val avg = measureAverageCopyTimeInNanos(bb1, bb2) + System.out.printf("Each putInt+getInt for %s took an average of %.1f ns%n", message, avg) + } + + for (singleBufferSize in BUFFER_SIZES) { + + xdescribe("$singleBufferSize bytes buffers") { + describe("direct buffers") { + + val bb1 = ByteBuffer.allocateDirect(singleBufferSize) + val bb2 = ByteBuffer.allocateDirect(singleBufferSize) + + it("should be heated up") { + measureAverageCopyTimeInNanos(bb1, bb2) + } + + it("should work fast") { + measureAndPrintAverageCopyTime("direct buffers of $singleBufferSize bytes", bb1, bb2) + } + } + + describe("on-heap buffers") { + + val bb1 = ByteBuffer.allocate(singleBufferSize) + val bb2 = ByteBuffer.allocate(singleBufferSize) + + it("should be heated up") { + measureAverageCopyTimeInNanos(bb1, bb2) + } + + it("should work fast") { + measureAndPrintAverageCopyTime("onheap buffers of $singleBufferSize bytes", bb1, bb2) + } + } + } + } + +})
\ No newline at end of file diff --git a/sources/hv-collector-ssl/pom.xml b/sources/hv-collector-ssl/pom.xml new file mode 100644 index 00000000..6001ba94 --- /dev/null +++ b/sources/hv-collector-ssl/pom.xml @@ -0,0 +1,118 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ ============LICENSE_START======================================================= + ~ dcaegen2-collectors-veshv + ~ ================================================================================ + ~ Copyright (C) 2018 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========================================================= + --> +<project xmlns="http://maven.apache.org/POM/4.0.0" + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + + <licenses> + <license> + <name>The Apache Software License, Version 2.0</name> + <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url> + </license> + </licenses> + + <parent> + <groupId>org.onap.dcaegen2.collectors.hv-ves</groupId> + <artifactId>hv-collector-sources</artifactId> + <version>1.1.0-SNAPSHOT</version> + <relativePath>..</relativePath> + </parent> + + <artifactId>hv-collector-ssl</artifactId> + <description>VES HighVolume Collector :: SSL</description> + + <properties> + <skipAnalysis>false</skipAnalysis> + </properties> + + <build> + <plugins> + <plugin> + <artifactId>kotlin-maven-plugin</artifactId> + <groupId>org.jetbrains.kotlin</groupId> + </plugin> + <plugin> + <artifactId>maven-surefire-plugin</artifactId> + <groupId>org.apache.maven.plugins</groupId> + </plugin> + </plugins> + </build> + + <dependencies> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-domain</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-utils</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-test-utils</artifactId> + <version>${project.parent.version}</version> + <scope>test</scope> + </dependency> + <dependency> + <groupId>io.netty</groupId> + <artifactId>netty-tcnative-boringssl-static</artifactId> + <classifier>${os.detected.classifier}</classifier> + <scope>test</scope> + </dependency> + <dependency> + <groupId>io.arrow-kt</groupId> + <artifactId>arrow-core</artifactId> + </dependency> + <dependency> + <groupId>io.arrow-kt</groupId> + <artifactId>arrow-syntax</artifactId> + </dependency> + <dependency> + <groupId>org.slf4j</groupId> + <artifactId>slf4j-api</artifactId> + </dependency> + <dependency> + <groupId>ch.qos.logback</groupId> + <artifactId>logback-classic</artifactId> + </dependency> + <dependency> + <groupId>org.assertj</groupId> + <artifactId>assertj-core</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.kotlin</groupId> + <artifactId>kotlin-test</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.spek</groupId> + <artifactId>spek-api</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.spek</groupId> + <artifactId>spek-junit-platform-engine</artifactId> + </dependency> + </dependencies> + + +</project> diff --git a/sources/hv-collector-ssl/src/main/kotlin/org/onap/dcae/collectors/veshv/ssl/boundary/ClientSslContextFactory.kt b/sources/hv-collector-ssl/src/main/kotlin/org/onap/dcae/collectors/veshv/ssl/boundary/ClientSslContextFactory.kt new file mode 100644 index 00000000..0ad3d7b4 --- /dev/null +++ b/sources/hv-collector-ssl/src/main/kotlin/org/onap/dcae/collectors/veshv/ssl/boundary/ClientSslContextFactory.kt @@ -0,0 +1,52 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ssl.boundary + +import io.netty.handler.ssl.SslContextBuilder +import io.netty.handler.ssl.SslProvider +import org.onap.dcae.collectors.veshv.domain.JdkKeys +import org.onap.dcae.collectors.veshv.domain.OpenSslKeys +import org.onap.dcae.collectors.veshv.ssl.impl.SslFactories.keyManagerFactory +import org.onap.dcae.collectors.veshv.ssl.impl.SslFactories.trustManagerFactory + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since September 2018 + */ +open class ClientSslContextFactory : SslContextFactory() { + + override fun openSslContext(openSslKeys: OpenSslKeys) = SslContextBuilder.forClient() + .keyManager(openSslKeys.cert.toFile(), openSslKeys.privateKey.toFile()) + .trustManager(openSslKeys.trustedCert.toFile()) + .sslProvider(SslProvider.OPENSSL)!! + + override fun jdkContext(jdkKeys: JdkKeys) = + try { + val kmf = keyManagerFactory(jdkKeys) + val tmf = trustManagerFactory(jdkKeys) + SslContextBuilder.forClient() + .keyManager(kmf) + .trustManager(tmf) + .sslProvider(SslProvider.JDK)!! + } finally { + jdkKeys.forgetPasswords() + } + +} diff --git a/sources/hv-collector-ssl/src/main/kotlin/org/onap/dcae/collectors/veshv/ssl/boundary/ServerSslContextFactory.kt b/sources/hv-collector-ssl/src/main/kotlin/org/onap/dcae/collectors/veshv/ssl/boundary/ServerSslContextFactory.kt new file mode 100644 index 00000000..d26937fc --- /dev/null +++ b/sources/hv-collector-ssl/src/main/kotlin/org/onap/dcae/collectors/veshv/ssl/boundary/ServerSslContextFactory.kt @@ -0,0 +1,50 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ssl.boundary + +import io.netty.handler.ssl.SslContextBuilder +import io.netty.handler.ssl.SslProvider +import org.onap.dcae.collectors.veshv.domain.JdkKeys +import org.onap.dcae.collectors.veshv.domain.OpenSslKeys +import org.onap.dcae.collectors.veshv.ssl.impl.SslFactories.keyManagerFactory +import org.onap.dcae.collectors.veshv.ssl.impl.SslFactories.trustManagerFactory + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since September 2018 + */ +open class ServerSslContextFactory : SslContextFactory() { + + override fun openSslContext(openSslKeys: OpenSslKeys) = SslContextBuilder + .forServer(openSslKeys.cert.toFile(), openSslKeys.privateKey.toFile()) + .trustManager(openSslKeys.trustedCert.toFile()) + .sslProvider(SslProvider.OPENSSL)!! + + override fun jdkContext(jdkKeys: JdkKeys) = + try { + val kmf = keyManagerFactory(jdkKeys) + val tmf = trustManagerFactory(jdkKeys) + SslContextBuilder.forServer(kmf) + .trustManager(tmf) + .sslProvider(SslProvider.JDK)!! + } finally { + jdkKeys.forgetPasswords() + } +} diff --git a/sources/hv-collector-ssl/src/main/kotlin/org/onap/dcae/collectors/veshv/ssl/boundary/SslContextFactory.kt b/sources/hv-collector-ssl/src/main/kotlin/org/onap/dcae/collectors/veshv/ssl/boundary/SslContextFactory.kt new file mode 100644 index 00000000..cad81eef --- /dev/null +++ b/sources/hv-collector-ssl/src/main/kotlin/org/onap/dcae/collectors/veshv/ssl/boundary/SslContextFactory.kt @@ -0,0 +1,58 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ssl.boundary + +import arrow.core.Option +import io.netty.handler.ssl.ClientAuth +import io.netty.handler.ssl.SslContext +import io.netty.handler.ssl.SslContextBuilder +import org.onap.dcae.collectors.veshv.domain.JdkKeys +import org.onap.dcae.collectors.veshv.domain.OpenSslKeys +import org.onap.dcae.collectors.veshv.domain.SecurityConfiguration + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since September 2018 + */ +abstract class SslContextFactory { + fun createSslContext(secConfig: SecurityConfiguration): Option<SslContext> = + if (secConfig.sslDisable) { + Option.empty() + } else { + createSslContextWithConfiguredCerts(secConfig) + .map { builder -> + builder.clientAuth(ClientAuth.REQUIRE) + .build() + } + } + + protected open fun createSslContextWithConfiguredCerts( + secConfig: SecurityConfiguration + ): Option<SslContextBuilder> = + secConfig.keys.map { keys -> + when (keys) { + is JdkKeys -> jdkContext(keys) + is OpenSslKeys -> openSslContext(keys) + } + } + + protected abstract fun openSslContext(openSslKeys: OpenSslKeys): SslContextBuilder + protected abstract fun jdkContext(jdkKeys: JdkKeys): SslContextBuilder +} diff --git a/sources/hv-collector-ssl/src/main/kotlin/org/onap/dcae/collectors/veshv/ssl/boundary/utils.kt b/sources/hv-collector-ssl/src/main/kotlin/org/onap/dcae/collectors/veshv/ssl/boundary/utils.kt new file mode 100644 index 00000000..d3640c87 --- /dev/null +++ b/sources/hv-collector-ssl/src/main/kotlin/org/onap/dcae/collectors/veshv/ssl/boundary/utils.kt @@ -0,0 +1,79 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ssl.boundary + +import arrow.core.None +import arrow.core.Option +import arrow.core.Some +import arrow.core.fix +import arrow.instances.option.monad.monad +import arrow.typeclasses.binding +import org.apache.commons.cli.CommandLine +import org.onap.dcae.collectors.veshv.domain.JdkKeys +import org.onap.dcae.collectors.veshv.domain.SecurityConfiguration +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption +import org.onap.dcae.collectors.veshv.utils.commandline.hasOption +import org.onap.dcae.collectors.veshv.utils.commandline.stringValue +import java.io.File + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since September 2018 + */ + + +const val KEY_STORE_FILE = "/etc/ves-hv/server.p12" +const val TRUST_STORE_FILE = "/etc/ves-hv/trust.p12" + +fun createSecurityConfiguration(cmdLine: CommandLine): Option<SecurityConfiguration> { + val sslDisable = cmdLine.hasOption(CommandLineOption.SSL_DISABLE) + + return if (sslDisable) disabledSecurityConfiguration(sslDisable) else enabledSecurityConfiguration(cmdLine) +} + +private fun disabledSecurityConfiguration(sslDisable: Boolean): Some<SecurityConfiguration> { + return Some(SecurityConfiguration( + sslDisable = sslDisable, + keys = None + )) +} + +private fun enabledSecurityConfiguration(cmdLine: CommandLine): Option<SecurityConfiguration> { + return Option.monad().binding { + val ksFile = cmdLine.stringValue(CommandLineOption.KEY_STORE_FILE, KEY_STORE_FILE) + val ksPass = cmdLine.stringValue(CommandLineOption.KEY_STORE_PASSWORD).bind() + val tsFile = cmdLine.stringValue(CommandLineOption.TRUST_STORE_FILE, TRUST_STORE_FILE) + val tsPass = cmdLine.stringValue(CommandLineOption.TRUST_STORE_PASSWORD).bind() + + val keys = JdkKeys( + keyStore = streamFromFile(ksFile), + keyStorePassword = ksPass.toCharArray(), + trustStore = streamFromFile(tsFile), + trustStorePassword = tsPass.toCharArray() + ) + + SecurityConfiguration( + sslDisable = false, + keys = Some(keys) + ) + }.fix() +} + +private fun streamFromFile(file: String) = { File(file).inputStream() } diff --git a/sources/hv-collector-ssl/src/main/kotlin/org/onap/dcae/collectors/veshv/ssl/impl/SslFactories.kt b/sources/hv-collector-ssl/src/main/kotlin/org/onap/dcae/collectors/veshv/ssl/impl/SslFactories.kt new file mode 100644 index 00000000..4a73a2aa --- /dev/null +++ b/sources/hv-collector-ssl/src/main/kotlin/org/onap/dcae/collectors/veshv/ssl/impl/SslFactories.kt @@ -0,0 +1,55 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ssl.impl + +import org.onap.dcae.collectors.veshv.domain.JdkKeys +import org.onap.dcae.collectors.veshv.domain.StreamProvider +import java.security.KeyStore +import javax.net.ssl.KeyManagerFactory +import javax.net.ssl.TrustManagerFactory + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since September 2018 + */ +internal object SslFactories { + + fun trustManagerFactory(jdkKeys: JdkKeys): TrustManagerFactory? { + val tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()) + val ts = loadKeyStoreFromFile(jdkKeys.trustStore, jdkKeys.trustStorePassword) + tmf.init(ts) + return tmf + } + + fun keyManagerFactory(jdkKeys: JdkKeys): KeyManagerFactory? { + val kmf = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm()) + val ks = loadKeyStoreFromFile(jdkKeys.keyStore, jdkKeys.keyStorePassword) + kmf.init(ks, jdkKeys.keyStorePassword) + return kmf + } + + private fun loadKeyStoreFromFile(streamProvider: StreamProvider, password: CharArray): KeyStore { + val ks = KeyStore.getInstance("pkcs12") + streamProvider().use { + ks.load(it, password) + } + return ks + } +} diff --git a/sources/hv-collector-ssl/src/test/kotlin/org/onap/dcae/collectors/veshv/ssl/boundary/ServerSslContextFactoryTest.kt b/sources/hv-collector-ssl/src/test/kotlin/org/onap/dcae/collectors/veshv/ssl/boundary/ServerSslContextFactoryTest.kt new file mode 100644 index 00000000..7e0bc609 --- /dev/null +++ b/sources/hv-collector-ssl/src/test/kotlin/org/onap/dcae/collectors/veshv/ssl/boundary/ServerSslContextFactoryTest.kt @@ -0,0 +1,160 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ssl.boundary + +import arrow.core.Some +import arrow.core.toOption +import io.netty.handler.ssl.ClientAuth +import io.netty.handler.ssl.JdkSslContext +import io.netty.handler.ssl.ReferenceCountedOpenSslContext +import io.netty.handler.ssl.SslContextBuilder +import org.assertj.core.api.Assertions +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.given +import org.jetbrains.spek.api.dsl.it +import org.jetbrains.spek.api.dsl.on +import org.onap.dcae.collectors.veshv.domain.JdkKeys +import org.onap.dcae.collectors.veshv.domain.OpenSslKeys +import org.onap.dcae.collectors.veshv.domain.SecurityConfiguration +import java.nio.file.Paths +import kotlin.test.assertTrue + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ +object ServerSslContextFactoryTest : Spek({ + val PASSWORD = "onap" + + describe("SslContextFactory (OpenSSL)") { + val keys = OpenSslKeys( + privateKey = Paths.get("/", "tmp", "pk.pem"), + cert = Paths.get("/", "tmp", "cert.crt"), + trustedCert = Paths.get("/", "tmp", "clientCa.crt")) + + given("config with security enabled") { + val sampleConfig = SecurityConfiguration(keys = Some(keys)) + + val cut = object : ServerSslContextFactory() { + override fun createSslContextWithConfiguredCerts(secConfig: SecurityConfiguration) = + SslContextBuilder.forServer(resource("/ssl/ca.crt"), resource("/ssl/server.key")).toOption() + + private fun resource(path: String) = ServerSslContextFactoryTest.javaClass.getResourceAsStream(path) + } + + on("creation of SSL context") { + val result = cut.createSslContext(sampleConfig) + + it("should be server context") { + assertTrue(result.exists { + it.isServer + }) + } + + it("should use OpenSSL provider") { + assertTrue(result.isDefined()) + } + + /* + * It is too important to leave it untested on unit level. + * Because of the Netty API design we need to do it this way. + */ + it("should turn on client authentication") { + val clientAuth: ClientAuth = ReferenceCountedOpenSslContext::class.java + .getDeclaredField("clientAuth") + .run { + isAccessible = true + get(result.orNull()) as ClientAuth + } + Assertions.assertThat(clientAuth).isEqualTo(ClientAuth.REQUIRE) + } + } + } + + given("config with SSL disabled") { + val securityConfiguration = SecurityConfiguration( + sslDisable = true, + keys = Some(keys) + ) + val cut = ServerSslContextFactory() + + on("creation of SSL context") { + val result = cut.createSslContext(securityConfiguration) + + it("should not create any SSL context ") { + assertThat(result.isDefined()).isFalse() + } + } + } + } + + describe("SslContextFactory (JDK)") { + val keys = JdkKeys( + keyStore = resourceStreamProvider("/ssl/server.ks.pkcs12"), + keyStorePassword = PASSWORD.toCharArray(), + trustStore = resourceStreamProvider("/ssl/trust.pkcs12"), + trustStorePassword = PASSWORD.toCharArray() + ) + + given("config without disabled SSL") { + val sampleConfig = SecurityConfiguration(keys = Some(keys)) + val cut = ServerSslContextFactory() + + on("creation of SSL context") { + val result = cut.createSslContext(sampleConfig) + + it("should work") { + assertTrue(result.isDefined()) + } + + it("should be server context") { + assertTrue(result.exists { + it.isServer + }) + } + + /* + * It is too important to leave it untested on unit level. + * Because of the Netty API design we need to do it this way. + */ + it("should turn on client authentication") { + val clientAuth: ClientAuth = JdkSslContext::class.java + .getDeclaredField("clientAuth") + .run { + isAccessible = true + get(result.orNull()) as ClientAuth + } + Assertions.assertThat(clientAuth).isEqualTo(ClientAuth.REQUIRE) + } + + it("should clear passwords so heap dumps won't contain them") { + val xedPassword = PASSWORD.toCharArray() + xedPassword.fill('x') + Assertions.assertThat(keys.keyStorePassword).isEqualTo(xedPassword) + Assertions.assertThat(keys.trustStorePassword).isEqualTo(xedPassword) + } + } + } + } +}) + +fun resourceStreamProvider(resource: String) = { ServerSslContextFactoryTest::class.java.getResourceAsStream(resource) } diff --git a/sources/hv-collector-ssl/src/test/resources/logback-test.xml b/sources/hv-collector-ssl/src/test/resources/logback-test.xml new file mode 100644 index 00000000..9a4eacfe --- /dev/null +++ b/sources/hv-collector-ssl/src/test/resources/logback-test.xml @@ -0,0 +1,35 @@ +<?xml version="1.0" encoding="UTF-8"?> +<configuration> + <property name="LOG_FILE" + value="${LOG_FILE:-${LOG_PATH:-${LOG_TEMP:-${java.io.tmpdir:-/tmp}}/}ves-hv.log}"/> + <property name="FILE_LOG_PATTERN" value="%d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %-5level [%-40.40logger{10}] - %msg%n"/> + + <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> + <encoder> + <pattern> + %d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %highlight(%-5level) [%-40.40logger{10}] - %msg%n + </pattern> + </encoder> + </appender> + + <appender name="ROLLING-FILE" + class="ch.qos.logback.core.rolling.RollingFileAppender"> + <encoder> + <pattern>${FILE_LOG_PATTERN}</pattern> + </encoder> + <file>${LOG_FILE}</file> + <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy"> + <fileNamePattern>${LOG_FILE}.%d{yyyy-MM-dd}.log</fileNamePattern> + <maxFileSize>50MB</maxFileSize> + <maxHistory>30</maxHistory> + <totalSizeCap>10GB</totalSizeCap> + </rollingPolicy> + </appender> + + <logger name="org.onap.dcae.collectors.veshv" level="TRACE"/> + + <root level="INFO"> + <appender-ref ref="CONSOLE"/> + <appender-ref ref="ROLLING-FILE"/> + </root> +</configuration> diff --git a/sources/hv-collector-ssl/src/test/resources/ssl/ca.crt b/sources/hv-collector-ssl/src/test/resources/ssl/ca.crt new file mode 100644 index 00000000..f9a05b90 --- /dev/null +++ b/sources/hv-collector-ssl/src/test/resources/ssl/ca.crt @@ -0,0 +1,21 @@ +-----BEGIN CERTIFICATE----- +MIIDbDCCAlSgAwIBAgIJAMsh09jY3jSBMA0GCSqGSIb3DQEBCwUAMEsxCzAJBgNV +BAYTAlBMMQswCQYDVQQIDAJETDEQMA4GA1UEBwwHV3JvY2xhdzEOMAwGA1UECgwF +Tm9raWExDTALBgNVBAsMBE1BTk8wHhcNMTgwODEzMTE1OTE5WhcNMTkwODEzMTE1 +OTE5WjBLMQswCQYDVQQGEwJQTDELMAkGA1UECAwCREwxEDAOBgNVBAcMB1dyb2Ns +YXcxDjAMBgNVBAoMBU5va2lhMQ0wCwYDVQQLDARNQU5PMIIBIjANBgkqhkiG9w0B +AQEFAAOCAQ8AMIIBCgKCAQEAyIs4sk5SlBRltDHZDDIGHdazzdduPRKdDBMUTE5i +++iZe1MU9WnbPWgbnfj/1DN+VbbJEa32agDXBhS9mPiSrflDe8oqPPk12miiflsY +TxGxY1bjH58kRNey164fPznzc9LqKqV1brUPktgBkHumM4zPVZYue5cp1T2FiWcE +nFdqOKK2F0mdby+Pim9JHil34YcvavOlMXULkqayR7hxfUIAmMZtl59BKVx9BcKD +9Sv9TQYRIYHhymrKnwIb8RolDaDdVUnr2ryREjZ8WfoBsOpraIYDUjKijF5FlgLs +Fp8wcIflF6JyMbtchTm5acQZkcqK1AGYTD1wxT33E2pdDQIDAQABo1MwUTAdBgNV +HQ4EFgQUOL9AR068MxqYCsKkEfgDzF2HrY8wHwYDVR0jBBgwFoAUOL9AR068MxqY +CsKkEfgDzF2HrY8wDwYDVR0TAQH/BAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEA +OXNDDqrk8bTJfy78mkMZame/bLEUQ2h4EvWT6trd5XYNfPwQW/u6laLJAtLFbzth +lpNBRK//sK+AYgwDURq0HcW5YczhgzZbfG9ab1J/7GUrFJh+DHL3bNL1YFX84JbI +coUbxaJFkmIvlQDLWDYfpQ/gNwUbuUPZjGZC2bWobZw9sRC+e51TIoAmgZwvTElC +v3vD2BwbIs7C4ylrIQU9Q1rY17MeWH9m8ZhEX1C4W4+N54V6jJ7czE9HQEnapeEu +5rmEoDqP191x+tOxp/Xg8j+wcAK/dVy1Q3xQQZCW84rv0TBHbdaAPamBLUzHeW3W +F7BpnVhn1NfYVOKx5W0NfA== +-----END CERTIFICATE----- diff --git a/sources/hv-collector-ssl/src/test/resources/ssl/server.crt b/sources/hv-collector-ssl/src/test/resources/ssl/server.crt new file mode 100644 index 00000000..2b06108b --- /dev/null +++ b/sources/hv-collector-ssl/src/test/resources/ssl/server.crt @@ -0,0 +1,19 @@ +-----BEGIN CERTIFICATE----- +MIIDEjCCAfoCCQCId29lGbm5LjANBgkqhkiG9w0BAQsFADBLMQswCQYDVQQGEwJQ +TDELMAkGA1UECAwCREwxEDAOBgNVBAcMB1dyb2NsYXcxDjAMBgNVBAoMBU5va2lh +MQ0wCwYDVQQLDARNQU5PMB4XDTE4MDgxMzEyMDAyNFoXDTE5MDgwODEyMDAyNFow +SzELMAkGA1UEBhMCUEwxCzAJBgNVBAgMAkRMMRAwDgYDVQQHDAdXcm9jbGF3MQ4w +DAYDVQQKDAVOb2tpYTENMAsGA1UECwwETUFOTzCCASIwDQYJKoZIhvcNAQEBBQAD +ggEPADCCAQoCggEBALbblOyqeJayotBiY6aZnGWnMHCHNTileJF8hkFS5P5qM9BU +XH7NW/p5wSNcQWBB81HrRVneHpm6zfZtUPtcqXC4vpTk/sy3WT8rlEU+uv1YYVP8 +r0jxXr1WZ8sGcxOjhiMMRpMqImoALRak3ombtktfMc2Yeab4J25941wpDSQoUjT8 +LlXhfyjbBijum0LY+cqsuV5qk2qrzo0ZLD6m51aaWEyeysQ/3JanYKvPZDKdvAYN ++98ud6d/rjdWIJXwxEGp1fpW0p+fHOUhjMB1a5gkPDIpU68ME6BZJ3xJZb1qqXLS +pkUitMIWZSWx5xNhcifsnxWyhZfnhd8GnxU3Lf8CAwEAATANBgkqhkiG9w0BAQsF +AAOCAQEAJCrPdc8R7kBOmeIPyVX8QBOsrVTBBDzhskpi51nMsAKvcGjtYdXknYPj +bw3YVASE2efrq5QWHC0maIxDAHzI/kmWGH8F0s8S6QRH/7fstvxaNaSArzQB/thW +qIWs7AufUSsLMwtcXQ6KdYNbnaMwTynuuK+ANilcLvV8GxRAzXgFdoWZ2OuJRyzu +BWQZS49WanyqX1kDLMsrLtdJyl/yVEfsmDcUyhapuuGwtconkWK/CVrJlV1vh1Z+ +Svh9IfvAF7WPPpGsM/y9+hGNtK6sPVUKl4acBzIAv6aN1QS5H4zRvtbwv7xpUp4M +P84PY3lv1X3NJCGrfVByh2lFWParKA== +-----END CERTIFICATE----- diff --git a/sources/hv-collector-ssl/src/test/resources/ssl/server.key b/sources/hv-collector-ssl/src/test/resources/ssl/server.key new file mode 100644 index 00000000..40e25932 --- /dev/null +++ b/sources/hv-collector-ssl/src/test/resources/ssl/server.key @@ -0,0 +1,28 @@ +-----BEGIN PRIVATE KEY----- +MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQC225TsqniWsqLQ +YmOmmZxlpzBwhzU4pXiRfIZBUuT+ajPQVFx+zVv6ecEjXEFgQfNR60VZ3h6Zus32 +bVD7XKlwuL6U5P7Mt1k/K5RFPrr9WGFT/K9I8V69VmfLBnMTo4YjDEaTKiJqAC0W +pN6Jm7ZLXzHNmHmm+CdufeNcKQ0kKFI0/C5V4X8o2wYo7ptC2PnKrLleapNqq86N +GSw+pudWmlhMnsrEP9yWp2Crz2QynbwGDfvfLnenf643ViCV8MRBqdX6VtKfnxzl +IYzAdWuYJDwyKVOvDBOgWSd8SWW9aqly0qZFIrTCFmUlsecTYXIn7J8VsoWX54Xf +Bp8VNy3/AgMBAAECggEATzN4o7GKnast/hg/lU9/gEAUKQlHMgvp1woalHy1FsUl +QBzqGzoTlr/Zudkhr/Gg1GCVH0Gn+2n//7aFlvohoeNDGPa+rijUDRpxFDUBhO4c +6eXOfkedg2DDgBqBCYaQeOm+P8vGMCd3YBF1GiFJqgfHaIecWYeufJsmOSrGuFvK +1OvHpvg4/FLLQqgVcVO812kD4UwSOKnZVnPuZ3pzQviUZvO8ZxI/LkzQB1EdH6u3 +rBtiGslYkiKl5cGpH39/Dx2nKhHfvSnkfsm7koB00Bl41yy61GPwdl4XUwg8LUhX +TbsuoIPGTJX/2FUMn0UnAdDJm29hE4eyHyYOhew8gQKBgQDlAeUcnFr9uxe0i7cg +6ctJlNIKJjlA1tH4qIMEytdn06STo9g2j8X5HVE0FX/3+gAYDtEVICTF66w8Y474 +aeazvf+TCfkxtEOiH2afvaNkIkfzKR0ceB48DECT0DCF7xha5rJVf/W4GpCz2WkZ +ojDzw5ZVvzbx/FaF9A/IseJ63wKBgQDMaSjiephhdlCERGPdwWMg3AfthEX/VHM0 +YugbVjjYjDbn2pMkntW2hUuVXP8HD+9DnQZo0/c/hxe28Q5b+2fjZephdctnY8tL +XWbaEerM2lxEjmrpA4jYTBZJ9nMsxkEYHGHb5I586aS2YaZ12e7DoKMFdl0EZzvi +zGPIxSzQ4QKBgAxVv8t8uIH2M96rr997+FEsTOvzBx5w87pbCUOW0WdsRO8W4ix+ +LgGvDJKrncrzklG5apWit5hZi1ttWWQUADMqRrvay6tbtFDlNBfilQxttEZqroC8 +D5TYbBoKGrL8H+m1h2GHlOqns6ecTEbvL4fRvyU7OXBrURXCAZ+jxTktAoGACbQI +O9AEAcRjCBRTBUjT0tB/E9hOllNE8LytNfb+1dC6HoFysK9Vh8eGEf4LISOxgO0o +S7ucJgjcqFODEfy6LsI8wQmdcTf8g4RYiIuHMNhAvwRfsNX5HgNmn3Yye3Khzmoy +fwS3etiAeCPkif2hZunuMykuOzJHVnnLVtF9UiECgYEA41d7FgUcnfPIyA5xLg7K +lRgjFMsc68uzoCBQww2kio0HNJpdOPBJlg6oHHfYKriv2r9793jETRVwjSNPlKZb +vqm9yhnbXuahYBZSgdo2W+NbhP6IbJ0vrF4t9g6byjancQptaCjNIr9St9g+Ugly +8m0n3gIT/+Lr+it63cgk8SA= +-----END PRIVATE KEY----- diff --git a/sources/hv-collector-ssl/src/test/resources/ssl/server.ks.pkcs12 b/sources/hv-collector-ssl/src/test/resources/ssl/server.ks.pkcs12 Binary files differnew file mode 100644 index 00000000..a97eb65a --- /dev/null +++ b/sources/hv-collector-ssl/src/test/resources/ssl/server.ks.pkcs12 diff --git a/sources/hv-collector-ssl/src/test/resources/ssl/trust.pkcs12 b/sources/hv-collector-ssl/src/test/resources/ssl/trust.pkcs12 Binary files differnew file mode 100644 index 00000000..01b61373 --- /dev/null +++ b/sources/hv-collector-ssl/src/test/resources/ssl/trust.pkcs12 diff --git a/sources/hv-collector-test-utils/pom.xml b/sources/hv-collector-test-utils/pom.xml new file mode 100644 index 00000000..aebdd654 --- /dev/null +++ b/sources/hv-collector-test-utils/pom.xml @@ -0,0 +1,85 @@ +<?xml version="1.0" encoding="UTF-8"?> +<project xmlns="http://maven.apache.org/POM/4.0.0" + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + + <licenses> + <license> + <name>The Apache Software License, Version 2.0</name> + <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url> + </license> + </licenses> + + <parent> + <groupId>org.onap.dcaegen2.collectors.hv-ves</groupId> + <artifactId>hv-collector-sources</artifactId> + <version>1.1.0-SNAPSHOT</version> + <relativePath>..</relativePath> + </parent> + + <artifactId>hv-collector-test-utils</artifactId> + <description>VES HighVolume Collector :: Test Utilities</description> + + <properties> + <failIfMissingUnitTests>false</failIfMissingUnitTests> + <failIfMissingComponentTests>false</failIfMissingComponentTests> + </properties> + + <build> + <plugins> + <plugin> + <artifactId>kotlin-maven-plugin</artifactId> + <groupId>org.jetbrains.kotlin</groupId> + </plugin> + <plugin> + <artifactId>maven-surefire-plugin</artifactId> + <groupId>org.apache.maven.plugins</groupId> + </plugin> + </plugins> + </build> + + <dependencies> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-domain</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-utils</artifactId> + <version>${project.parent.version}</version> + <scope>compile</scope> + </dependency> + <dependency> + <groupId>org.assertj</groupId> + <artifactId>assertj-core</artifactId> + <scope>compile</scope> + </dependency> + <dependency> + <groupId>org.jetbrains.kotlin</groupId> + <artifactId>kotlin-test</artifactId> + <scope>compile</scope> + </dependency> + <dependency> + <groupId>org.jetbrains.spek</groupId> + <artifactId>spek-api</artifactId> + <scope>compile</scope> + </dependency> + <dependency> + <groupId>org.jetbrains.spek</groupId> + <artifactId>spek-junit-platform-engine</artifactId> + <scope>compile</scope> + </dependency> + <dependency> + <groupId>io.projectreactor</groupId> + <artifactId>reactor-test</artifactId> + <scope>compile</scope> + </dependency> + <dependency> + <groupId>com.nhaarman.mockitokotlin2</groupId> + <artifactId>mockito-kotlin</artifactId> + <scope>compile</scope> + </dependency> + </dependencies> +</project>
\ No newline at end of file diff --git a/sources/hv-collector-test-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/tests/utils/arrow.kt b/sources/hv-collector-test-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/tests/utils/arrow.kt new file mode 100644 index 00000000..54913744 --- /dev/null +++ b/sources/hv-collector-test-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/tests/utils/arrow.kt @@ -0,0 +1,62 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.tests.utils + +import arrow.core.Either +import arrow.core.identity +import org.assertj.core.api.AbstractAssert +import org.assertj.core.api.Assertions.assertThat +import org.assertj.core.api.ObjectAssert + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since September 2018 + */ +class EitherAssert<A, B>(actual: Either<A, B>) + : AbstractAssert<EitherAssert<A, B>, Either<A, B>>(actual, EitherAssert::class.java) { + + fun isLeft(): EitherAssert<A, B> { + isNotNull() + isInstanceOf(Either.Left::class.java) + return myself + } + + fun left(): ObjectAssert<A> { + isLeft() + val left = actual.fold( + ::identity, + { throw AssertionError("should be left") }) + return assertThat(left) + } + + fun isRight(): EitherAssert<A, B> { + isNotNull() + isInstanceOf(Either.Right::class.java) + return myself + } + + fun right(): ObjectAssert<B> { + isRight() + val right = actual.fold( + { throw AssertionError("should be right") }, + ::identity) + return assertThat(right) + } +} diff --git a/sources/hv-collector-test-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/tests/utils/assertions.kt b/sources/hv-collector-test-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/tests/utils/assertions.kt new file mode 100644 index 00000000..d017b31b --- /dev/null +++ b/sources/hv-collector-test-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/tests/utils/assertions.kt @@ -0,0 +1,55 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.tests.utils + +import arrow.core.Either +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import java.time.Duration + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since September 2018 + */ + +private val logger = Logger("org.onap.dcae.collectors.veshv.tests.utils") + +object Assertions : org.assertj.core.api.Assertions() { + fun <A,B> assertThat(actual: Either<A, B>) = EitherAssert(actual) +} + + +fun waitUntilSucceeds(action: () -> Unit) = waitUntilSucceeds(50, Duration.ofMillis(10), action) + +fun waitUntilSucceeds(retries: Int, sleepTime: Duration, action: () -> Unit) { + var tryNum = 0 + while (tryNum <= retries) { + tryNum++ + try { + logger.debug("Try number $tryNum") + action() + break + } catch (ex: Throwable) { + if (tryNum >= retries) + throw ex + else + Thread.sleep(sleepTime.toMillis()) + } + } +} diff --git a/sources/hv-collector-test-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/tests/utils/configurations.kt b/sources/hv-collector-test-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/tests/utils/configurations.kt new file mode 100644 index 00000000..57843b45 --- /dev/null +++ b/sources/hv-collector-test-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/tests/utils/configurations.kt @@ -0,0 +1,37 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.tests.utils + +import arrow.core.identity +import org.onap.dcae.collectors.veshv.utils.commandline.ArgBasedConfiguration +import org.onap.dcae.collectors.veshv.utils.commandline.WrongArgumentError + + +fun <T> ArgBasedConfiguration<T>.parseExpectingSuccess(vararg cmdLine: String): T = + parse(cmdLine).fold( + { throw AssertionError("Parsing result should be present") }, + ::identity + ) + +fun <T> ArgBasedConfiguration<T>.parseExpectingFailure(vararg cmdLine: String): WrongArgumentError = + parse(cmdLine).fold( + ::identity, + { throw AssertionError("parsing should have failed") } + )
\ No newline at end of file diff --git a/sources/hv-collector-test-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/tests/utils/messages.kt b/sources/hv-collector-test-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/tests/utils/messages.kt new file mode 100644 index 00000000..db7777c2 --- /dev/null +++ b/sources/hv-collector-test-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/tests/utils/messages.kt @@ -0,0 +1,85 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.tests.utils + +import com.google.protobuf.ByteString +import io.netty.buffer.ByteBuf +import io.netty.buffer.ByteBufAllocator +import io.netty.buffer.PooledByteBufAllocator +import org.onap.dcae.collectors.veshv.domain.WireFrameMessage.Companion.RESERVED_BYTE_COUNT +import org.onap.dcae.collectors.veshv.domain.VesEventDomain +import org.onap.dcae.collectors.veshv.domain.VesEventDomain.PERF3GPP +import org.onap.dcae.collectors.veshv.domain.VesEventDomain.OTHER + +import java.util.UUID.randomUUID + + +val allocator: ByteBufAllocator = PooledByteBufAllocator.DEFAULT + +private fun ByteBuf.writeValidWireFrameHeaders() { + writeByte(0xAA) // always 0xAA + writeByte(0x01) // major version + writeByte(0x00) // minor version + writeZero(RESERVED_BYTE_COUNT) // reserved + writeShort(0x0001) // content type = GPB +} + +fun vesWireFrameMessage(domain: VesEventDomain = OTHER, + id: String = randomUUID().toString()): ByteBuf = + allocator.buffer().run { + writeValidWireFrameHeaders() + + val gpb = vesEvent(domain, id).toByteString().asReadOnlyByteBuffer() + writeInt(gpb.limit()) // ves event size in bytes + writeBytes(gpb) // ves event as GPB bytes + } + +fun wireFrameMessageWithInvalidPayload(): ByteBuf = allocator.buffer().run { + writeValidWireFrameHeaders() + + val invalidGpb = "some random data".toByteArray(Charsets.UTF_8) + writeInt(invalidGpb.size) // ves event size in bytes + writeBytes(invalidGpb) +} + +fun garbageFrame(): ByteBuf = allocator.buffer().run { + writeBytes("the meaning of life is &@)(*_!".toByteArray()) +} + +fun invalidWireFrame(): ByteBuf = allocator.buffer().run { + writeByte(0xAA) + writeByte(0x01) // version major + writeByte(0x01) // version minor +} + +fun vesMessageWithPayloadOfSize(payloadSizeBytes: Int, domain: VesEventDomain = PERF3GPP): ByteBuf = + allocator.buffer().run { + writeValidWireFrameHeaders() + + val gpb = vesEvent( + domain = domain, + eventFields = ByteString.copyFrom(ByteArray(payloadSizeBytes)) + ).toByteString().asReadOnlyByteBuffer() + + writeInt(gpb.limit()) // ves event size in bytes + writeBytes(gpb) // ves event as GPB bytes + } + + diff --git a/sources/hv-collector-test-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/tests/utils/vesEvents.kt b/sources/hv-collector-test-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/tests/utils/vesEvents.kt new file mode 100644 index 00000000..569f1a90 --- /dev/null +++ b/sources/hv-collector-test-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/tests/utils/vesEvents.kt @@ -0,0 +1,73 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.tests.utils + + +import com.google.protobuf.ByteString +import com.google.protobuf.MessageLite +import org.onap.dcae.collectors.veshv.domain.ByteData +import org.onap.dcae.collectors.veshv.domain.VesEventDomain +import org.onap.dcae.collectors.veshv.domain.VesEventDomain.PERF3GPP +import org.onap.ves.VesEventOuterClass +import org.onap.ves.VesEventOuterClass.CommonEventHeader +import org.onap.ves.VesEventOuterClass.CommonEventHeader.Priority +import java.util.UUID.randomUUID + +fun vesEvent(domain: VesEventDomain = PERF3GPP, + id: String = randomUUID().toString(), + eventFields: ByteString = ByteString.EMPTY +): VesEventOuterClass.VesEvent = vesEvent(commonHeader(domain, id), eventFields) + +fun vesEvent(commonEventHeader: CommonEventHeader, + eventFields: ByteString = ByteString.EMPTY): VesEventOuterClass.VesEvent = + VesEventOuterClass.VesEvent.newBuilder() + .setCommonEventHeader(commonEventHeader) + .setEventFields(eventFields) + .build() + +fun commonHeader(domain: VesEventDomain = PERF3GPP, + id: String = randomUUID().toString(), + priority: Priority = Priority.NORMAL, + vesEventListenerVersion: String = "7.0.2"): CommonEventHeader = + CommonEventHeader.newBuilder() + .setVersion("sample-version") + .setDomain(domain.domainName) + .setSequence(1) + .setPriority(priority) + .setEventId(id) + .setEventName("sample-event-name") + .setEventType("sample-event-type") + .setStartEpochMicrosec(120034455) + .setLastEpochMicrosec(120034455) + .setNfNamingCode("sample-nf-naming-code") + .setNfcNamingCode("sample-nfc-naming-code") + .setNfVendorName("vendor-name") + .setReportingEntityId(ByteString.copyFromUtf8("sample-reporting-entity-id")) + .setReportingEntityName("sample-reporting-entity-name") + .setSourceId(ByteString.copyFromUtf8("sample-source-id")) + .setSourceName("sample-source-name") + .setTimeZoneOffset("+1") + .setVesEventListenerVersion(vesEventListenerVersion) + .build() + +fun vesEventBytes(commonHeader: CommonEventHeader, byteString: ByteString = ByteString.EMPTY): ByteData = + vesEvent(commonHeader, byteString).toByteData() + +fun MessageLite.toByteData(): ByteData = ByteData(toByteArray())
\ No newline at end of file diff --git a/sources/hv-collector-test-utils/src/main/resources/mockito-extensions/org.mockito.plugins.MockMaker b/sources/hv-collector-test-utils/src/main/resources/mockito-extensions/org.mockito.plugins.MockMaker new file mode 100644 index 00000000..ca6ee9ce --- /dev/null +++ b/sources/hv-collector-test-utils/src/main/resources/mockito-extensions/org.mockito.plugins.MockMaker @@ -0,0 +1 @@ +mock-maker-inline
\ No newline at end of file diff --git a/sources/hv-collector-utils/pom.xml b/sources/hv-collector-utils/pom.xml new file mode 100644 index 00000000..66879ff7 --- /dev/null +++ b/sources/hv-collector-utils/pom.xml @@ -0,0 +1,140 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ ============LICENSE_START======================================================= + ~ dcaegen2-collectors-veshv + ~ ================================================================================ + ~ Copyright (C) 2018 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========================================================= + --> +<project xmlns="http://maven.apache.org/POM/4.0.0" + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + + <licenses> + <license> + <name>The Apache Software License, Version 2.0</name> + <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url> + </license> + </licenses> + + <properties> + <skipAnalysis>false</skipAnalysis> + </properties> + + <parent> + <groupId>org.onap.dcaegen2.collectors.hv-ves</groupId> + <artifactId>hv-collector-sources</artifactId> + <version>1.1.0-SNAPSHOT</version> + <relativePath>..</relativePath> + </parent> + + <artifactId>hv-collector-utils</artifactId> + <description>VES HighVolume Collector :: Utilities</description> + + + <build> + <plugins> + <plugin> + <artifactId>kotlin-maven-plugin</artifactId> + <groupId>org.jetbrains.kotlin</groupId> + </plugin> + <plugin> + <artifactId>maven-surefire-plugin</artifactId> + <groupId>org.apache.maven.plugins</groupId> + </plugin> + </plugins> + </build> + + <dependencies> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-domain</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>commons-cli</groupId> + <artifactId>commons-cli</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.kotlin</groupId> + <artifactId>kotlin-reflect</artifactId> + </dependency> + <dependency> + <groupId>io.arrow-kt</groupId> + <artifactId>arrow-instances-data</artifactId> + </dependency> + <dependency> + <groupId>io.arrow-kt</groupId> + <artifactId>arrow-effects</artifactId> + </dependency> + <dependency> + <groupId>io.arrow-kt</groupId> + <artifactId>arrow-syntax</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.kotlinx</groupId> + <artifactId>kotlinx-coroutines-core</artifactId> + </dependency> + <dependency> + <groupId>io.ratpack</groupId> + <artifactId>ratpack-core</artifactId> + <optional>true</optional> + </dependency> + <dependency> + <groupId>javax.json</groupId> + <artifactId>javax.json-api</artifactId> + <optional>true</optional> + </dependency> + <dependency> + <groupId>org.slf4j</groupId> + <artifactId>slf4j-api</artifactId> + </dependency> + <dependency> + <groupId>org.assertj</groupId> + <artifactId>assertj-core</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.kotlin</groupId> + <artifactId>kotlin-test</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.spek</groupId> + <artifactId>spek-api</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.spek</groupId> + <artifactId>spek-junit-platform-engine</artifactId> + </dependency> + <dependency> + <groupId>io.projectreactor</groupId> + <artifactId>reactor-test</artifactId> + </dependency> + <dependency> + <groupId>com.nhaarman.mockitokotlin2</groupId> + <artifactId>mockito-kotlin</artifactId> + </dependency> + <dependency> + <groupId>ch.qos.logback</groupId> + <artifactId>logback-classic</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.glassfish</groupId> + <artifactId>javax.json</artifactId> + <scope>provided</scope> + </dependency> + </dependencies> +</project>
\ No newline at end of file diff --git a/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/arrow/core.kt b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/arrow/core.kt new file mode 100644 index 00000000..7381592d --- /dev/null +++ b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/arrow/core.kt @@ -0,0 +1,47 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.utils.arrow + +import arrow.core.Either +import arrow.core.Option +import arrow.core.identity +import arrow.syntax.collections.firstOption +import java.util.concurrent.atomic.AtomicReference + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since July 2018 + */ + +fun <A> Either<A, A>.flatten() = fold(::identity, ::identity) + +fun <B> Either<Throwable, B>.rightOrThrow() = fold({ throw it }, ::identity) + +fun <A, B> Either<A, B>.rightOrThrow(mapper: (A) -> Throwable) = fold({ throw mapper(it) }, ::identity) + +fun <A> AtomicReference<A>.getOption() = Option.fromNullable(get()) + +fun <A> Option.Companion.fromNullablesChain(firstValue: A?, vararg nextValues: () -> A?): Option<A> = + if (firstValue != null) + Option.just(firstValue) + else nextValues.asSequence() + .map { it() } + .filter { it != null } + .firstOption() diff --git a/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/arrow/effects.kt b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/arrow/effects.kt new file mode 100644 index 00000000..05d13094 --- /dev/null +++ b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/arrow/effects.kt @@ -0,0 +1,69 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.utils.arrow + +import arrow.core.Either +import arrow.core.Left +import arrow.core.Right +import arrow.effects.IO +import reactor.core.publisher.Flux +import reactor.core.publisher.Mono +import kotlin.system.exitProcess + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ + +sealed class ExitCode { + abstract val code: Int + + fun io() = IO { + exitProcess(code) + } +} + +object ExitSuccess : ExitCode() { + override val code = 0 +} + +data class ExitFailure(override val code: Int) : ExitCode() + +fun Either<IO<Unit>, IO<Unit>>.unsafeRunEitherSync(onError: (Throwable) -> ExitCode, onSuccess: () -> Unit) = + flatten().attempt().unsafeRunSync().fold({ onError(it).io().unsafeRunSync() }, { onSuccess() }) + + +fun IO<Any>.unit() = map { Unit } + +fun <T> Mono<T>.asIo() = IO.async<T> { callback -> + subscribe({ + callback(Right(it)) + }, { + callback(Left(it)) + }) +} + +fun <T> Flux<IO<T>>.evaluateIo(): Flux<T> = + flatMap { io -> + io.attempt().unsafeRunSync().fold( + { Flux.error<T>(it) }, + { Flux.just<T>(it) } + ) + } diff --git a/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/commandline/ArgBasedConfiguration.kt b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/commandline/ArgBasedConfiguration.kt new file mode 100644 index 00000000..b14f1be5 --- /dev/null +++ b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/commandline/ArgBasedConfiguration.kt @@ -0,0 +1,57 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.utils.commandline + +import arrow.core.Either +import arrow.core.Option +import arrow.core.Try +import arrow.core.flatMap +import org.apache.commons.cli.CommandLine +import org.apache.commons.cli.CommandLineParser +import org.apache.commons.cli.Options +import java.io.File +import java.nio.file.Path +import java.nio.file.Paths + +abstract class ArgBasedConfiguration<T>(private val parser: CommandLineParser) { + abstract val cmdLineOptionsList: List<CommandLineOption> + + fun parse(args: Array<out String>): Either<WrongArgumentError, T> { + val parseResult = Try { + val commandLineOptions = cmdLineOptionsList.map { it.option }.fold(Options(), Options::addOption) + parser.parse(commandLineOptions, args) + } + return parseResult + .toEither() + .mapLeft { ex -> WrongArgumentError(ex, cmdLineOptionsList) } + .map(this::getConfiguration) + .flatMap { + it.toEither { + WrongArgumentError( + message = "Unexpected error when parsing command line arguments", + cmdLineOptionsList = cmdLineOptionsList) + } + } + } + + protected abstract fun getConfiguration(cmdLine: CommandLine): Option<T> + + protected fun stringPathToPath(path: String): Path = Paths.get(File(path).toURI()) +} diff --git a/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/commandline/CommandLineOption.kt b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/commandline/CommandLineOption.kt new file mode 100644 index 00000000..9439bff5 --- /dev/null +++ b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/commandline/CommandLineOption.kt @@ -0,0 +1,142 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.utils.commandline + +import org.apache.commons.cli.Option + + +enum class CommandLineOption(val option: Option, val required: Boolean = false) { + HEALTH_CHECK_API_PORT(Option.builder("H") + .longOpt("health-check-api-port") + .hasArg() + .desc("Health check rest api listen port") + .build() + ), + LISTEN_PORT(Option.builder("p") + .longOpt("listen-port") + .hasArg() + .desc("Listen port") + .build(), + required = true + ), + CONSUL_CONFIG_URL(Option.builder("c") + .longOpt("config-url") + .hasArg() + .desc("URL of ves configuration on consul") + .build(), + required = true + ), + CONSUL_FIRST_REQUEST_DELAY(Option.builder("d") + .longOpt("first-request-delay") + .hasArg() + .desc("Delay of first request to consul in seconds") + .build() + ), + CONSUL_REQUEST_INTERVAL(Option.builder("I") + .longOpt("request-interval") + .hasArg() + .desc("Interval of consul configuration requests in seconds") + .build() + ), + VES_HV_PORT(Option.builder("v") + .longOpt("ves-port") + .hasArg() + .desc("VesHvCollector port") + .build(), + required = true + ), + VES_HV_HOST(Option.builder("h") + .longOpt("ves-host") + .hasArg() + .desc("VesHvCollector host") + .build(), + required = true + ), + KAFKA_SERVERS(Option.builder("s") + .longOpt("kafka-bootstrap-servers") + .hasArg() + .desc("Comma-separated Kafka bootstrap servers in <host>:<port> format") + .build(), + required = true + ), + KAFKA_TOPICS(Option.builder("f") + .longOpt("kafka-topics") + .hasArg() + .desc("Comma-separated Kafka topics") + .build(), + required = true + ), + SSL_DISABLE(Option.builder("l") + .longOpt("ssl-disable") + .desc("Disable SSL encryption") + .build() + ), + KEY_STORE_FILE(Option.builder("k") + .longOpt("key-store") + .hasArg() + .desc("Key store in PKCS12 format") + .build() + ), + KEY_STORE_PASSWORD(Option.builder("kp") + .longOpt("key-store-password") + .hasArg() + .desc("Key store password") + .build() + ), + TRUST_STORE_FILE(Option.builder("t") + .longOpt("trust-store") + .hasArg() + .desc("File with trusted certificate bundle in PKCS12 format") + .build() + ), + TRUST_STORE_PASSWORD(Option.builder("tp") + .longOpt("trust-store-password") + .hasArg() + .desc("Trust store password") + .build() + ), + IDLE_TIMEOUT_SEC(Option.builder("i") + .longOpt("idle-timeout-sec") + .hasArg() + .desc("""Idle timeout for remote hosts. After given time without any data exchange the + |connection might be closed.""".trimMargin()) + .build() + ), + MAXIMUM_PAYLOAD_SIZE_BYTES(Option.builder("m") + .longOpt("max-payload-size") + .hasArg() + .desc("Maximum supported payload size in bytes") + .build() + ), + DUMMY_MODE(Option.builder("u") + .longOpt("dummy") + .desc("If present will start in dummy mode (dummy external services)") + .build() + ); + + fun environmentVariableName(prefix: String = DEFAULT_ENV_PREFIX): String = + option.longOpt.toUpperCase().replace('-', '_').let { mainPart -> + "${prefix}_${mainPart}" + } + + companion object { + private const val DEFAULT_ENV_PREFIX = "VESHV" + } +} diff --git a/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/commandline/WrongArgumentError.kt b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/commandline/WrongArgumentError.kt new file mode 100644 index 00000000..9c2a20c1 --- /dev/null +++ b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/commandline/WrongArgumentError.kt @@ -0,0 +1,70 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.utils.commandline + +import arrow.core.Option +import org.apache.commons.cli.HelpFormatter +import org.apache.commons.cli.Options + + +data class WrongArgumentError( + val message: String, + val cause: Throwable? = null, + val cmdLineOptionsList: List<CommandLineOption>) { + + constructor(par: Throwable, cmdLineOptionsList: List<CommandLineOption>) : + this(par.message ?: "", + par, + cmdLineOptionsList) + + fun printMessage() { + println(message) + } + + fun printHelp(programName: String) { + val formatter = HelpFormatter() + val footer = "All parameters can be specified as environment variables using upper-snake-case full " + + "name with prefix `VESHV_`." + + formatter.printHelp( + programName, + generateRequiredParametersNote(cmdLineOptionsList), + getOptions(), + footer) + } + + private fun getOptions() = cmdLineOptionsList.map { it.option }.fold(Options(), Options::addOption) + + companion object { + fun generateRequiredParametersNote(cmdLineOptionsList: List<CommandLineOption>): String { + val requiredParams = Option.fromNullable(cmdLineOptionsList.filter { it.required } + .takeUnless { it.isEmpty() }) + return requiredParams.fold( + { "" }, + { + it.map { commandLineOption -> commandLineOption.option.opt } + .joinToString(prefix = "Required parameters: ", separator = ", ") + } + ) + } + } + +} + diff --git a/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/commandline/extensions.kt b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/commandline/extensions.kt new file mode 100644 index 00000000..a8414472 --- /dev/null +++ b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/commandline/extensions.kt @@ -0,0 +1,66 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.utils.commandline + +import arrow.core.Option +import arrow.core.getOrElse +import arrow.effects.IO +import arrow.syntax.function.curried +import org.apache.commons.cli.CommandLine +import org.onap.dcae.collectors.veshv.utils.arrow.ExitFailure +import org.onap.dcae.collectors.veshv.utils.arrow.fromNullablesChain + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ + +fun handleWrongArgumentError(programName: String, err: WrongArgumentError): IO<Unit> = IO { + err.printMessage() + err.printHelp(programName) +}.flatMap { ExitFailure(2).io() } + +val handleWrongArgumentErrorCurried = ::handleWrongArgumentError.curried() + +fun CommandLine.longValue(cmdLineOpt: CommandLineOption, default: Long): Long = + longValue(cmdLineOpt).getOrElse { default } + +fun CommandLine.stringValue(cmdLineOpt: CommandLineOption, default: String): String = + optionValue(cmdLineOpt).getOrElse { default } + +fun CommandLine.intValue(cmdLineOpt: CommandLineOption, default: Int): Int = + intValue(cmdLineOpt).getOrElse { default } + +fun CommandLine.intValue(cmdLineOpt: CommandLineOption): Option<Int> = + optionValue(cmdLineOpt).map(String::toInt) + +fun CommandLine.longValue(cmdLineOpt: CommandLineOption): Option<Long> = + optionValue(cmdLineOpt).map(String::toLong) + +fun CommandLine.stringValue(cmdLineOpt: CommandLineOption): Option<String> = + optionValue(cmdLineOpt) + +fun CommandLine.hasOption(cmdLineOpt: CommandLineOption): Boolean = + this.hasOption(cmdLineOpt.option.opt) || + System.getenv(cmdLineOpt.environmentVariableName()) != null + +private fun CommandLine.optionValue(cmdLineOpt: CommandLineOption) = Option.fromNullablesChain( + getOptionValue(cmdLineOpt.option.opt), + { System.getenv(cmdLineOpt.environmentVariableName()) }) diff --git a/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/http/http.kt b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/http/http.kt new file mode 100644 index 00000000..c5c46397 --- /dev/null +++ b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/http/http.kt @@ -0,0 +1,81 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.utils.http + +import arrow.typeclasses.Show +import java.util.* +import javax.json.Json + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since August 2018 + */ +object HttpConstants { + const val STATUS_OK = 200 + const val STATUS_ACCEPTED = 202 + const val STATUS_BAD_REQUEST = 400 + const val STATUS_NOT_FOUND = 404 + const val STATUS_INTERNAL_SERVER_ERROR = 500 + const val STATUS_SERVICE_UNAVAILABLE = 503 + + const val CONTENT_TYPE_JSON = "application/json" + const val CONTENT_TYPE_TEXT = "text/plain" +} + +enum class HttpStatus(val number: Int) { + OK(HttpConstants.STATUS_OK), + ACCEPTED(HttpConstants.STATUS_ACCEPTED), + BAD_REQUEST(HttpConstants.STATUS_BAD_REQUEST), + NOT_FOUND(HttpConstants.STATUS_NOT_FOUND), + INTERNAL_SERVER_ERROR(HttpConstants.STATUS_INTERNAL_SERVER_ERROR), + SERVICE_UNAVAILABLE(HttpConstants.STATUS_SERVICE_UNAVAILABLE) +} + + +enum class ContentType(val value: String) { + JSON(HttpConstants.CONTENT_TYPE_JSON), + TEXT(HttpConstants.CONTENT_TYPE_TEXT) +} + +data class Response(val status: HttpStatus, val content: Content<Any>) +data class Content<T>(val type: ContentType, val value: T, val serializer: Show<T> = Show.any()) + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since September 2018 + */ +object Responses { + + fun acceptedResponse(id: UUID): Response { + return Response( + HttpStatus.ACCEPTED, + Content(ContentType.TEXT, id) + ) + } + + fun statusResponse(name: String, message: String, httpStatus: HttpStatus = HttpStatus.OK): Response { + return Response(httpStatus, + Content(ContentType.JSON, + Json.createObjectBuilder() + .add("status", name) + .add("message", message) + .build())) + } +} diff --git a/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/http/ratpack.kt b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/http/ratpack.kt new file mode 100644 index 00000000..0282d0c7 --- /dev/null +++ b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/http/ratpack.kt @@ -0,0 +1,77 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.utils.http + +import arrow.core.Either +import arrow.effects.IO +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import javax.json.Json + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since August 2018 + */ + +private val logger = Logger("org.onap.dcae.collectors.veshv.utils.arrow.ratpack") + +fun ratpack.http.Response.sendOrError(action: IO<Unit>) { + sendAndHandleErrors(action.map { + Response( + HttpStatus.OK, + Content( + ContentType.JSON, + Json.createObjectBuilder().add("response", "Request accepted").build())) + }) +} + +fun <A> ratpack.http.Response.sendEitherErrorOrResponse(response: Either<A, Response>) { + when(response) { + is Either.Left -> send(errorResponse(response.a.toString())) + is Either.Right -> sendAndHandleErrors(IO.just(response.b)) + } +} + +fun ratpack.http.Response.sendAndHandleErrors(response: IO<Response>) { + response.attempt().unsafeRunSync().fold( + { err -> + logger.warn("Error occurred. Sending .", err) + val message = err.message + send(errorResponse(message)) + }, + ::send + ) +} + +private fun errorResponse(message: String?): Response { + return Response( + HttpStatus.INTERNAL_SERVER_ERROR, + Content( + ContentType.JSON, + Json.createObjectBuilder().add("error", message).build())) +} + +fun ratpack.http.Response.send(response: Response) { + val respWithStatus = status(response.status.number) + response.content.apply { + respWithStatus.send( + type.value, + serializer.run { value.show() }) + } +} diff --git a/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/logging/Logger.kt b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/logging/Logger.kt new file mode 100644 index 00000000..033dd5e5 --- /dev/null +++ b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/logging/Logger.kt @@ -0,0 +1,137 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.utils.logging + +import kotlin.reflect.KClass +import org.slf4j.LoggerFactory + +@Suppress("TooManyFunctions", "SuboptimalLoggerUsage") +class Logger(val logger: org.slf4j.Logger) { + constructor(clazz: KClass<out Any>) : this(LoggerFactory.getLogger(clazz.java)) + constructor(name: String) : this(LoggerFactory.getLogger(name)) + + // + // TRACE + // + + val traceEnabled: Boolean + get() = logger.isTraceEnabled + + fun trace(messageProvider: () -> String) { + if (logger.isTraceEnabled) { + logger.trace(messageProvider()) + } + } + + // + // DEBUG + // + + fun debug(message: String) { + logger.debug(message) + } + + fun debug(message: String, t: Throwable) { + logger.debug(message, t) + } + + fun debug(messageProvider: () -> String) { + if (logger.isDebugEnabled) { + logger.debug(messageProvider()) + } + } + + fun debug(t: Throwable, messageProvider: () -> String) { + if (logger.isDebugEnabled) { + logger.debug(messageProvider(), t) + } + } + + // + // INFO + // + fun info(message: String) { + logger.info(message) + } + + fun info(messageProvider: () -> String) { + if (logger.isInfoEnabled) { + logger.info(messageProvider()) + } + } + + fun info(message: String, t: Throwable) { + logger.info(message, t) + } + + fun info(t: Throwable, messageProvider: () -> String) { + if (logger.isInfoEnabled) { + logger.info(messageProvider(), t) + } + } + + // + // WARN + // + + fun warn(message: String) { + logger.warn(message) + } + + fun warn(message: String, t: Throwable) { + logger.warn(message, t) + } + + fun warn(messageProvider: () -> String) { + if (logger.isWarnEnabled) { + logger.warn(messageProvider()) + } + } + + fun warn(t: Throwable, messageProvider: () -> String) { + if (logger.isWarnEnabled) { + logger.warn(messageProvider(), t) + } + } + + // + // ERROR + // + + fun error(message: String) { + logger.error(message) + } + + fun error(message: String, t: Throwable) { + logger.error(message, t) + } + + fun error(messageProvider: () -> String) { + if (logger.isErrorEnabled) { + logger.error(messageProvider()) + } + } + + fun error(t: Throwable, messageProvider: () -> String) { + if (logger.isErrorEnabled) { + logger.error(messageProvider(), t) + } + } +} diff --git a/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/logging/reactive_logging.kt b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/logging/reactive_logging.kt new file mode 100644 index 00000000..714702d3 --- /dev/null +++ b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/logging/reactive_logging.kt @@ -0,0 +1,28 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.utils.logging + +import reactor.core.publisher.Flux + +fun <T> Logger.handleReactiveStreamError(ex: Throwable, returnFlux: Flux<T> = Flux.empty()): Flux<T> { + logger.warn("Error while handling message stream: ${ex::class.qualifiedName} (${ex.message})") + logger.debug("Detailed stack trace", ex) + return returnFlux +} diff --git a/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/server_handle.kt b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/server_handle.kt new file mode 100644 index 00000000..bdb63b68 --- /dev/null +++ b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/server_handle.kt @@ -0,0 +1,46 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.utils + +import arrow.effects.IO +import reactor.netty.DisposableServer + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since August 2018 + */ +abstract class ServerHandle(val host: String, val port: Int) { + abstract fun shutdown(): IO<Unit> + abstract fun await(): IO<Unit> +} + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since August 2018 + */ +class NettyServerHandle(private val ctx: DisposableServer) : ServerHandle(ctx.host(), ctx.port()) { + override fun shutdown() = IO { + ctx.disposeNow() + } + + override fun await() = IO<Unit> { + ctx.channel().closeFuture().sync() + } +} diff --git a/sources/hv-collector-utils/src/test/kotlin/org/onap/dcae/collectors/veshv/utils/arrow/CoreKtTest.kt b/sources/hv-collector-utils/src/test/kotlin/org/onap/dcae/collectors/veshv/utils/arrow/CoreKtTest.kt new file mode 100644 index 00000000..2eb11b27 --- /dev/null +++ b/sources/hv-collector-utils/src/test/kotlin/org/onap/dcae/collectors/veshv/utils/arrow/CoreKtTest.kt @@ -0,0 +1,141 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.utils.arrow + +import arrow.core.None +import arrow.core.Option +import arrow.core.Some +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.given +import org.jetbrains.spek.api.dsl.it +import org.jetbrains.spek.api.dsl.on +import java.util.concurrent.atomic.AtomicReference + + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk></piotr.jaszczyk>@nokia.com> + * @since August 2018 + */ +internal class CoreKtTest : Spek({ + describe("AtomicReference.getOption") { + given("empty atomic reference") { + val atomicReference = AtomicReference<String>() + + on("getOption") { + val result = atomicReference.getOption() + + it("should be None") { + assertThat(result).isEqualTo(None) + } + } + } + given("non-empty atomic reference") { + val initialValue = "reksio" + val atomicReference = AtomicReference(initialValue) + + on("getOption") { + val result = atomicReference.getOption() + + it("should be Some($initialValue)") { + assertThat(result).isEqualTo(Some(initialValue)) + } + } + } + } + + describe("Option.fromNullablesChain") { + given("one non-null element") { + val just = "some text" + on("calling factory") { + val result = Option.fromNullablesChain(just) + + it("should return Some($just)") { + assertThat(result).isEqualTo(Some(just)) + } + } + } + + given("one null element") { + val just: String? = null + on("calling factory") { + val result = Option.fromNullablesChain(just) + + it("should return None") { + assertThat(result).isEqualTo(None) + } + } + } + + given("first non-null element") { + val first = "some text" + val second: String? = null + var secondAskedForValue = false + on("calling factory") { + val result = Option.fromNullablesChain(first, { secondAskedForValue = true; second }) + + it("should return Some($first)") { + assertThat(result).isEqualTo(Some(first)) + } + + it("should have not called second provider (should be lazy)") { + assertThat(secondAskedForValue).isFalse() + } + } + } + + given("two non-null elements") { + val first = "some text" + val second = "another text" + on("calling factory") { + val result = Option.fromNullablesChain(first, { second }) + + it("should return Some($first)") { + assertThat(result).isEqualTo(Some(first)) + } + } + } + + given("two null elements") { + val first: String? = null + val second: String? = null + on("calling factory") { + val result = Option.fromNullablesChain(first, { second }) + + it("should return None") { + assertThat(result).isEqualTo(None) + } + } + } + + given("second non-null element") { + val first: String? = null + val second = "another text" + on("calling factory") { + val result = Option.fromNullablesChain(first, { second }) + + it("should return Some($second)") { + assertThat(result).isEqualTo(Some(second)) + } + } + } + } +}) diff --git a/sources/hv-collector-utils/src/test/kotlin/org/onap/dcae/collectors/veshv/utils/commandline/CommandLineOptionTest.kt b/sources/hv-collector-utils/src/test/kotlin/org/onap/dcae/collectors/veshv/utils/commandline/CommandLineOptionTest.kt new file mode 100644 index 00000000..f36df043 --- /dev/null +++ b/sources/hv-collector-utils/src/test/kotlin/org/onap/dcae/collectors/veshv/utils/commandline/CommandLineOptionTest.kt @@ -0,0 +1,62 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.utils.commandline + +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.given +import org.jetbrains.spek.api.dsl.it +import org.jetbrains.spek.api.dsl.on + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since September 2018 + */ +class CommandLineOptionTest : Spek({ + describe("command line options enum") { + describe("environment variables") { + given("sample option and prefix") { + val opt = CommandLineOption.KAFKA_SERVERS + val prefix = "CONFIG" + + on("calling environmentVariableName") { + val result = opt.environmentVariableName(prefix) + + it("should return prefixed upper snake cased long option name") { + assertThat(result).isEqualTo("CONFIG_KAFKA_BOOTSTRAP_SERVERS") + } + } + } + + given("sample option without prefix") { + val opt = CommandLineOption.DUMMY_MODE + + on("calling environmentVariableName") { + val result = opt.environmentVariableName() + + it("should return prefixed upper snake cased long option name") { + assertThat(result).isEqualTo("VESHV_DUMMY") + } + } + } + } + } +}) diff --git a/sources/hv-collector-utils/src/test/kotlin/org/onap/dcae/collectors/veshv/utils/commandline/WrongArgumentErrorTest.kt b/sources/hv-collector-utils/src/test/kotlin/org/onap/dcae/collectors/veshv/utils/commandline/WrongArgumentErrorTest.kt new file mode 100644 index 00000000..63d9eb82 --- /dev/null +++ b/sources/hv-collector-utils/src/test/kotlin/org/onap/dcae/collectors/veshv/utils/commandline/WrongArgumentErrorTest.kt @@ -0,0 +1,61 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.utils.commandline + +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.given +import org.jetbrains.spek.api.dsl.it +import org.jetbrains.spek.api.dsl.on +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.HEALTH_CHECK_API_PORT +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.LISTEN_PORT + +class WrongArgumentErrorTest : Spek ({ + + describe("help message logic test") { + + given("at least one required option") { + val filledOptionList = listOf( + HEALTH_CHECK_API_PORT, + LISTEN_PORT) + on("help message preparation") { + val requiredParameters = WrongArgumentError.generateRequiredParametersNote(filledOptionList) + + it("should print out required fields") { + assertThat(requiredParameters).isEqualTo("Required parameters: p") + } + } + } + + given("no required option") { + val filledOptionList = listOf(HEALTH_CHECK_API_PORT) + + on("help message preparation") { + val requiredParameters = WrongArgumentError.generateRequiredParametersNote(filledOptionList) + + it("should not print required fields") { + assertThat(requiredParameters).isEqualTo("") + } + } + } + } + +})
\ No newline at end of file diff --git a/sources/hv-collector-utils/src/test/kotlin/org/onap/dcae/collectors/veshv/utils/http/ResponsesTest.kt b/sources/hv-collector-utils/src/test/kotlin/org/onap/dcae/collectors/veshv/utils/http/ResponsesTest.kt new file mode 100644 index 00000000..f9f716a1 --- /dev/null +++ b/sources/hv-collector-utils/src/test/kotlin/org/onap/dcae/collectors/veshv/utils/http/ResponsesTest.kt @@ -0,0 +1,101 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.utils.http + +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.given +import org.jetbrains.spek.api.dsl.it +import org.jetbrains.spek.api.dsl.on +import java.util.* +import javax.json.JsonObject + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since September 2018 + */ +internal class ResponsesTest : Spek({ + describe("response factory") { + describe("accepted response") { + given("uuid") { + val uuid = UUID.randomUUID() + + on("calling acceptedResponse") { + val result = Responses.acceptedResponse(uuid) + + it ("should have ACCEPTED status") { + assertThat(result.status).isEqualTo(HttpStatus.ACCEPTED) + } + + it ("should have text body") { + assertThat(result.content.type).isEqualTo(ContentType.TEXT) + } + + it ("should contain UUID text in the body") { + val serialized = result.content.serializer.run { result.content.value.show() } + assertThat(serialized).isEqualTo(uuid.toString()) + } + } + } + } + describe("status response") { + given("all params are specified") { + val status = "ok" + val message = "good job" + val httpStatus = HttpStatus.OK + + on("calling statusResponse") { + val result = Responses.statusResponse(status, message, httpStatus) + val json = result.content.value as JsonObject + + it ("should have OK status") { + assertThat(result.status).isEqualTo(HttpStatus.OK) + } + + it ("should have json body") { + assertThat(result.content.type).isEqualTo(ContentType.JSON) + } + + it ("should contain status as string") { + assertThat(json.getString("status")).isEqualTo(status) + } + + it ("should contain message") { + assertThat(json.getString("message")).isEqualTo(message) + } + } + } + + given("default params are omitted") { + val status = "ok" + val message = "good job" + + on("calling statusResponse") { + val result = Responses.statusResponse(status, message) + + it ("should have OK status") { + assertThat(result.status).isEqualTo(HttpStatus.OK) + } + } + } + } + } +}) diff --git a/sources/hv-collector-utils/src/test/kotlin/org/onap/dcae/collectors/veshv/utils/logging/LoggerTest.kt b/sources/hv-collector-utils/src/test/kotlin/org/onap/dcae/collectors/veshv/utils/logging/LoggerTest.kt new file mode 100644 index 00000000..c27fb8c8 --- /dev/null +++ b/sources/hv-collector-utils/src/test/kotlin/org/onap/dcae/collectors/veshv/utils/logging/LoggerTest.kt @@ -0,0 +1,230 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.utils.logging + +import com.nhaarman.mockitokotlin2.mock +import com.nhaarman.mockitokotlin2.verify +import com.nhaarman.mockitokotlin2.verifyNoMoreInteractions +import com.nhaarman.mockitokotlin2.whenever +import org.jetbrains.spek.api.Spek +import org.jetbrains.spek.api.dsl.describe +import org.jetbrains.spek.api.dsl.it + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since May 2018 + */ +object LoggerTest : Spek({ + + lateinit var slf4jLogger: org.slf4j.Logger + lateinit var cut: Logger + + beforeEachTest { + slf4jLogger = mock() + cut = Logger(slf4jLogger) + } + + afterEachTest { + verifyNoMoreInteractions(slf4jLogger) + } + + describe("Thin Kotlin logging facade for Slf4j") { + val message = "sample message" + val exception = Exception("fail") + + describe("debug levels") { + it("should log message") { + cut.debug(message) + verify(slf4jLogger).debug(message) + } + + it("should log message with exception") { + cut.debug(message, exception) + verify(slf4jLogger).debug(message, exception) + } + + describe("lazy logging message") { + + it("should log when debug is ON") { + whenever(slf4jLogger.isDebugEnabled).thenReturn(true) + cut.debug { message } + verify(slf4jLogger).isDebugEnabled + verify(slf4jLogger).debug(message) + } + + it("should not log when debug is OFF") { + whenever(slf4jLogger.isDebugEnabled).thenReturn(false) + cut.debug { message } + verify(slf4jLogger).isDebugEnabled + } + } + + describe("lazy logging message with exception") { + + it("should log when debug is ON") { + whenever(slf4jLogger.isDebugEnabled).thenReturn(true) + cut.debug(exception) { message } + verify(slf4jLogger).isDebugEnabled + verify(slf4jLogger).debug(message, exception) + } + + it("should not log when debug is OFF") { + whenever(slf4jLogger.isDebugEnabled).thenReturn(false) + cut.debug(exception) { message } + verify(slf4jLogger).isDebugEnabled + } + } + } + + describe("info levels") { + it("should log message") { + cut.info(message) + verify(slf4jLogger).info(message) + } + + it("should log message with exception") { + cut.info(message, exception) + verify(slf4jLogger).info(message, exception) + } + + describe("lazy logging message") { + + it("should log when debug is ON") { + whenever(slf4jLogger.isInfoEnabled).thenReturn(true) + cut.info { message } + verify(slf4jLogger).isInfoEnabled + verify(slf4jLogger).info(message) + } + + it("should not log when debug is OFF") { + whenever(slf4jLogger.isInfoEnabled).thenReturn(false) + cut.info { message } + verify(slf4jLogger).isInfoEnabled + } + } + + describe("lazy logging message with exception") { + + it("should log when debug is ON") { + whenever(slf4jLogger.isInfoEnabled).thenReturn(true) + cut.info(exception) { message } + verify(slf4jLogger).isInfoEnabled + verify(slf4jLogger).info(message, exception) + } + + it("should not log when debug is OFF") { + whenever(slf4jLogger.isInfoEnabled).thenReturn(false) + cut.info(exception) { message } + verify(slf4jLogger).isInfoEnabled + } + } + } + + describe("warning levels") { + it("should log message") { + cut.warn(message) + verify(slf4jLogger).warn(message) + } + + it("should log message with exception") { + cut.warn(message, exception) + verify(slf4jLogger).warn(message, exception) + } + + describe("lazy logging message") { + + it("should log when debug is ON") { + whenever(slf4jLogger.isWarnEnabled).thenReturn(true) + cut.warn { message } + verify(slf4jLogger).isWarnEnabled + verify(slf4jLogger).warn(message) + } + + it("should not log when debug is OFF") { + whenever(slf4jLogger.isWarnEnabled).thenReturn(false) + cut.warn { message } + verify(slf4jLogger).isWarnEnabled + } + } + + describe("lazy logging message with exception") { + + it("should log when debug is ON") { + whenever(slf4jLogger.isWarnEnabled).thenReturn(true) + cut.warn(exception) { message } + verify(slf4jLogger).isWarnEnabled + verify(slf4jLogger).warn(message, exception) + } + + it("should not log when debug is OFF") { + whenever(slf4jLogger.isWarnEnabled).thenReturn(false) + cut.warn(exception) { message } + verify(slf4jLogger).isWarnEnabled + } + } + } + + describe("error levels") { + it("should log message") { + cut.error(message) + verify(slf4jLogger).error(message) + } + + it("should log message with exception") { + cut.error(message, exception) + verify(slf4jLogger).error(message, exception) + } + + describe("lazy logging message") { + + it("should log when debug is ON") { + whenever(slf4jLogger.isErrorEnabled).thenReturn(true) + cut.error { message } + verify(slf4jLogger).isErrorEnabled + verify(slf4jLogger).error(message) + } + + it("should not log when debug is OFF") { + whenever(slf4jLogger.isErrorEnabled).thenReturn(false) + cut.error { message } + verify(slf4jLogger).isErrorEnabled + } + } + + describe("lazy logging message with exception") { + + it("should log when debug is ON") { + whenever(slf4jLogger.isErrorEnabled).thenReturn(true) + cut.error(exception) { message } + verify(slf4jLogger).isErrorEnabled + verify(slf4jLogger).error(message, exception) + } + + it("should not log when debug is OFF") { + whenever(slf4jLogger.isErrorEnabled).thenReturn(false) + cut.error(exception) { message } + verify(slf4jLogger).isErrorEnabled + } + } + } + + + } +}) diff --git a/sources/hv-collector-ves-message-generator/pom.xml b/sources/hv-collector-ves-message-generator/pom.xml new file mode 100644 index 00000000..c5ce7f65 --- /dev/null +++ b/sources/hv-collector-ves-message-generator/pom.xml @@ -0,0 +1,115 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ ============LICENSE_START======================================================= + ~ dcaegen2-collectors-veshv + ~ ================================================================================ + ~ Copyright (C) 2018 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========================================================= + --> +<project xmlns="http://maven.apache.org/POM/4.0.0" + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + + <licenses> + <license> + <name>The Apache Software License, Version 2.0</name> + <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url> + </license> + </licenses> + + <parent> + <groupId>org.onap.dcaegen2.collectors.hv-ves</groupId> + <artifactId>hv-collector-sources</artifactId> + <version>1.1.0-SNAPSHOT</version> + <relativePath>..</relativePath> + </parent> + + <artifactId>hv-collector-ves-message-generator</artifactId> + <description>VES HighVolume Collector :: VES message generator</description> + + <properties> + <skipAnalysis>false</skipAnalysis> + </properties> + + <build> + <plugins> + <plugin> + <artifactId>kotlin-maven-plugin</artifactId> + <groupId>org.jetbrains.kotlin</groupId> + </plugin> + <plugin> + <artifactId>maven-surefire-plugin</artifactId> + <groupId>org.apache.maven.plugins</groupId> + </plugin> + </plugins> + </build> + <dependencies> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-domain</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-test-utils</artifactId> + <version>${project.parent.version}</version> + <scope>test</scope> + </dependency> + <dependency> + <groupId>com.google.protobuf</groupId> + <artifactId>protobuf-java-util</artifactId> + </dependency> + <dependency> + <groupId>org.slf4j</groupId> + <artifactId>slf4j-api</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.kotlin</groupId> + <artifactId>kotlin-stdlib-jdk8</artifactId> + </dependency> + <dependency> + <groupId>org.assertj</groupId> + <artifactId>assertj-core</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.kotlin</groupId> + <artifactId>kotlin-test</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.spek</groupId> + <artifactId>spek-api</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.spek</groupId> + <artifactId>spek-junit-platform-engine</artifactId> + </dependency> + <dependency> + <groupId>io.projectreactor</groupId> + <artifactId>reactor-test</artifactId> + </dependency> + <dependency> + <groupId>ch.qos.logback</groupId> + <artifactId>logback-classic</artifactId> + <scope>runtime</scope> + </dependency> + <dependency> + <groupId>org.glassfish</groupId> + <artifactId>javax.json</artifactId> + </dependency> + </dependencies> + + +</project>
\ No newline at end of file diff --git a/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/api/MessageGenerator.kt b/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/api/MessageGenerator.kt new file mode 100644 index 00000000..076c06be --- /dev/null +++ b/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/api/MessageGenerator.kt @@ -0,0 +1,36 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ves.message.generator.api + +import org.onap.dcae.collectors.veshv.domain.WireFrameMessage +import reactor.core.publisher.Flux + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since June 2018 + */ +interface MessageGenerator { + fun createMessageFlux(messageParameters: List<MessageParameters>): Flux<WireFrameMessage> + + companion object { + const val FIXED_PAYLOAD_SIZE = 100 + } +} + diff --git a/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/api/MessageParameters.kt b/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/api/MessageParameters.kt new file mode 100644 index 00000000..047d863c --- /dev/null +++ b/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/api/MessageParameters.kt @@ -0,0 +1,30 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ves.message.generator.api + +import org.onap.ves.VesEventOuterClass.CommonEventHeader + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since June 2018 + */ +data class MessageParameters(val commonEventHeader: CommonEventHeader, + val messageType: MessageType, + val amount: Long = -1) diff --git a/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/api/MessageParametersParser.kt b/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/api/MessageParametersParser.kt new file mode 100644 index 00000000..754fa31f --- /dev/null +++ b/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/api/MessageParametersParser.kt @@ -0,0 +1,37 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ves.message.generator.api + +import arrow.core.Either +import arrow.core.Option +import org.onap.dcae.collectors.veshv.ves.message.generator.impl.MessageParametersParserImpl +import javax.json.JsonArray + +interface MessageParametersParser { + fun parse(request: JsonArray): Either<ParsingError, List<MessageParameters>> + + companion object { + val INSTANCE: MessageParametersParser by lazy { + MessageParametersParserImpl() + } + } +} + +data class ParsingError(val message: String, val cause: Option<Throwable>) diff --git a/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/api/MessageType.kt b/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/api/MessageType.kt new file mode 100644 index 00000000..22c88252 --- /dev/null +++ b/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/api/MessageType.kt @@ -0,0 +1,32 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ves.message.generator.api + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since July 2018 + */ +enum class MessageType { + VALID, + TOO_BIG_PAYLOAD, + FIXED_PAYLOAD, + INVALID_WIRE_FRAME, + INVALID_GPB_DATA, +} diff --git a/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/factory/MessageGeneratorFactory.kt b/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/factory/MessageGeneratorFactory.kt new file mode 100644 index 00000000..e2269c20 --- /dev/null +++ b/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/factory/MessageGeneratorFactory.kt @@ -0,0 +1,33 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ves.message.generator.factory + +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageGenerator +import org.onap.dcae.collectors.veshv.ves.message.generator.impl.MessageGeneratorImpl +import org.onap.dcae.collectors.veshv.ves.message.generator.impl.PayloadGenerator + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since October 2018 + */ +object MessageGeneratorFactory { + fun create(maxPayloadSizeBytes: Int): MessageGenerator = + MessageGeneratorImpl(PayloadGenerator(), maxPayloadSizeBytes) +} diff --git a/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/CommonEventHeaderParser.kt b/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/CommonEventHeaderParser.kt new file mode 100644 index 00000000..909db5e4 --- /dev/null +++ b/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/CommonEventHeaderParser.kt @@ -0,0 +1,50 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ves.message.generator.impl + +import arrow.core.Option +import com.google.protobuf.util.JsonFormat +import org.onap.dcae.collectors.veshv.domain.headerRequiredFieldDescriptors +import org.onap.ves.VesEventOuterClass.CommonEventHeader +import javax.json.JsonObject + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since July 2018 + */ +class CommonEventHeaderParser { + fun parse(json: JsonObject): Option<CommonEventHeader> = + Option.fromNullable( + CommonEventHeader.newBuilder() + .apply { JsonFormat.parser().merge(json.toString(), this) } + .build() + .takeUnless { !isValid(it) } + ) + + + private fun isValid(header: CommonEventHeader): Boolean { + return allMandatoryFieldsArePresent(header) + } + + private fun allMandatoryFieldsArePresent(header: CommonEventHeader) = + headerRequiredFieldDescriptors + .all { fieldDescriptor -> header.hasField(fieldDescriptor) } + +} diff --git a/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/MessageGeneratorImpl.kt b/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/MessageGeneratorImpl.kt new file mode 100644 index 00000000..fa39ed16 --- /dev/null +++ b/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/MessageGeneratorImpl.kt @@ -0,0 +1,109 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ves.message.generator.impl + +import com.google.protobuf.ByteString +import org.onap.dcae.collectors.veshv.domain.ByteData +import org.onap.dcae.collectors.veshv.domain.PayloadContentType +import org.onap.dcae.collectors.veshv.domain.WireFrameMessage +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageGenerator +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageParameters +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageType +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageType.FIXED_PAYLOAD +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageType.INVALID_GPB_DATA +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageType.INVALID_WIRE_FRAME +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageType.TOO_BIG_PAYLOAD +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageType.VALID +import org.onap.ves.VesEventOuterClass.CommonEventHeader +import org.onap.ves.VesEventOuterClass.VesEvent +import reactor.core.publisher.Flux +import reactor.core.publisher.Mono +import java.nio.charset.Charset + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since June 2018 + */ +class MessageGeneratorImpl internal constructor( + private val payloadGenerator: PayloadGenerator, + private val maxPayloadSizeBytes: Int +) : MessageGenerator { + + override fun createMessageFlux(messageParameters: List<MessageParameters>): Flux<WireFrameMessage> = Flux + .fromIterable(messageParameters) + .flatMap { createMessageFlux(it) } + + private fun createMessageFlux(parameters: MessageParameters): Flux<WireFrameMessage> = + Mono.fromCallable { createMessage(parameters.commonEventHeader, parameters.messageType) } + .let { + when { + parameters.amount < 0 -> + // repeat forever + it.repeat() + parameters.amount == 0L -> + // do not generate any message + Flux.empty() + else -> + // send original message and additional amount-1 messages + it.repeat(parameters.amount - 1) + } + } + + private fun createMessage(commonEventHeader: CommonEventHeader, messageType: MessageType): WireFrameMessage = + when (messageType) { + VALID -> + WireFrameMessage(vesEvent(commonEventHeader, payloadGenerator.generatePayload())) + TOO_BIG_PAYLOAD -> + WireFrameMessage(vesEvent(commonEventHeader, oversizedPayload())) + FIXED_PAYLOAD -> + WireFrameMessage(vesEvent(commonEventHeader, fixedPayload())) + INVALID_WIRE_FRAME -> { + val payload = ByteData(vesEvent(commonEventHeader, payloadGenerator.generatePayload())) + WireFrameMessage( + payload, + UNSUPPORTED_VERSION, + UNSUPPORTED_VERSION, + PayloadContentType.GOOGLE_PROTOCOL_BUFFER.hexValue, + payload.size()) + } + INVALID_GPB_DATA -> + WireFrameMessage("invalid vesEvent".toByteArray(Charset.defaultCharset())) + } + + private fun vesEvent(commonEventHeader: CommonEventHeader, eventFields: ByteString): ByteArray { + return createVesEvent(commonEventHeader, eventFields).toByteArray() + } + + private fun createVesEvent(commonEventHeader: CommonEventHeader, payload: ByteString): VesEvent = + VesEvent.newBuilder() + .setCommonEventHeader(commonEventHeader) + .setEventFields(payload) + .build() + + private fun oversizedPayload() = + payloadGenerator.generateRawPayload(maxPayloadSizeBytes + 1) + + private fun fixedPayload() = + payloadGenerator.generateRawPayload(MessageGenerator.FIXED_PAYLOAD_SIZE) + + companion object { + private const val UNSUPPORTED_VERSION: Short = 2 + } +} diff --git a/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/MessageParametersParserImpl.kt b/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/MessageParametersParserImpl.kt new file mode 100644 index 00000000..6ef6d53a --- /dev/null +++ b/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/MessageParametersParserImpl.kt @@ -0,0 +1,57 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ves.message.generator.impl + +import arrow.core.Option +import arrow.core.Try +import arrow.core.identity +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageParameters +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageParametersParser +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageType +import org.onap.dcae.collectors.veshv.ves.message.generator.api.ParsingError +import javax.json.JsonArray + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since July 2018 + */ +internal class MessageParametersParserImpl( + private val commonEventHeaderParser: CommonEventHeaderParser = CommonEventHeaderParser() +) : MessageParametersParser { + + override fun parse(request: JsonArray) = + Try { + request + .map { it.asJsonObject() } + .map { json -> + val commonEventHeader = commonEventHeaderParser + .parse(json.getJsonObject("commonEventHeader")) + .fold({ throw IllegalStateException("Invalid common header") }, ::identity) + val messageType = MessageType.valueOf(json.getString("messageType")) + val messagesAmount = json.getJsonNumber("messagesAmount")?.longValue() + ?: throw NullPointerException("\"messagesAmount\" could not be parsed.") + MessageParameters(commonEventHeader, messageType, messagesAmount) + } + }.toEither().mapLeft { ex -> + ParsingError( + ex.message ?: "Unable to parse message parameters", + Option.fromNullable(ex)) + } +} diff --git a/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/PayloadGenerator.kt b/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/PayloadGenerator.kt new file mode 100644 index 00000000..545e237c --- /dev/null +++ b/sources/hv-collector-ves-message-generator/src/main/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/PayloadGenerator.kt @@ -0,0 +1,40 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ves.message.generator.impl + +import com.google.protobuf.ByteString +import java.util.* +import kotlin.streams.asSequence + +internal class PayloadGenerator { + + private val randomGenerator = Random() + + fun generateRawPayload(size: Int): ByteString = + ByteString.copyFrom(ByteArray(size)) + + fun generatePayload(numOfCountMeasurements: Long = 2): ByteString = + ByteString.copyFrom( + randomGenerator.ints(numOfCountMeasurements, 0, 256) + .asSequence() + .toString() + .toByteArray() + ) +} diff --git a/sources/hv-collector-ves-message-generator/src/test/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/CommonEventHeaderParserTest.kt b/sources/hv-collector-ves-message-generator/src/test/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/CommonEventHeaderParserTest.kt new file mode 100644 index 00000000..3a33c44a --- /dev/null +++ b/sources/hv-collector-ves-message-generator/src/test/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/CommonEventHeaderParserTest.kt @@ -0,0 +1,82 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ves.message.generator.impl + +import arrow.core.Option +import arrow.core.identity +import com.google.protobuf.util.JsonFormat +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.given +import org.jetbrains.spek.api.dsl.it +import org.onap.dcae.collectors.veshv.domain.VesEventDomain.STATE_CHANGE +import org.onap.dcae.collectors.veshv.tests.utils.commonHeader +import org.onap.ves.VesEventOuterClass.CommonEventHeader +import java.io.ByteArrayInputStream +import javax.json.Json +import kotlin.test.fail + +class CommonEventHeaderParserTest : Spek({ + + describe("Common event header parser") { + val parser = CommonEventHeaderParser() + + given("valid header in JSON format") { + val commonEventHeader = commonHeader( + domain = STATE_CHANGE, + id = "sample-event-id") + val json = JsonFormat.printer().print(commonEventHeader).byteInputStream() + + it("should parse common event header") { + val result = + parser.parse(jsonObject(json)) + .fold({ fail() }, ::identity) + + assertThat(result).describedAs("common event header").isEqualTo(commonEventHeader) + } + } + + given("invalid header in JSON format") { + val json = "{}".byteInputStream() + + it("should throw exception") { + val result = parser.parse(jsonObject(json)) + + assertFailed(result) + } + } + + given("invalid JSON") { + val json = "{}}}}".byteInputStream() + + it("should throw exception") { + val result = parser.parse(jsonObject(json)) + + assertFailed(result) + } + } + } +}) + +fun assertFailed(result: Option<CommonEventHeader>) = + result.fold({}, { fail() }) + +fun jsonObject(json: ByteArrayInputStream) = Json.createReader(json).readObject()!!
\ No newline at end of file diff --git a/sources/hv-collector-ves-message-generator/src/test/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/MessageGeneratorImplTest.kt b/sources/hv-collector-ves-message-generator/src/test/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/MessageGeneratorImplTest.kt new file mode 100644 index 00000000..e2aec7df --- /dev/null +++ b/sources/hv-collector-ves-message-generator/src/test/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/MessageGeneratorImplTest.kt @@ -0,0 +1,227 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ves.message.generator.impl + +import com.google.protobuf.ByteString +import com.google.protobuf.InvalidProtocolBufferException +import org.assertj.core.api.Assertions.assertThat +import org.assertj.core.api.Assertions.assertThatExceptionOfType +import org.jetbrains.spek.api.Spek +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.onap.dcae.collectors.veshv.domain.ByteData +import org.onap.dcae.collectors.veshv.domain.WireFrameMessage +import org.onap.dcae.collectors.veshv.domain.VesEventDomain.PERF3GPP +import org.onap.dcae.collectors.veshv.domain.VesEventDomain.FAULT +import org.onap.dcae.collectors.veshv.domain.VesEventDomain.HEARTBEAT +import org.onap.dcae.collectors.veshv.tests.utils.commonHeader +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageGenerator +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageParameters +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageType +import org.onap.ves.VesEventOuterClass.CommonEventHeader +import org.onap.ves.VesEventOuterClass.VesEvent +import reactor.test.test + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since June 2018 + */ +object MessageGeneratorImplTest : Spek({ + describe("message factory") { + val maxPayloadSizeBytes = 1024 + val generator = MessageGeneratorImpl(PayloadGenerator(), maxPayloadSizeBytes) + given("single message parameters") { + + on("messages amount not specified in parameters") { + it("should create infinite flux") { + val limit = 1000L + generator + .createMessageFlux(listOf(MessageParameters( + commonHeader(PERF3GPP), + MessageType.VALID + ))) + .take(limit) + .test() + .expectNextCount(limit) + .verifyComplete() + } + } + + on("messages amount = 0 specified in parameters") { + it("should create empty message flux") { + generator + .createMessageFlux(listOf(MessageParameters( + commonHeader(PERF3GPP), + MessageType.VALID, + 0 + ))) + .test() + .verifyComplete() + } + } + + on("messages amount specified in parameters") { + it("should create message flux of specified size") { + generator + .createMessageFlux(listOf(MessageParameters( + commonHeader(PERF3GPP), + MessageType.VALID, + 5 + ))) + .test() + .expectNextCount(5) + .verifyComplete() + } + } + + on("message type requesting valid message") { + it("should create flux of valid messages with given domain") { + generator + .createMessageFlux(listOf(MessageParameters( + commonHeader(FAULT), + MessageType.VALID, + 1 + ))) + .test() + .assertNext { + assertThat(it.isValid()).isTrue() + assertThat(it.payloadSize).isLessThan(maxPayloadSizeBytes) + assertThat(extractCommonEventHeader(it.payload).domain).isEqualTo(FAULT.domainName) + } + .verifyComplete() + } + } + + on("message type requesting too big payload") { + it("should create flux of messages with given domain and payload exceeding threshold") { + + generator + .createMessageFlux(listOf(MessageParameters( + commonHeader(PERF3GPP), + MessageType.TOO_BIG_PAYLOAD, + 1 + ))) + .test() + .assertNext { + assertThat(it.isValid()).isTrue() + assertThat(it.payloadSize).isGreaterThan(maxPayloadSizeBytes) + assertThat(extractCommonEventHeader(it.payload).domain).isEqualTo(PERF3GPP.domainName) + } + .verifyComplete() + } + } + + on("message type requesting invalid GPB data ") { + it("should create flux of messages with invalid payload") { + generator + .createMessageFlux(listOf(MessageParameters( + commonHeader(PERF3GPP), + MessageType.INVALID_GPB_DATA, + 1 + ))) + .test() + .assertNext { + assertThat(it.isValid()).isTrue() + assertThat(it.payloadSize).isLessThan(maxPayloadSizeBytes) + assertThatExceptionOfType(InvalidProtocolBufferException::class.java) + .isThrownBy { extractCommonEventHeader(it.payload) } + } + .verifyComplete() + } + } + + on("message type requesting invalid wire frame ") { + it("should create flux of messages with invalid version") { + generator + .createMessageFlux(listOf(MessageParameters( + commonHeader(PERF3GPP), + MessageType.INVALID_WIRE_FRAME, + 1 + ))) + .test() + .assertNext { + assertThat(it.isValid()).isFalse() + assertThat(it.payloadSize).isLessThan(maxPayloadSizeBytes) + assertThat(extractCommonEventHeader(it.payload).domain).isEqualTo(PERF3GPP.domainName) + assertThat(it.versionMajor).isNotEqualTo(WireFrameMessage.SUPPORTED_VERSION_MINOR) + } + .verifyComplete() + } + } + + on("message type requesting fixed payload") { + it("should create flux of valid messages with fixed payload") { + generator + .createMessageFlux(listOf(MessageParameters( + commonHeader(FAULT), + MessageType.FIXED_PAYLOAD, + 1 + ))) + .test() + .assertNext { + assertThat(it.isValid()).isTrue() + assertThat(it.payloadSize).isLessThan(maxPayloadSizeBytes) + assertThat(extractEventFields(it.payload).size()).isEqualTo(MessageGenerator.FIXED_PAYLOAD_SIZE) + assertThat(extractCommonEventHeader(it.payload).domain).isEqualTo(FAULT.domainName) + } + .verifyComplete() + } + } + } + given("list of message parameters") { + it("should create concatenated flux of messages") { + val singleFluxSize = 5L + val messageParameters = listOf( + MessageParameters(commonHeader(PERF3GPP), MessageType.VALID, singleFluxSize), + MessageParameters(commonHeader(FAULT), MessageType.TOO_BIG_PAYLOAD, singleFluxSize), + MessageParameters(commonHeader(HEARTBEAT), MessageType.VALID, singleFluxSize) + ) + generator.createMessageFlux(messageParameters) + .test() + .assertNext { + assertThat(it.payloadSize).isLessThan(maxPayloadSizeBytes) + assertThat(extractCommonEventHeader(it.payload).domain).isEqualTo(PERF3GPP.domainName) + } + .expectNextCount(singleFluxSize - 1) + .assertNext { + assertThat(it.payloadSize).isGreaterThan(maxPayloadSizeBytes) + assertThat(extractCommonEventHeader(it.payload).domain).isEqualTo(FAULT.domainName) + } + .expectNextCount(singleFluxSize - 1) + .assertNext { + assertThat(it.payloadSize).isLessThan(maxPayloadSizeBytes) + assertThat(extractCommonEventHeader(it.payload).domain).isEqualTo(HEARTBEAT.domainName) + } + .expectNextCount(singleFluxSize - 1) + .verifyComplete() + } + } + } +}) + +fun extractCommonEventHeader(bytes: ByteData): CommonEventHeader = + VesEvent.parseFrom(bytes.unsafeAsArray()).commonEventHeader + + +fun extractEventFields(bytes: ByteData): ByteString = + VesEvent.parseFrom(bytes.unsafeAsArray()).eventFields + diff --git a/sources/hv-collector-ves-message-generator/src/test/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/MessageParametersParserTest.kt b/sources/hv-collector-ves-message-generator/src/test/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/MessageParametersParserTest.kt new file mode 100644 index 00000000..134ebb2d --- /dev/null +++ b/sources/hv-collector-ves-message-generator/src/test/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/MessageParametersParserTest.kt @@ -0,0 +1,64 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ves.message.generator.impl + +import org.assertj.core.api.Assertions.assertThat +import org.assertj.core.api.Assertions.fail +import org.jetbrains.spek.api.Spek +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.onap.dcae.collectors.veshv.ves.message.generator.api.MessageType + +private const val EXPECTED_MESSAGES_AMOUNT = 25000L + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since July 2018 + */ +object MessageParametersParserTest : Spek({ + describe("Messages parameters parser") { + val messageParametersParser = MessageParametersParserImpl() + + given("parameters json array") { + on("valid parameters json") { + it("should parse MessagesParameters object successfully") { + val result = messageParametersParser.parse(validMessagesParametesJson()) + + result.fold({ fail("should have succeeded") }) { rightResult -> + assertThat(rightResult).hasSize(2) + val firstMessage = rightResult.first() + assertThat(firstMessage.messageType).isEqualTo(MessageType.VALID) + assertThat(firstMessage.amount).isEqualTo(EXPECTED_MESSAGES_AMOUNT) + + } + } + } + + on("invalid parameters json") { + it("should throw exception") { + val result = messageParametersParser.parse(invalidMessagesParametesJson()) + assertThat(result.isLeft()).describedAs("is left").isTrue() + } + } + } + } +}) diff --git a/sources/hv-collector-ves-message-generator/src/test/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/PayloadGeneratorTest.kt b/sources/hv-collector-ves-message-generator/src/test/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/PayloadGeneratorTest.kt new file mode 100644 index 00000000..bb91245d --- /dev/null +++ b/sources/hv-collector-ves-message-generator/src/test/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/PayloadGeneratorTest.kt @@ -0,0 +1,51 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ves.message.generator.impl + +import org.assertj.core.api.Assertions.assertThat +import org.jetbrains.spek.api.Spek +import org.jetbrains.spek.api.dsl.given +import org.jetbrains.spek.api.dsl.it +import org.jetbrains.spek.api.dsl.on + +object PayloadGeneratorTest : Spek({ + + given("payload factory object") { + val payloadGenerator = PayloadGenerator() + + on("raw payload generation") { + val size = 100 + val generatedPayload = payloadGenerator.generateRawPayload(size) + + it("should generate sequence of zeros") { + assertThat(generatedPayload.size()).isEqualTo(size) + assertThat(generatedPayload.toByteArray()).isEqualTo(ByteArray(size)) + } + } + + on("two generated payloads") { + val generatedPayload0 = payloadGenerator.generatePayload() + val generatedPayload1 = payloadGenerator.generatePayload() + it("should be different") { + assertThat(generatedPayload0 != generatedPayload1).isTrue() + } + } + } +}) diff --git a/sources/hv-collector-ves-message-generator/src/test/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/parameters.kt b/sources/hv-collector-ves-message-generator/src/test/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/parameters.kt new file mode 100644 index 00000000..78cfa028 --- /dev/null +++ b/sources/hv-collector-ves-message-generator/src/test/kotlin/org/onap/dcae/collectors/veshv/ves/message/generator/impl/parameters.kt @@ -0,0 +1,106 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.ves.message.generator.impl + +import javax.json.Json + +private const val validMessageParameters = +"""[ + { + "commonEventHeader": { + "version": "sample-version", + "domain": "perf3gpp", + "sequence": 1, + "priority": 1, + "eventId": "sample-event-id", + "eventName": "sample-event-name", + "eventType": "sample-event-type", + "startEpochMicrosec": 120034455, + "lastEpochMicrosec": 120034455, + "nfNamingCode": "sample-nf-naming-code", + "nfcNamingCode": "sample-nfc-naming-code", + "reportingEntityId": "sample-reporting-entity-id", + "reportingEntityName": "sample-reporting-entity-name", + "sourceId": "sample-source-id", + "sourceName": "sample-source-name", + "vesEventListenerVersion": "another-version" + }, + "messageType": "VALID", + "messagesAmount": 25000 + }, + { + "commonEventHeader": { + "version": "sample-version", + "domain": "perf3gpp", + "sequence": 1, + "priority": 1, + "eventId": "sample-event-id", + "eventName": "sample-event-name", + "eventType": "sample-event-type", + "startEpochMicrosec": 120034455, + "lastEpochMicrosec": 120034455, + "nfNamingCode": "sample-nf-naming-code", + "nfcNamingCode": "sample-nfc-naming-code", + "reportingEntityId": "sample-reporting-entity-id", + "reportingEntityName": "sample-reporting-entity-name", + "sourceId": "sample-source-id", + "sourceName": "sample-source-name", + "vesEventListenerVersion": "another-version" + }, + "messageType": "TOO_BIG_PAYLOAD", + "messagesAmount": 100 + } + ] +""" + +private const val invalidMessageParameters = +""" + [ + { + "commonEventHeader": { + "version": "sample-version", + "domain": "perf3gpp", + "sequence": 1, + "priority": 1, + "eventId": "sample-event-id", + "eventName": "sample-event-name", + "eventType": "sample-event-type", + "startEpochMicrosec": 120034455, + "lastEpochMicrosec": 120034455, + "nfNamingCode": "sample-nf-naming-code", + "nfcNamingCode": "sample-nfc-naming-code", + "reportingEntityId": "sample-reporting-entity-id", + "reportingEntityName": "sample-reporting-entity-name", + "sourceId": "sample-source-id", + "sourceName": "sample-source-name", + "vesEventListenerVersion": "another-version" + }, + "messagesAmount": 3 + } + ] +""" + +fun validMessagesParametesJson() = Json + .createReader(validMessageParameters.reader()) + .readArray()!! + +fun invalidMessagesParametesJson() = Json + .createReader(invalidMessageParameters.reader()) + .readArray()!! diff --git a/sources/hv-collector-xnf-simulator/Dockerfile b/sources/hv-collector-xnf-simulator/Dockerfile new file mode 100644 index 00000000..53e126b2 --- /dev/null +++ b/sources/hv-collector-xnf-simulator/Dockerfile @@ -0,0 +1,18 @@ +FROM docker.io/openjdk:11-jre-slim + +LABEL copyright="Copyright (C) 2018 NOKIA" +LABEL license.name="The Apache Software License, Version 2.0" +LABEL license.url="http://www.apache.org/licenses/LICENSE-2.0" +LABEL maintainer="Nokia Wroclaw ONAP Team" + +RUN apt-get update \ + && apt-get install -y --no-install-recommends curl \ + && apt-get clean + +WORKDIR /opt/ves-hv-client-simulator + +ENTRYPOINT ["java", "-cp", "*:", "org.onap.dcae.collectors.veshv.simulators.xnf.MainKt"] + +COPY target/libs/external/* ./ +COPY target/libs/internal/* ./ +COPY target/hv-collector-xnf-simulator-*.jar ./ diff --git a/sources/hv-collector-xnf-simulator/pom.xml b/sources/hv-collector-xnf-simulator/pom.xml new file mode 100644 index 00000000..de262567 --- /dev/null +++ b/sources/hv-collector-xnf-simulator/pom.xml @@ -0,0 +1,157 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ ============LICENSE_START======================================================= + ~ dcaegen2-collectors-veshv + ~ ================================================================================ + ~ Copyright (C) 2018 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========================================================= + --> +<project xmlns="http://maven.apache.org/POM/4.0.0" + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + + <licenses> + <license> + <name>The Apache Software License, Version 2.0</name> + <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url> + </license> + </licenses> + + <parent> + <groupId>org.onap.dcaegen2.collectors.hv-ves</groupId> + <artifactId>hv-collector-sources</artifactId> + <version>1.1.0-SNAPSHOT</version> + <relativePath>..</relativePath> + </parent> + + <artifactId>hv-collector-xnf-simulator</artifactId> + <description>VES HighVolume Collector :: XNF simulator</description> + + <properties> + <skipAnalysis>false</skipAnalysis> + </properties> + + <build> + <plugins> + <plugin> + <artifactId>kotlin-maven-plugin</artifactId> + <groupId>org.jetbrains.kotlin</groupId> + </plugin> + <plugin> + <artifactId>maven-surefire-plugin</artifactId> + <groupId>org.apache.maven.plugins</groupId> + </plugin> + </plugins> + </build> + <profiles> + <profile> + <id>docker</id> + <activation> + <property> + <name>!skipDocker</name> + </property> + </activation> + <build> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-dependency-plugin</artifactId> + </plugin> + <plugin> + <groupId>io.fabric8</groupId> + <artifactId>docker-maven-plugin</artifactId> + </plugin> + </plugins> + </build> + </profile> + </profiles> + + <dependencies> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-domain</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-ssl</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-utils</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-ves-message-generator</artifactId> + <version>${project.parent.version}</version> + </dependency> + <dependency> + <groupId>${project.parent.groupId}</groupId> + <artifactId>hv-collector-test-utils</artifactId> + <version>${project.parent.version}</version> + <scope>test</scope> + </dependency> + <dependency> + <groupId>io.arrow-kt</groupId> + <artifactId>arrow-effects</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.kotlinx</groupId> + <artifactId>kotlinx-coroutines-core</artifactId> + </dependency> + <dependency> + <groupId>commons-cli</groupId> + <artifactId>commons-cli</artifactId> + </dependency> + <dependency> + <groupId>org.slf4j</groupId> + <artifactId>slf4j-api</artifactId> + </dependency> + <dependency> + <groupId>org.jetbrains.kotlin</groupId> + <artifactId>kotlin-stdlib-jdk8</artifactId> + </dependency> + <!-- See comment in main pom + <dependency> + <groupId>io.netty</groupId> + <artifactId>netty-transport-native-epoll</artifactId> + <classifier>${os.detected.classifier}</classifier> + </dependency> + <dependency> + <groupId>io.netty</groupId> + <artifactId>netty-tcnative-boringssl-static</artifactId> + <classifier>${os.detected.classifier}</classifier> + </dependency> + --> + <dependency> + <groupId>ch.qos.logback</groupId> + <artifactId>logback-classic</artifactId> + <scope>runtime</scope> + </dependency> + <dependency> + <groupId>org.glassfish</groupId> + <artifactId>javax.json</artifactId> + </dependency> + <dependency> + <groupId>io.ratpack</groupId> + <artifactId>ratpack-core</artifactId> + </dependency> + </dependencies> + + +</project>
\ No newline at end of file diff --git a/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/impl/XnfSimulator.kt b/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/impl/XnfSimulator.kt new file mode 100644 index 00000000..ee4734ae --- /dev/null +++ b/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/impl/XnfSimulator.kt @@ -0,0 +1,57 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.xnf.impl + +import arrow.core.Either +import arrow.core.Some +import arrow.core.Try +import arrow.core.fix +import arrow.effects.IO +import arrow.instances.either.monad.monad +import arrow.typeclasses.binding +import org.onap.dcae.collectors.veshv.simulators.xnf.impl.adapters.VesHvClient +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageGenerator +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageParametersParser +import org.onap.dcae.collectors.veshv.ves.message.generator.api.ParsingError +import java.io.InputStream +import javax.json.Json + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since August 2018 + */ +class XnfSimulator( + private val vesClient: VesHvClient, + private val messageGenerator: MessageGenerator, + private val messageParametersParser: MessageParametersParser = MessageParametersParser.INSTANCE) { + + fun startSimulation(messageParameters: InputStream): Either<ParsingError, IO<Unit>> = + Either.monad<ParsingError>().binding { + val json = parseJsonArray(messageParameters).bind() + val parsed = messageParametersParser.parse(json).bind() + val generatedMessages = messageGenerator.createMessageFlux(parsed) + vesClient.sendIo(generatedMessages) + }.fix() + + private fun parseJsonArray(jsonStream: InputStream) = + Try { + Json.createReader(jsonStream).readArray() + }.toEither().mapLeft { ParsingError("failed to parse JSON", Some(it)) } +} diff --git a/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/impl/adapters/VesHvClient.kt b/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/impl/adapters/VesHvClient.kt new file mode 100644 index 00000000..57aaf3db --- /dev/null +++ b/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/impl/adapters/VesHvClient.kt @@ -0,0 +1,109 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.xnf.impl.adapters + +import arrow.core.Option +import arrow.core.getOrElse +import io.netty.handler.ssl.SslContext +import org.onap.dcae.collectors.veshv.domain.WireFrameMessage +import org.onap.dcae.collectors.veshv.domain.SecurityConfiguration +import org.onap.dcae.collectors.veshv.domain.WireFrameEncoder +import org.onap.dcae.collectors.veshv.simulators.xnf.impl.config.SimulatorConfiguration +import org.onap.dcae.collectors.veshv.ssl.boundary.ClientSslContextFactory +import org.onap.dcae.collectors.veshv.utils.arrow.asIo +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import org.reactivestreams.Publisher +import reactor.core.publisher.Flux +import reactor.core.publisher.Mono +import reactor.core.publisher.ReplayProcessor +import reactor.netty.NettyOutbound +import reactor.netty.tcp.TcpClient + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since June 2018 + */ +class VesHvClient(private val configuration: SimulatorConfiguration) { + + private val client: TcpClient = TcpClient.create() + .host(configuration.vesHost) + .port(configuration.vesPort) + .configureSsl() + + private fun TcpClient.configureSsl() = + createSslContext(configuration.security) + .map { sslContext -> this.secure(sslContext) } + .getOrElse { this } + + fun sendIo(messages: Flux<WireFrameMessage>) = + sendRx(messages).then(Mono.just(Unit)).asIo() + + private fun sendRx(messages: Flux<WireFrameMessage>): Mono<Void> { + val complete = ReplayProcessor.create<Void>(1) + client + .handle { _, output -> handler(complete, messages, output) } + .connect() + .doOnError { + logger.info("Failed to connect to VesHvCollector on " + + "${configuration.vesHost}:${configuration.vesPort}") + } + .subscribe { + logger.info("Connected to VesHvCollector on " + + "${configuration.vesHost}:${configuration.vesPort}") + } + return complete.then() + } + + private fun handler(complete: ReplayProcessor<Void>, + messages: Flux<WireFrameMessage>, + nettyOutbound: NettyOutbound): Publisher<Void> { + + val allocator = nettyOutbound.alloc() + val encoder = WireFrameEncoder(allocator) + val frames = messages + .map(encoder::encode) + .window(MAX_BATCH_SIZE) + + return nettyOutbound + .logConnectionClosed() + .options { it.flushOnBoundary() } + .sendGroups(frames) + .then { + logger.info("Messages have been sent") + complete.onComplete() + } + .then() + } + + private fun createSslContext(config: SecurityConfiguration): Option<SslContext> = + ClientSslContextFactory().createSslContext(config) + + private fun NettyOutbound.logConnectionClosed() = + withConnection { conn -> + conn.onTerminate().subscribe { + logger.info { "Connection to ${conn.address()} has been closed" } + } + } + + companion object { + private val logger = Logger(VesHvClient::class) + private const val MAX_BATCH_SIZE = 128 + } +} diff --git a/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/impl/adapters/XnfApiServer.kt b/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/impl/adapters/XnfApiServer.kt new file mode 100644 index 00000000..06f1cffe --- /dev/null +++ b/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/impl/adapters/XnfApiServer.kt @@ -0,0 +1,90 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.xnf.impl.adapters + +import arrow.core.Either +import arrow.effects.IO +import org.onap.dcae.collectors.veshv.simulators.xnf.impl.OngoingSimulations +import org.onap.dcae.collectors.veshv.simulators.xnf.impl.XnfSimulator +import org.onap.dcae.collectors.veshv.utils.http.HttpConstants +import org.onap.dcae.collectors.veshv.utils.http.Response +import org.onap.dcae.collectors.veshv.utils.http.Responses +import org.onap.dcae.collectors.veshv.utils.http.sendAndHandleErrors +import org.onap.dcae.collectors.veshv.utils.http.sendEitherErrorOrResponse +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import org.onap.dcae.collectors.veshv.ves.message.generator.api.ParsingError +import ratpack.handling.Chain +import ratpack.handling.Context +import ratpack.http.TypedData +import ratpack.server.RatpackServer +import ratpack.server.ServerConfig +import java.util.* + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since June 2018 + */ +internal class XnfApiServer( + private val xnfSimulator: XnfSimulator, + private val ongoingSimulations: OngoingSimulations) { + + fun start(port: Int): IO<RatpackServer> = IO { + RatpackServer.start { server -> + server.serverConfig(ServerConfig.embedded().port(port)) + .handlers(this::configureHandlers) + } + } + + private fun configureHandlers(chain: Chain) { + chain + .post("simulator", ::startSimulationHandler) + .post("simulator/async", ::startSimulationHandler) + .get("simulator/:id", ::simulatorStatusHandler) + .get("healthcheck") { ctx -> + logger.info("Checking health") + ctx.response.status(HttpConstants.STATUS_OK).send() + } + } + + private fun startSimulationHandler(ctx: Context) { + logger.info("Starting asynchronous scenario") + ctx.request.body.then { body -> + val id = startSimulation(body) + ctx.response.sendEitherErrorOrResponse(id) + } + } + + private fun startSimulation(body: TypedData): Either<ParsingError, Response> { + return xnfSimulator.startSimulation(body.inputStream) + .map(ongoingSimulations::startAsynchronousSimulation) + .map(Responses::acceptedResponse) + } + + private fun simulatorStatusHandler(ctx: Context) { + val id = UUID.fromString(ctx.pathTokens["id"]) + val status = ongoingSimulations.status(id) + val response = Responses.statusResponse(status.toString(), status.message) + ctx.response.sendAndHandleErrors(IO.just(response)) + } + + companion object { + private val logger = Logger(XnfApiServer::class) + } +} diff --git a/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/impl/config/ArgXnfSimulatorConfiguration.kt b/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/impl/config/ArgXnfSimulatorConfiguration.kt new file mode 100644 index 00000000..0b321362 --- /dev/null +++ b/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/impl/config/ArgXnfSimulatorConfiguration.kt @@ -0,0 +1,74 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.xnf.impl.config + +import arrow.core.Option +import arrow.core.fix +import arrow.instances.option.monad.monad +import arrow.typeclasses.binding +import org.apache.commons.cli.CommandLine +import org.apache.commons.cli.DefaultParser +import org.onap.dcae.collectors.veshv.domain.WireFrameMessage +import org.onap.dcae.collectors.veshv.ssl.boundary.createSecurityConfiguration +import org.onap.dcae.collectors.veshv.utils.commandline.ArgBasedConfiguration +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.KEY_STORE_FILE +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.KEY_STORE_PASSWORD +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.LISTEN_PORT +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.MAXIMUM_PAYLOAD_SIZE_BYTES +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.SSL_DISABLE +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.TRUST_STORE_FILE +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.TRUST_STORE_PASSWORD +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.VES_HV_HOST +import org.onap.dcae.collectors.veshv.utils.commandline.CommandLineOption.VES_HV_PORT +import org.onap.dcae.collectors.veshv.utils.commandline.intValue +import org.onap.dcae.collectors.veshv.utils.commandline.stringValue + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since June 2018 + */ +internal class ArgXnfSimulatorConfiguration : ArgBasedConfiguration<SimulatorConfiguration>(DefaultParser()) { + override val cmdLineOptionsList = listOf( + VES_HV_PORT, + VES_HV_HOST, + LISTEN_PORT, + MAXIMUM_PAYLOAD_SIZE_BYTES, + SSL_DISABLE, + KEY_STORE_FILE, + KEY_STORE_PASSWORD, + TRUST_STORE_FILE, + TRUST_STORE_PASSWORD) + + override fun getConfiguration(cmdLine: CommandLine): Option<SimulatorConfiguration> = + Option.monad().binding { + val listenPort = cmdLine.intValue(LISTEN_PORT).bind() + val vesHost = cmdLine.stringValue(VES_HV_HOST).bind() + val vesPort = cmdLine.intValue(VES_HV_PORT).bind() + val maxPayloadSizeBytes = cmdLine.intValue(MAXIMUM_PAYLOAD_SIZE_BYTES, + WireFrameMessage.DEFAULT_MAX_PAYLOAD_SIZE_BYTES) + + SimulatorConfiguration( + listenPort, + vesHost, + vesPort, + maxPayloadSizeBytes, + createSecurityConfiguration(cmdLine).bind()) + }.fix() +} diff --git a/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/impl/config/SimulatorConfiguration.kt b/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/impl/config/SimulatorConfiguration.kt new file mode 100644 index 00000000..3395d282 --- /dev/null +++ b/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/impl/config/SimulatorConfiguration.kt @@ -0,0 +1,33 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.xnf.impl.config + +import org.onap.dcae.collectors.veshv.domain.SecurityConfiguration + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since June 2018 + */ +data class SimulatorConfiguration( + val listenPort: Int, + val vesHost: String, + val vesPort: Int, + val maxPayloadSizeBytes: Int, + val security: SecurityConfiguration) diff --git a/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/impl/simulations.kt b/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/impl/simulations.kt new file mode 100644 index 00000000..21748ae8 --- /dev/null +++ b/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/impl/simulations.kt @@ -0,0 +1,76 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.xnf.impl + +import arrow.effects.IO +import kotlinx.coroutines.asCoroutineDispatcher +import org.onap.dcae.collectors.veshv.simulators.xnf.impl.adapters.XnfApiServer +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import java.util.* +import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.Executor +import java.util.concurrent.Executors + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since August 2018 + */ +class OngoingSimulations(executor: Executor = Executors.newCachedThreadPool()) { + private val asyncSimulationContext = executor.asCoroutineDispatcher() + private val simulations = ConcurrentHashMap<UUID, Status>() + + fun startAsynchronousSimulation(simulationIo: IO<Unit>): UUID { + val id = UUID.randomUUID() + simulations[id] = StatusOngoing + + simulationIo.continueOn(asyncSimulationContext).unsafeRunAsync { result -> + result.fold( + { err -> + logger.warn("Error", err) + simulations[id] = StatusFailure(err) + }, + { + logger.info("Finished sending messages") + simulations[id] = StatusSuccess + } + ) + } + return id + } + + fun status(id: UUID) = simulations.getOrDefault(id, StatusNotFound) + + internal fun clear() { + simulations.clear() + } + + companion object { + private val logger = Logger(XnfApiServer::class) + } +} + +sealed class Status(val message: String) { + override fun toString() = this::class.simpleName ?: "null" +} + +object StatusNotFound : Status("not found") +object StatusOngoing : Status("ongoing") +object StatusSuccess : Status("success") +data class StatusFailure(val cause: Throwable) : Status("Error ${cause.message}") diff --git a/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/main.kt b/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/main.kt new file mode 100644 index 00000000..4512dfbf --- /dev/null +++ b/sources/hv-collector-xnf-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/xnf/main.kt @@ -0,0 +1,61 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.simulators.xnf + +import org.onap.dcae.collectors.veshv.simulators.xnf.impl.OngoingSimulations +import org.onap.dcae.collectors.veshv.simulators.xnf.impl.XnfSimulator +import org.onap.dcae.collectors.veshv.simulators.xnf.impl.config.ArgXnfSimulatorConfiguration +import org.onap.dcae.collectors.veshv.simulators.xnf.impl.adapters.XnfApiServer +import org.onap.dcae.collectors.veshv.simulators.xnf.impl.adapters.VesHvClient +import org.onap.dcae.collectors.veshv.utils.arrow.ExitFailure +import org.onap.dcae.collectors.veshv.utils.arrow.unsafeRunEitherSync +import org.onap.dcae.collectors.veshv.utils.arrow.unit +import org.onap.dcae.collectors.veshv.utils.commandline.handleWrongArgumentErrorCurried +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import org.onap.dcae.collectors.veshv.ves.message.generator.factory.MessageGeneratorFactory + +private const val PACKAGE_NAME = "org.onap.dcae.collectors.veshv.simulators.xnf" +private val logger = Logger(PACKAGE_NAME) +const val PROGRAM_NAME = "java $PACKAGE_NAME.MainKt" + +/** + * @author Jakub Dudycz <jakub.dudycz@nokia.com> + * @since June 2018 + */ +fun main(args: Array<String>) = ArgXnfSimulatorConfiguration().parse(args) + .mapLeft(handleWrongArgumentErrorCurried(PROGRAM_NAME)) + .map { config -> + logger.info("Using configuration: $config") + val xnfSimulator = XnfSimulator( + VesHvClient(config), + MessageGeneratorFactory.create(config.maxPayloadSizeBytes)) + XnfApiServer(xnfSimulator, OngoingSimulations()) + .start(config.listenPort) + .unit() + } + .unsafeRunEitherSync( + { ex -> + logger.error("Failed to start a server", ex) + ExitFailure(1) + }, + { + logger.info("Started xNF Simulator API server") + } + ) diff --git a/sources/hv-collector-xnf-simulator/src/main/resources/logback.xml b/sources/hv-collector-xnf-simulator/src/main/resources/logback.xml new file mode 100644 index 00000000..809f62d4 --- /dev/null +++ b/sources/hv-collector-xnf-simulator/src/main/resources/logback.xml @@ -0,0 +1,35 @@ +<?xml version="1.0" encoding="UTF-8"?> +<configuration> + <property name="LOG_FILE" + value="${LOG_FILE:-${LOG_PATH:-${LOG_TEMP:-${java.io.tmpdir:-/tmp}}/}ves-hv.log}"/> + <property name="FILE_LOG_PATTERN" value="%d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %-5level [%-40.40logger{10}] - %msg%n"/> + + <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> + <encoder> + <pattern> + %d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %highlight(%-5level) [%-40.40logger{10}] - %msg%n + </pattern> + </encoder> + </appender> + + <appender name="ROLLING-FILE" + class="ch.qos.logback.core.rolling.RollingFileAppender"> + <encoder> + <pattern>${FILE_LOG_PATTERN}</pattern> + </encoder> + <file>${LOG_FILE}</file> + <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy"> + <fileNamePattern>${LOG_FILE}.%d{yyyy-MM-dd}.log</fileNamePattern> + <maxFileSize>50MB</maxFileSize> + <maxHistory>30</maxHistory> + <totalSizeCap>10GB</totalSizeCap> + </rollingPolicy> + </appender> + + <logger name="org.onap.dcae.collectors.veshv" level="DEBUG"/> + + <root level="INFO"> + <appender-ref ref="CONSOLE"/> + <appender-ref ref="ROLLING-FILE"/> + </root> +</configuration>
\ No newline at end of file diff --git a/sources/hv-collector-xnf-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/main/OngoingSimulationsTest.kt b/sources/hv-collector-xnf-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/main/OngoingSimulationsTest.kt new file mode 100644 index 00000000..a04da7bf --- /dev/null +++ b/sources/hv-collector-xnf-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/main/OngoingSimulationsTest.kt @@ -0,0 +1,106 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.main + +import arrow.effects.IO +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.given +import org.jetbrains.spek.api.dsl.it +import org.jetbrains.spek.api.dsl.on +import org.onap.dcae.collectors.veshv.simulators.xnf.impl.OngoingSimulations +import org.onap.dcae.collectors.veshv.simulators.xnf.impl.StatusFailure +import org.onap.dcae.collectors.veshv.simulators.xnf.impl.StatusNotFound +import org.onap.dcae.collectors.veshv.simulators.xnf.impl.StatusOngoing +import org.onap.dcae.collectors.veshv.simulators.xnf.impl.StatusSuccess +import org.onap.dcae.collectors.veshv.tests.utils.waitUntilSucceeds +import java.util.* +import java.util.concurrent.Executors + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since September 2018 + */ +internal class OngoingSimulationsTest : Spek({ + val executor = Executors.newSingleThreadExecutor() + val cut = OngoingSimulations(executor) + + describe("simulations repository") { + given("not existing task task id") { + val id = UUID.randomUUID() + + on("status") { + val result = cut.status(id) + + it("should have 'not found' status") { + assertThat(result).isEqualTo(StatusNotFound) + } + } + } + + given("never ending task") { + val task = IO.async<Unit> { } + + on("startAsynchronousSimulation") { + val result = cut.startAsynchronousSimulation(task) + + it("should have ongoing status") { + assertThat(cut.status(result)).isEqualTo(StatusOngoing) + } + } + } + + given("failing task") { + val cause = RuntimeException("facepalm") + val task = IO.raiseError<Unit>(cause) + + on("startAsynchronousSimulation") { + val result = cut.startAsynchronousSimulation(task) + + it("should have failing status") { + waitUntilSucceeds { + assertThat(cut.status(result)).isEqualTo(StatusFailure(cause)) + } + } + } + } + + given("successful task") { + val task = IO { println("great success!") } + + on("startAsynchronousSimulation") { + val result = cut.startAsynchronousSimulation(task) + + it("should have successful status") { + waitUntilSucceeds { + assertThat(cut.status(result)).isEqualTo(StatusSuccess) + } + } + } + } + + afterGroup { + executor.shutdown() + } + } + + afterEachTest { cut.clear() } +}) diff --git a/sources/hv-collector-xnf-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/main/XnfSimulatorTest.kt b/sources/hv-collector-xnf-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/main/XnfSimulatorTest.kt new file mode 100644 index 00000000..95510e77 --- /dev/null +++ b/sources/hv-collector-xnf-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/main/XnfSimulatorTest.kt @@ -0,0 +1,114 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 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.main + +import arrow.core.Left +import arrow.core.None +import arrow.core.Right +import arrow.effects.IO +import com.nhaarman.mockitokotlin2.any +import com.nhaarman.mockitokotlin2.mock +import com.nhaarman.mockitokotlin2.whenever +import org.jetbrains.spek.api.Spek +import org.jetbrains.spek.api.dsl.describe +import org.jetbrains.spek.api.dsl.it +import org.onap.dcae.collectors.veshv.domain.WireFrameMessage +import org.onap.dcae.collectors.veshv.simulators.xnf.impl.XnfSimulator +import org.onap.dcae.collectors.veshv.simulators.xnf.impl.adapters.VesHvClient +import org.onap.dcae.collectors.veshv.tests.utils.Assertions.assertThat +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageGenerator +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageParameters +import org.onap.dcae.collectors.veshv.ves.message.generator.api.MessageParametersParser +import org.onap.dcae.collectors.veshv.ves.message.generator.api.ParsingError +import reactor.core.publisher.Flux +import java.io.ByteArrayInputStream + +/** + * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com> + * @since September 2018 + */ +internal class XnfSimulatorTest : Spek({ + lateinit var cut: XnfSimulator + lateinit var vesClient: VesHvClient + lateinit var messageParametersParser: MessageParametersParser + lateinit var messageGenerator: MessageGenerator + + beforeEachTest { + vesClient = mock() + messageParametersParser = mock() + messageGenerator = mock() + cut = XnfSimulator(vesClient, messageGenerator, messageParametersParser) + } + + describe("startSimulation") { + it("should fail when empty input stream") { + // given + val emptyInputStream = ByteArrayInputStream(byteArrayOf()) + + // when + val result = cut.startSimulation(emptyInputStream) + + // then + assertThat(result).isLeft() + } + + it("should fail when invalid JSON") { + // given + val invalidJson = "invalid json".byteInputStream() + + // when + val result = cut.startSimulation(invalidJson) + + // then + assertThat(result).isLeft() + } + + it("should fail when JSON syntax is valid but content is invalid") { + // given + val json = "[1,2,3]".byteInputStream() + val cause = ParsingError("epic fail", None) + whenever(messageParametersParser.parse(any())).thenReturn( + Left(cause)) + + // when + val result = cut.startSimulation(json) + + // then + assertThat(result).left().isEqualTo(cause) + } + + it("should return generated messages") { + // given + val json = "[true]".byteInputStream() + val messageParams = listOf<MessageParameters>() + val generatedMessages = Flux.empty<WireFrameMessage>() + val sendingIo = IO {} + whenever(messageParametersParser.parse(any())).thenReturn(Right(messageParams)) + whenever(messageGenerator.createMessageFlux(messageParams)).thenReturn(generatedMessages) + whenever(vesClient.sendIo(generatedMessages)).thenReturn(sendingIo) + + // when + val result = cut.startSimulation(json) + + // then + assertThat(result).right().isSameAs(sendingIo) + } + } +}) diff --git a/sources/pom.xml b/sources/pom.xml new file mode 100644 index 00000000..06726bbb --- /dev/null +++ b/sources/pom.xml @@ -0,0 +1,135 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ ============LICENSE_START======================================================= + ~ dcaegen2-collectors-veshv + ~ ================================================================================ + ~ Copyright (C) 2018 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========================================================= + --> +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + + <modelVersion>4.0.0</modelVersion> + + <licenses> + <license> + <name>The Apache Software License, Version 2.0</name> + <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url> + </license> + </licenses> + + <parent> + <groupId>org.onap.dcaegen2.collectors.hv-ves</groupId> + <artifactId>ves-hv-collector</artifactId> + <version>1.1.0-SNAPSHOT</version> + <relativePath>..</relativePath> + </parent> + + <artifactId>hv-collector-sources</artifactId> + <version>1.1.0-SNAPSHOT</version> + <description>VES HighVolume Collector :: Sources</description> + <packaging>pom</packaging> + + <build> + <plugins> + <plugin> + <artifactId>maven-checkstyle-plugin</artifactId> + <version>2.17</version> + <dependencies> + <dependency> + <groupId>org.onap.oparent</groupId> + <artifactId>checkstyle</artifactId> + <version>1.1.1</version> + </dependency> + </dependencies> + <executions> + <execution> + <!-- To override oparent configuration different id must be used + We need to override it to include .kt files in check. --> + <id>check-license-kotlin</id> + <goals> + <goal>check</goal> + </goals> + <phase>process-sources</phase> + <configuration> + <configLocation>onap-checkstyle/check-license.xml</configLocation> + <includeResources>false</includeResources> + <includeTestSourceDirectory>true</includeTestSourceDirectory> + <includeTestResources>false</includeTestResources> + <includes>**\/*.kt</includes> + <consoleOutput>true</consoleOutput> + <!--<failOnViolation>false</failOnViolation>--> + </configuration> + </execution> + </executions> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-antrun-plugin</artifactId> + <version>1.8</version> + <executions> + <execution> + <!-- This can be run separately with mvn antrun:run@detekt --> + <id>detekt</id> + <phase>verify</phase> + <goals> + <goal>run</goal> + </goals> + <configuration> + <target name="detekt" unless="${skipAnalysis}"> + <java taskname="detekt" dir="${basedir}" fork="true" failonerror="true" classname="io.gitlab.arturbosch.detekt.cli.Main" classpathref="maven.plugin.classpath"> + <arg value="--input"/> + <arg value="${basedir}/src/main/kotlin"/> + <arg value="--config-resource"/> + <arg value="onap-detekt-config.yml"/> + <arg value="--filters"/> + <arg value=".*/target/.*,.*/resources/.*"/> + <arg value="--report"/> + <arg value="html:${basedir}/target/detekt-report.html"/> + </java> + </target> + </configuration> + </execution> + </executions> + <dependencies> + <dependency> + <groupId>io.gitlab.arturbosch.detekt</groupId> + <artifactId>detekt-cli</artifactId> + <version>${detekt.version}</version> + </dependency> + <dependency> + <groupId>${project.groupId}</groupId> + <artifactId>hv-collector-analysis</artifactId> + <version>1.1.0-SNAPSHOT</version> + </dependency> + </dependencies> + </plugin> + </plugins> + </build> + + <modules> + <module>hv-collector-core</module> + <module>hv-collector-ct</module> + <module>hv-collector-dcae-app-simulator</module> + <module>hv-collector-domain</module> + <module>hv-collector-health-check</module> + <module>hv-collector-main</module> + <module>hv-collector-ssl</module> + <module>hv-collector-test-utils</module> + <module>hv-collector-utils</module> + <module>hv-collector-ves-message-generator</module> + <module>hv-collector-xnf-simulator</module> + </modules> +</project> |