summaryrefslogtreecommitdiffstats
path: root/sources/hv-collector-dcae-app-simulator
diff options
context:
space:
mode:
authorkjaniak <kornel.janiak@nokia.com>2019-06-26 15:57:29 +0200
committerkjaniak <kornel.janiak@nokia.com>2019-07-02 14:53:54 +0200
commit3e4f108d66fb31e5ea713fef1cf7bc99c0510c21 (patch)
tree74ca728ac2c4b43d6bbd1d39ce4335deca4f6040 /sources/hv-collector-dcae-app-simulator
parent31dffbef6a4aab8e19d2c5a9f5ac8531d931197a (diff)
Reorganize kafka module
Change-Id: I2eb7a8a6e92c9d89586b877f4cae438497b62ae2 Issue-ID: DCAEGEN2-1635 Signed-off-by: kjaniak <kornel.janiak@nokia.com>
Diffstat (limited to 'sources/hv-collector-dcae-app-simulator')
-rw-r--r--sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/adapters/KafkaSource.kt54
-rw-r--r--sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/consumer.kt26
-rw-r--r--sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/ConsumerTest.kt5
-rw-r--r--sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/DcaeAppConsumerFactoryTest.kt54
-rw-r--r--sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/adapters/KafkaSourceTest.kt84
-rw-r--r--sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/kafka.kt26
6 files changed, 236 insertions, 13 deletions
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..1b664edc
--- /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,54 @@
+/*
+ * ============LICENSE_START=======================================================
+ * dcaegen2-collectors-veshv
+ * ================================================================================
+ * Copyright (C) 2018,2019 NOKIA
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * 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.onap.dcae.collectors.veshv.kafka.api.KafkaPropertiesFactory
+import org.onap.dcae.collectors.veshv.utils.logging.Logger
+import reactor.core.publisher.Flux
+import reactor.kafka.receiver.KafkaReceiver
+import reactor.kafka.receiver.ReceiverOptions
+import reactor.kafka.receiver.ReceiverRecord
+
+/**
+ * @author Piotr Jaszczyk <piotr.jaszczyk@nokia.com>
+ * @since May 2018
+ */
+internal class KafkaSource(private val receiver: KafkaReceiver<ByteArray, ByteArray>) {
+ fun start(): Flux<ReceiverRecord<ByteArray, ByteArray>> =
+ receiver.receive()
+ .doOnNext { it.receiverOffset().acknowledge() }
+ .also { logger.info { "Started Kafka source" } }
+
+ companion object {
+ private val logger = Logger(KafkaSource::class)
+
+ fun create(bootstrapServers: String, topics: Set<String>) =
+ KafkaSource(KafkaReceiver.create(createReceiverOptions(bootstrapServers, topics)))
+
+ fun createReceiverOptions(bootstrapServers: String,
+ topics: Set<String>): ReceiverOptions<ByteArray, ByteArray>? {
+ val props = KafkaPropertiesFactory.create(bootstrapServers)
+ 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/consumer.kt b/sources/hv-collector-dcae-app-simulator/src/main/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/consumer.kt
index 8a7aafbe..6ee640a4 100644
--- 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
@@ -20,8 +20,7 @@
package org.onap.dcae.collectors.veshv.simulators.dcaeapp.impl
import org.apache.kafka.clients.consumer.ConsumerRecord
-import org.onap.dcae.collectors.veshv.kafka.api.ConsumerFactory
-import org.onap.dcae.collectors.veshv.kafka.api.KafkaConsumer
+import org.onap.dcae.collectors.veshv.simulators.dcaeapp.impl.adapters.KafkaSource
import org.onap.dcae.collectors.veshv.utils.logging.Logger
import java.util.concurrent.ConcurrentLinkedQueue
@@ -41,9 +40,10 @@ internal class ConsumerState(private val messages: ConcurrentLinkedQueue<ByteArr
internal interface ConsumerStateProvider {
fun currentState(): ConsumerState
+ fun reset()
}
-internal class Consumer : KafkaConsumer, ConsumerStateProvider {
+internal class Consumer : ConsumerStateProvider {
private var consumedMessages: ConcurrentLinkedQueue<ByteArray> = ConcurrentLinkedQueue()
@@ -51,7 +51,7 @@ internal class Consumer : KafkaConsumer, ConsumerStateProvider {
override fun reset() = consumedMessages.clear()
- override fun update(record: ConsumerRecord<ByteArray, ByteArray>) {
+ fun update(record: ConsumerRecord<ByteArray, ByteArray>) {
logger.trace { "Updating stats for message from ${record.topic()}:${record.partition()}" }
consumedMessages.add(record.value())
}
@@ -63,9 +63,19 @@ internal class Consumer : KafkaConsumer, ConsumerStateProvider {
internal class DcaeAppConsumerFactory(private val kafkaBootstrapServers: String) {
- private val consumerProvider = { Consumer() }
-
fun createConsumersFor(topics: Set<String>) =
- ConsumerFactory.createConsumersForTopics(kafkaBootstrapServers, topics, consumerProvider)
- .mapValues { it.value as Consumer }
+ KafkaSource.create(kafkaBootstrapServers, topics).let { kafkaSource ->
+ val topicToConsumer = topics.associateWith { Consumer() }
+ kafkaSource.start()
+ .map {
+ val topic = it.topic()
+ topicToConsumer.get(topic)?.update(it)
+ ?: logger.warn { "No consumer configured for topic $topic" }
+ }.subscribe()
+ topicToConsumer
+ }
+
+ companion object {
+ private val logger = Logger(DcaeAppConsumerFactory::class)
+ }
}
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
index a594215b..728eb2fd 100644
--- 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
@@ -19,12 +19,10 @@
*/
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
/**
@@ -77,6 +75,3 @@ private fun assertState(cut: Consumer, vararg values: ByteArray) {
assertThat(cut.currentState().messagesCount)
.isEqualTo(values.size)
}
-
-private 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/DcaeAppConsumerFactoryTest.kt b/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/DcaeAppConsumerFactoryTest.kt
new file mode 100644
index 00000000..71f31aba
--- /dev/null
+++ b/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/DcaeAppConsumerFactoryTest.kt
@@ -0,0 +1,54 @@
+/*
+ * ============LICENSE_START=======================================================
+ * dcaegen2-collectors-veshv
+ * ================================================================================
+ * Copyright (C) 2019 NOKIA
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ * ============LICENSE_END=========================================================
+ */
+package org.onap.dcae.collectors.veshv.simulators.dcaeapp.impl
+
+import org.assertj.core.api.Assertions.assertThat
+import org.jetbrains.spek.api.Spek
+import org.jetbrains.spek.api.dsl.describe
+import org.jetbrains.spek.api.dsl.it
+import org.jetbrains.spek.api.dsl.on
+
+object DcaeAppConsumerFactoryTest : Spek({
+ describe("DcaeAppConsumerFactory") {
+ val kafkaBootstrapServers = "0.0.0.0:40,0.0.0.1:41"
+ val dcaeAppConsumerFactory = DcaeAppConsumerFactory(kafkaBootstrapServers)
+
+ on("creation of consumer") {
+ val kafkaTopics = setOf("topic1", "topic2")
+ val consumer = dcaeAppConsumerFactory.createConsumersFor(kafkaTopics)
+
+ it("should create consumer") {
+ assertThat(consumer).isNotEmpty.hasSize(2)
+ assertThat(consumer).containsOnlyKeys("topic1", "topic2")
+ }
+ }
+
+ on("empty kafkaTopics set") {
+ val emptyKafkaTopics = emptySet<String>()
+ val consumer = dcaeAppConsumerFactory.createConsumersFor(emptyKafkaTopics)
+
+ it("should not create consumer") {
+ assertThat(consumer).isEmpty()
+ }
+ }
+
+
+ }
+}) \ 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/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..5bfbc91c
--- /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,84 @@
+/*
+ * ============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 com.nhaarman.mockitokotlin2.doNothing
+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 reactor.core.publisher.Flux
+import reactor.kafka.receiver.KafkaReceiver
+import reactor.kafka.receiver.ReceiverOffset
+import reactor.kafka.receiver.ReceiverRecord
+import reactor.test.StepVerifier
+
+/**
+ * @author Piotr Jaszczyk <piotr.jaszczyk></piotr.jaszczyk>@nokia.com>
+ * @since August 2018
+ */
+internal class KafkaSourceTest : Spek({
+
+ describe("KafkaSource"){
+ given("mocked Kafka Receiver"){
+ val mockedKafkaReceiver = mock<KafkaReceiver<ByteArray, ByteArray>>()
+ val mockedReceiverRecord = mock<ReceiverRecord<ByteArray, ByteArray>>()
+ whenever(mockedKafkaReceiver.receive()).thenReturn(Flux.just(mockedReceiverRecord))
+ on("function that starts KafkaSource") {
+ val mockedReceiverOffset = mock<ReceiverOffset>()
+ whenever(mockedReceiverRecord.receiverOffset()).thenReturn(mockedReceiverOffset)
+ doNothing().`when`(mockedReceiverOffset).acknowledge()
+
+ val testedFunction = { KafkaSource(mockedKafkaReceiver).start() }
+ it("should emmit receiver record") {
+ StepVerifier.create(testedFunction())
+ .expectSubscription()
+ .expectNext(mockedReceiverRecord)
+ .expectComplete()
+ .verify()
+ }
+ }
+ }
+ }
+
+ given("parameters for factory methods") {
+ val servers = "kafka1:9080,kafka2:9080"
+ val topics = setOf("topic1", "topic2")
+
+ on("createReceiverOptions call with topics set") {
+ val options = KafkaSource.createReceiverOptions(servers, topics)
+ it("should generate options with provided topics") {
+ assertThat(options!!.subscriptionTopics()).contains("topic1", "topic2")
+ }
+ }
+
+ on("create call"){
+ val kafkaSource = KafkaSource.create(servers, topics)
+ it("should generate KafkaSource object") {
+ assertThat(kafkaSource).isInstanceOf(KafkaSource::class.java)
+ }
+ }
+ }
+
+})
diff --git a/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/kafka.kt b/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/kafka.kt
new file mode 100644
index 00000000..ac26cf17
--- /dev/null
+++ b/sources/hv-collector-dcae-app-simulator/src/test/kotlin/org/onap/dcae/collectors/veshv/simulators/dcaeapp/impl/kafka.kt
@@ -0,0 +1,26 @@
+/*
+ * ============LICENSE_START=======================================================
+ * dcaegen2-collectors-veshv
+ * ================================================================================
+ * Copyright (C) 2019 NOKIA
+ * ================================================================================
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ * ============LICENSE_END=========================================================
+ */
+package org.onap.dcae.collectors.veshv.simulators.dcaeapp.impl
+
+import org.apache.kafka.clients.consumer.ConsumerRecord
+import reactor.kafka.receiver.ReceiverRecord
+
+internal fun receiverRecord(topic: String, key: ByteArray, value: ByteArray) =
+ ReceiverRecord(ConsumerRecord(topic, 1, 100L, key, value), null)