diff options
13 files changed, 216 insertions, 30 deletions
@@ -28,8 +28,7 @@ meetings: repeats: 'weekly' time: '13:00 UTC (DST), 15:30 UTC (post DST)' repositories: - - 'dcaegen2-collectors-hv-ves' - - 'dcaegen2-services-prh' + - 'dcaegen2/collectors/hv-ves' committers: - <<: *onap_dcaegen2_ptl - name: 'Lusheng Ji' @@ -62,11 +61,6 @@ committers: company: 'ATT' id: 'jflucas' timezone: 'America/New_York' - - name: 'Piotr Jaszczyk' - email: 'piotr.jaszczyk@nokia.com' - company: 'Nokia' - id: 'jaszczur' - timezone: 'Europe/Warsaw' - name: 'Piotr Wielebski' email: 'piotr.wielebski@nokia.com' company: 'Nokia' @@ -74,11 +68,16 @@ committers: timezone: 'Europe/Warsaw' tsc: approval: 'https://lists.onap.org/pipermail/onap-tsc' - changes: + changes: - type: 'Removal' name: 'Przemyslaw Wasala' link: 'https://lists.onap.org/g/onap-tsc/message/4248' - type: 'Addition' name: 'Piotr Jaszczyk' + link: 'https://lists.onap.org/g/onap-tsc/message/4259' + - type: 'Addition' name: 'Piotr Wielebski' link: 'https://lists.onap.org/g/onap-tsc/message/4259' + - type: 'Removal' + name: 'Piotr Jaszczyk' + link: 'https://lists.onap.org/g/onap-tsc/message/5539' 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 index 28b28203..41993e62 100644 --- 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 @@ -20,6 +20,7 @@ package org.onap.dcae.collectors.veshv.boundary import org.onap.dcae.collectors.veshv.domain.RoutedMessage +import org.onap.dcae.collectors.veshv.domain.VesMessage import org.onap.dcae.collectors.veshv.domain.WireFrameMessage import org.onap.dcae.collectors.veshv.domain.logging.ClientContext import org.onap.dcae.collectors.veshv.model.ClientRejectionCause @@ -42,6 +43,7 @@ interface SinkFactory : Closeable { interface Metrics { fun notifyBytesReceived(size: Int) fun notifyMessageReceived(msg: WireFrameMessage) + fun notifyMessageReadyForRouting(msg: VesMessage) fun notifyMessageSent(msg: RoutedMessage) fun notifyMessageDropped(cause: MessageDropCause) fun notifyClientDisconnected() diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/HvVesCollector.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/HvVesCollector.kt index ac7c3917..f0d1465b 100644 --- a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/HvVesCollector.kt +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/HvVesCollector.kt @@ -92,6 +92,7 @@ internal class HvVesCollector( } private fun route(flux: Flux<VesMessage>) = flux + .doOnNext(metrics::notifyMessageReadyForRouting) .flatMap(router::route) .doOnNext(this::updateSinkMetrics) 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 index a450b794..3b01d137 100644 --- 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 @@ -24,6 +24,7 @@ import org.onap.dcae.collectors.veshv.domain.WireFrameMessage import org.onap.dcae.collectors.veshv.model.ClientRejectionCause import org.onap.dcae.collectors.veshv.model.MessageDropCause import org.onap.dcae.collectors.veshv.domain.RoutedMessage +import org.onap.dcae.collectors.veshv.domain.VesMessage import java.time.Duration import java.time.Instant import kotlin.test.fail @@ -38,6 +39,7 @@ class FakeMetrics : Metrics { var messageBytesReceived: Int = 0; private set var messagesDroppedCount: Int = 0; private set var lastProcessingTimeMicros: Double = -1.0; private set + var lastProcessingTimeWithoutRoutingMicros: Double = -1.0; private set var messagesSentCount: Int = 0; private set var clientRejectionCause = mutableMapOf<ClientRejectionCause, Int>(); private set @@ -52,6 +54,10 @@ class FakeMetrics : Metrics { messageBytesReceived += msg.payloadSize } + override fun notifyMessageReadyForRouting(msg: VesMessage) { + lastProcessingTimeWithoutRoutingMicros = Duration.between(msg.wtpFrame.receivedAt, Instant.now()).toNanos() / 1000.0 + } + override fun notifyMessageSent(msg: RoutedMessage) { messagesSentCount++ messagesSentToTopic.compute(msg.targetTopic) { k, _ -> diff --git a/sources/hv-collector-kafka-consumer/src/main/kotlin/org/onap/dcae/collectors/veshv/kafkaconsumer/metrics/MicrometerMetrics.kt b/sources/hv-collector-kafka-consumer/src/main/kotlin/org/onap/dcae/collectors/veshv/kafkaconsumer/metrics/MicrometerMetrics.kt index 906fce23..64dbdf49 100644 --- a/sources/hv-collector-kafka-consumer/src/main/kotlin/org/onap/dcae/collectors/veshv/kafkaconsumer/metrics/MicrometerMetrics.kt +++ b/sources/hv-collector-kafka-consumer/src/main/kotlin/org/onap/dcae/collectors/veshv/kafkaconsumer/metrics/MicrometerMetrics.kt @@ -42,6 +42,7 @@ internal class MicrometerMetrics constructor( }.memoize<String, AtomicLong>() private val travelTime = Timer.builder(name(TRAVEL,TIME)) + .maximumExpectedValue(MAX_BUCKET_DURATION) .publishPercentileHistogram(true) .register(registry) @@ -65,6 +66,7 @@ internal class MicrometerMetrics constructor( private const val TRAVEL = "travel" private const val TIME = "time" private const val PREFIX = "hv-kafka-consumer" + internal val MAX_BUCKET_DURATION = Duration.ofSeconds(300L) private fun name(vararg name: String) = "$PREFIX.${name.joinToString(".")}" } } diff --git a/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/metrics/MicrometerMetrics.kt b/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/metrics/MicrometerMetrics.kt index c04c2c95..9d417a28 100644 --- a/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/metrics/MicrometerMetrics.kt +++ b/sources/hv-collector-main/src/main/kotlin/org/onap/dcae/collectors/veshv/main/metrics/MicrometerMetrics.kt @@ -34,6 +34,7 @@ import org.onap.dcae.collectors.veshv.domain.WireFrameMessage import org.onap.dcae.collectors.veshv.model.ClientRejectionCause import org.onap.dcae.collectors.veshv.model.MessageDropCause import org.onap.dcae.collectors.veshv.domain.RoutedMessage +import org.onap.dcae.collectors.veshv.domain.VesMessage import org.onap.dcae.collectors.veshv.utils.TimeUtils.epochMicroToInstant import java.time.Duration import java.time.Instant @@ -46,7 +47,6 @@ import java.time.Instant class MicrometerMetrics internal constructor( private val registry: PrometheusMeterRegistry = PrometheusMeterRegistry(PrometheusConfig.DEFAULT) ) : Metrics { - private val receivedBytes = registry.counter(name(DATA, RECEIVED, BYTES)) private val receivedMessages = registry.counter(name(MESSAGES, RECEIVED)) private val receivedMessagesPayloadBytes = registry.counter(name(MESSAGES, RECEIVED, PAYLOAD, BYTES)) @@ -55,9 +55,14 @@ class MicrometerMetrics internal constructor( private val disconnections = registry.counter(name(DISCONNECTIONS)) private val processingTime = Timer.builder(name(MESSAGES, PROCESSING, TIME)) + .maximumExpectedValue(MAX_BUCKET_DURATION) + .publishPercentileHistogram(true) + .register(registry) + private val processingTimeWithoutRouting = Timer.builder(name(MESSAGES, PROCESSING, TIME, WITHOUT, ROUTING)) .publishPercentileHistogram(true) .register(registry) private val totalLatency = Timer.builder(name(MESSAGES, LATENCY)) + .maximumExpectedValue(MAX_BUCKET_DURATION) .publishPercentileHistogram(true) .register(registry) @@ -65,12 +70,10 @@ class MicrometerMetrics internal constructor( private val sentMessagesByTopic = { topic: String -> registry.counter(name(MESSAGES, SENT, TOPIC), TOPIC, topic) }.memoize<String, Counter>() - private val droppedMessages = registry.counter(name(MESSAGES, DROPPED)) private val messagesDroppedByCause = { cause: String -> registry.counter(name(MESSAGES, DROPPED, CAUSE), CAUSE, cause) }.memoize<String, Counter>() - private val clientsRejected = registry.counter(name(CLIENTS, REJECTED)) private val clientsRejectedByCause = { cause: String -> registry.counter(name(CLIENTS, REJECTED, CAUSE), CAUSE, cause) @@ -95,6 +98,10 @@ class MicrometerMetrics internal constructor( receivedBytes.increment(size.toDouble()) } + override fun notifyMessageReadyForRouting(msg: VesMessage) { + processingTimeWithoutRouting.record(Duration.between(msg.wtpFrame.receivedAt, Instant.now())) + } + override fun notifyMessageReceived(msg: WireFrameMessage) { receivedMessages.increment() receivedMessagesPayloadBytes.increment(msg.payloadSize.toDouble()) @@ -147,6 +154,9 @@ class MicrometerMetrics internal constructor( internal const val TIME = "time" internal const val LATENCY = "latency" internal const val PAYLOAD = "payload" + internal val MAX_BUCKET_DURATION = Duration.ofSeconds(300L) + internal const val WITHOUT = "without" + internal const val ROUTING = "routing" internal fun name(vararg name: String) = "$PREFIX.${name.joinToString(".")}" } } 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 index 66f3a5fc..a3471d46 100644 --- 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 @@ -20,6 +20,7 @@ package org.onap.dcae.collectors.veshv.main import arrow.core.Option +import com.google.protobuf.ByteString import io.micrometer.core.instrument.Counter import io.micrometer.core.instrument.Meter import io.micrometer.core.instrument.Tags @@ -39,7 +40,9 @@ import org.onap.dcae.collectors.veshv.model.ClientRejectionCause.PAYLOAD_SIZE_EX import org.onap.dcae.collectors.veshv.model.MessageDropCause.INVALID_MESSAGE import org.onap.dcae.collectors.veshv.model.MessageDropCause.ROUTE_NOT_FOUND import org.onap.dcae.collectors.veshv.domain.RoutedMessage +import org.onap.dcae.collectors.veshv.domain.VesEventDomain import org.onap.dcae.collectors.veshv.domain.VesMessage +import org.onap.dcae.collectors.veshv.tests.utils.commonHeader import org.onap.dcae.collectors.veshv.tests.utils.emptyWireProtocolFrame import org.onap.dcae.collectors.veshv.tests.utils.verifyCounter import org.onap.dcae.collectors.veshv.tests.utils.verifyGauge @@ -203,6 +206,25 @@ object MicrometerMetricsTest : Spek({ } } + on("$PREFIX.messages.processing.time.without.routing") { + val counterName = "$PREFIX.messages.processing.time.without.routing" + val processingTimeMs = 100L + + it("should update timer") { + + cut.notifyMessageReadyForRouting(vesMessageReceivedAt(Instant.now().minusMillis(processingTimeMs))) + + registry.verifyTimer(counterName) { timer -> + assertThat(timer.mean(TimeUnit.MILLISECONDS)).isGreaterThanOrEqualTo(processingTimeMs.toDouble()) + } + verifyCountersAndTimersAreUnchangedBut( + counterName, + "$PREFIX.messages.sent.topic", + "$PREFIX.messages.sent", + "$PREFIX.messages.latency") + } + } + on("$PREFIX.messages.latency") { val counterName = "$PREFIX.messages.latency" val latencyMs = 1666L @@ -362,13 +384,19 @@ object MicrometerMetricsTest : Spek({ } }) -fun routedMessage(topic: String, partition: Int = 0) = +private fun vesMessageReceivedAt(receivedAt: Temporal, domain: VesEventDomain = VesEventDomain.PERF3GPP): VesMessage { + val commonHeader = commonHeader(domain) + return VesMessage(commonHeader, + wireProtocolFrame(commonHeader, ByteString.copyFromUtf8("highvolume measurements"), receivedAt)) +} + +private fun routedMessage(topic: String, partition: Int = 0) = vesEvent().run { toRoutedMessage(topic, partition) } -fun routedMessageReceivedAt(topic: String, receivedAt: Temporal, partition: Int = 0) = +private fun routedMessageReceivedAt(topic: String, receivedAt: Temporal, partition: Int = 0) = vesEvent().run { toRoutedMessage(topic, partition, receivedAt) } -fun routedMessageSentAt(topic: String, sentAt: Instant, partition: Int = 0) = +private fun routedMessageSentAt(topic: String, sentAt: Instant, partition: Int = 0) = vesEvent().run { val builder = toBuilder() builder.commonEventHeaderBuilder.lastEpochMicrosec = sentAt.epochSecond * 1000000 + sentAt.nano / 1000 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 index ba60d1b0..3013e904 100644 --- 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 @@ -30,6 +30,8 @@ import org.onap.dcae.collectors.veshv.domain.WireFrameMessage import org.onap.ves.VesEventOuterClass import org.onap.ves.VesEventOuterClass.CommonEventHeader import org.onap.ves.VesEventOuterClass.CommonEventHeader.Priority +import java.time.Instant +import java.time.temporal.Temporal import java.util.UUID.randomUUID fun vesEvent(domain: VesEventDomain = PERF3GPP, @@ -53,7 +55,7 @@ fun commonHeader(domain: VesEventDomain = PERF3GPP, vesEventListenerVersion: String = "7.0.2", priority: Priority = Priority.NORMAL, lastEpochMicrosec: Long = 100000005 - ): CommonEventHeader = +): CommonEventHeader = CommonEventHeader.newBuilder() .setVersion("sample-version") .setDomain(domain.domainName) @@ -86,14 +88,17 @@ fun wireProtocolFrameWithPayloadSize(size: Int): WireFrameMessage = WireFrameMes payloadType = PayloadContentType.GOOGLE_PROTOCOL_BUFFER.hexValue ) -fun wireProtocolFrame(commonHeader: CommonEventHeader, eventFields: ByteString = ByteString.EMPTY): WireFrameMessage = +fun wireProtocolFrame(commonHeader: CommonEventHeader, + eventFields: ByteString = ByteString.EMPTY, + receivedAt: Temporal = Instant.now()): WireFrameMessage = vesEventBytes(commonHeader, eventFields).let { payload -> WireFrameMessage( payload = payload, versionMajor = WireFrameMessage.SUPPORTED_VERSION_MAJOR, versionMinor = WireFrameMessage.SUPPORTED_VERSION_MINOR, payloadSize = payload.size(), - payloadType = PayloadContentType.GOOGLE_PROTOCOL_BUFFER.hexValue + payloadType = PayloadContentType.GOOGLE_PROTOCOL_BUFFER.hexValue, + receivedAt = receivedAt ) } diff --git a/tools/performance/cloud/cloud-based-performance-test.sh b/tools/performance/cloud/cloud-based-performance-test.sh index 8e2b3840..1b5f1f90 100755 --- a/tools/performance/cloud/cloud-based-performance-test.sh +++ b/tools/performance/cloud/cloud-based-performance-test.sh @@ -21,6 +21,7 @@ SCRIPT_DIRECTORY="$(pwd "$0")" CONTAINERS_COUNT=1 COMPLETED_PRODUCERS_SUM=0 LOAD_TEST="false" +TEST_CONFIG_MAP=performance-test-config PROPERTIES_FILE=${SCRIPT_DIRECTORY}/test.properties PRODUCER_APPS_LABEL=hv-collector-producer CONSUMER_APPS_LABEL=hv-collector-kafka-consumer @@ -37,12 +38,18 @@ PRODUCERS_TO_RECREATE=0 NAME_REASON_PATTERN="custom-columns=NAME:.metadata.name,REASON:.status.containerStatuses[].state.waiting.reason" HVVES_POD_NAME=$(kubectl -n ${ONAP_NAMESPACE} get pods --no-headers=true -o custom-columns=:metadata.name | grep hv-ves-collector) HVVES_CERT_PATH=/etc/ves-hv/ssl/ +KAFKA_RETENTION_TIME_MINUTES=60 +MILISECONDS_IN_MINUTE=60000 +CALC_RETENTION_TIME_IN_MS_CMD='expr $KAFKA_RETENTION_TIME_MINUTES \* $MILISECONDS_IN_MINUTE' +KAFKA_ROUTER_0_POD_NAME=$(kubectl -n ${ONAP_NAMESPACE} get pods --no-headers=true -o custom-columns=:metadata.name | grep router-kafka-0) +KAFKA_SET_TOPIC_RETENTION_TIME_CMD='kafka-topics.sh --zookeeper message-router-zookeeper:2181 --alter --topic HV_VES_PERF3GPP --config retention.ms=' +HIDE_OUTPUT='grep abc | grep 123' function clean() { echo "Cleaning up environment" echo "Attempting to delete test parameters ConfigMap" - kubectl delete configmap performance-test-config -n ${ONAP_NAMESPACE} + kubectl delete configmap ${TEST_CONFIG_MAP} -n ${ONAP_NAMESPACE} echo "Attempting to delete prometheus ConfigMap" kubectl delete configmap -l name=${PROMETHEUS_CONF_LABEL} -n ${ONAP_NAMESPACE} @@ -53,13 +60,13 @@ function clean() { echo "Attempting to delete grafana deployment and service" kubectl delete service,deployments -l app=${GRAFANA_APPS_LABEL} -n ${ONAP_NAMESPACE} - echo "Attempting to delete grafana ConfigMap(DASHBOARD)" + echo "Attempting to delete grafana ConfigMap (DASHBOARD)" kubectl delete configmap -l name=${GRAFANA_DASHBOARD} -n ${ONAP_NAMESPACE} - echo "Attempting to delete grafana ConfigMap(GRAFANA_DASHBOARD_PROVIDERS)" + echo "Attempting to delete grafana ConfigMap (GRAFANA_DASHBOARD_PROVIDERS)" kubectl delete configmap -l name=${GRAFANA_DASHBOARD_PROVIDERS} -n ${ONAP_NAMESPACE} - echo "Attempting to delete grafana ConfigMap(GRAFANA_DATASOURCE)" + echo "Attempting to delete grafana ConfigMap (GRAFANA_DATASOURCE)" kubectl delete configmap -l name=${GRAFANA_DATASOURCE} -n ${ONAP_NAMESPACE} echo "Attempting to delete consumer deployments" @@ -88,8 +95,8 @@ function copy_certs_to_hvves() { function create_producers() { echo "Recreating test properties ConfigMap from: $PROPERTIES_FILE" - kubectl delete configmap performance-test-config -n ${ONAP_NAMESPACE} - kubectl create configmap performance-test-config --from-env-file=${PROPERTIES_FILE} -n ${ONAP_NAMESPACE} + kubectl delete configmap ${TEST_CONFIG_MAP} -n ${ONAP_NAMESPACE} + kubectl create configmap ${TEST_CONFIG_MAP} --from-env-file=${PROPERTIES_FILE} -n ${ONAP_NAMESPACE} set -e for i in $(seq 1 ${CONTAINERS_COUNT}); @@ -133,6 +140,7 @@ function print_test_setup_info() { echo "Test configuration:" echo "Producer containers count: ${CONTAINERS_COUNT}" echo "Properties file path: ${PROPERTIES_FILE}" + echo "Retention time of kafka messages in minutes: ${KAFKA_RETENTION_TIME_MINUTES}" echo "________________________________________" } @@ -144,9 +152,10 @@ function usage() { echo " setup : set up ConfigMap and consumers" echo " start : create producers - start the performance test" echo " Optional parameters:" - echo " --load : should test keep defined containers number till script interruption (false)" - echo " --containers : number of producer containers to create (1)" - echo " --properties-file : path to file with benchmark properties (./test.properties)" + echo " --load : should test keep defined containers number till script interruption (false)" + echo " --containers : number of producer containers to create (1)" + echo " --properties-file : path to file with benchmark properties (./test.properties)" + echo " --retention-time-minutes : messages retention time on kafka in minutes - only for load tests (60)" echo " clean : remove ConfigMap, HV-VES consumers and producers" echo " help : print usage" echo "Example invocations:" @@ -155,6 +164,7 @@ function usage() { echo "./cloud-based-performance-test.sh start" echo "./cloud-based-performance-test.sh start --containers 10" echo "./cloud-based-performance-test.sh start --load true --containers 10" + echo "./cloud-based-performance-test.sh start --load true --containers 10 --retention-time-minutes 50" echo "./cloud-based-performance-test.sh start --properties-file ~/other_test.properties" echo "./cloud-based-performance-test.sh clean" exit 1 @@ -162,6 +172,7 @@ function usage() { function setup_environment() { echo "Setting up environment" + echo "Copying certs to hv-ves pod" copy_certs_to_hvves @@ -173,7 +184,7 @@ function setup_environment() { ./configure-consul.sh false echo "Creating test properties ConfigMap from: $PROPERTIES_FILE" - kubectl create configmap performance-test-config --from-env-file=${PROPERTIES_FILE} -n ${ONAP_NAMESPACE} + kubectl create configmap ${TEST_CONFIG_MAP} --from-env-file=${PROPERTIES_FILE} -n ${ONAP_NAMESPACE} echo "Creating consumer deployment" kubectl apply -f consumer-deployment.yaml @@ -212,6 +223,9 @@ function setup_environment() { function start_load_tests() { print_test_setup_info + echo "Setting message retention time" + kubectl exec -it ${KAFKA_ROUTER_0_POD_NAME} -n ${ONAP_NAMESPACE} -- ${KAFKA_SET_TOPIC_RETENTION_TIME_CMD}$(eval $CALC_RETENTION_TIME_IN_MS_CMD) | eval $HIDE_OUTPUT + echo "CTRL + C to stop/interrupt this script" create_producers @@ -288,6 +302,9 @@ else --properties-file) PROPERTIES_FILE=${2} ;; + --retention-time-minutes) + KAFKA_RETENTION_TIME_MINUTES=${2} + ;; *) echo "Unknown option: ${1}" usage diff --git a/tools/performance/cloud/consumer-deployment.yaml b/tools/performance/cloud/consumer-deployment.yaml index 199bf720..179f246c 100755 --- a/tools/performance/cloud/consumer-deployment.yaml +++ b/tools/performance/cloud/consumer-deployment.yaml @@ -42,7 +42,7 @@ spec: spec: containers: - name: kafka-consumer-counting - image: nexus3.dyn.nesc.nokia.net:10001/onap/org.onap.dcaegen2.collectors.hv-ves.hv-collector-kafka-consumer:1.3.0 + image: nexus3.dyn.nesc.nokia.net:10001/onap/org.onap.dcaegen2.collectors.hv-ves.hv-collector-kafka-consumer:latest ports: - containerPort: 8080 env: @@ -86,7 +86,7 @@ spec: spec: containers: - name: kafka-processing-consumer - image: nexus3.dyn.nesc.nokia.net:10001/onap/org.onap.dcaegen2.collectors.hv-ves.hv-collector-kafka-consumer:1.3.0 + image: nexus3.dyn.nesc.nokia.net:10001/onap/org.onap.dcaegen2.collectors.hv-ves.hv-collector-kafka-consumer:latest ports: - containerPort: 8080 env: diff --git a/tools/performance/cloud/grafana/dashboards/processing.yaml b/tools/performance/cloud/grafana/dashboards/processing.yaml index 8e429009..bfa8517c 100644 --- a/tools/performance/cloud/grafana/dashboards/processing.yaml +++ b/tools/performance/cloud/grafana/dashboards/processing.yaml @@ -465,7 +465,7 @@ data: "refId": "B" }, { - "expr": "histogram_quantile(0.90, sum( rate(hvves_messages_latency_seconds_bucket[1h]) )by(le) )", + "expr": "histogram_quantile(0.95, sum( rate(hvves_messages_latency_seconds_bucket[1h]) )by(le) )", "legendFormat": "95th percentile", "refId": "C" }, diff --git a/tools/performance/cloud/reboot-test-environment.sh b/tools/performance/cloud/reboot-test-environment.sh new file mode 100755 index 00000000..853df148 --- /dev/null +++ b/tools/performance/cloud/reboot-test-environment.sh @@ -0,0 +1,116 @@ +#!/usr/bin/env bash +# ============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========================================================= + +ONAP_NAMESPACE=onap +HVVES_POD_NAME=$(kubectl -n ${ONAP_NAMESPACE} get pods --no-headers=true -o custom-columns=:metadata.name | grep hv-ves-collector) +HVVES_CONTAINER_NAME=dep-dcae-hv-ves-collector +HV_VES_IMAGE="nexus3.dyn.nesc.nokia.net:10001/onap/org.onap.dcaegen2.collectors.hv-ves.hv-collector-main:latest" +KAFKA_ROUTER_0_POD_NAME=$(kubectl -n ${ONAP_NAMESPACE} get pods --no-headers=true -o custom-columns=:metadata.name | grep router-kafka-0) +KAFKA_TOPIC_RESET_CMD='kafka-topics.sh --delete --zookeeper message-router-zookeeper:2181 --topic HV_VES_PERF3GPP' +HIDE_OUTPUT='grep abc | grep 123' + +function usage() { + echo "" + echo "Reebot test environment for performance tests" + echo "Usage $0 reboot|help" + echo " reboot: reboots the test environment" + echo " help : print usage" + echo "Example invocation:" + echo "./reboot-test-environment.sh reboot" + echo "" +} + +function rebootEnvironment(){ + ./cloud-based-performance-test.sh clean + + redeployPod + + waitForPodRedeployment + + echo "Updating HV-VES image" + kubectl patch pod ${HVVES_POD_NAME} -n ${ONAP_NAMESPACE} --type='json' -p='[{"op": "replace", "path": "/spec/containers/0/image", "value":"'${HV_VES_IMAGE}'"}]' + + echo "Deleting Kafka topic" + kubectl exec -it ${KAFKA_ROUTER_0_POD_NAME} -n ${ONAP_NAMESPACE} -- ${KAFKA_TOPIC_RESET_CMD} | eval $HIDE_OUTPUT + + waitForHvVesRunning + + ./cloud-based-performance-test.sh setup +} + + +function redeployPod(){ + kubectl scale --replicas=0 deploy ${HVVES_CONTAINER_NAME} -n ${ONAP_NAMESPACE} + waitForPodTermination + kubectl scale --replicas=1 deploy ${HVVES_CONTAINER_NAME} -n ${ONAP_NAMESPACE} + sleep 10s +} + +function waitForPodTermination(){ + echo "Waiting for pod termination..." + + while [ "${HVVES_POD_NAME}" != "" ] ; do + HVVES_POD_NAME=$(kubectl -n ${ONAP_NAMESPACE} get pods --no-headers=true -o custom-columns=:metadata.name | grep hv-ves-collector) + sleep 1s + done +} + +function waitForPodRedeployment(){ + HVVES_POD_NAME="" + echo "Waiting for pod redeploy..." + + while [ "${HVVES_POD_NAME}" = "" ] ; do + HVVES_POD_NAME=$(kubectl -n ${ONAP_NAMESPACE} get pods --no-headers=true -o custom-columns=:metadata.name | grep hv-ves-collector) + sleep 1s + done +} + +function waitForHvVesRunning(){ + echo "Waiting for HV-VES running..." + POD_STATUS=""; + while [ "${POD_STATUS}" != "2/2" ]; do + POD_STATUS=$(kubectl get pod ${HVVES_POD_NAME} -n ${ONAP_NAMESPACE} --no-headers=true | awk '{print $2}') + sleep 1s + done +} + +function loadHvVesPodName(){ + HVVES_POD_NAME=$(kubectl -n ${ONAP_NAMESPACE} get pods --no-headers=true -o custom-columns=:metadata.name | grep hv-ves-collector) +} + + +if [[ $# -eq 0 ]]; then + usage +else + for arg in ${@} + do + case ${arg} in + reboot) + rebootEnvironment + ;; + help) + usage + ;; + *) + echo "Unknown action: ${arg}" >&2 + usage + ;; + esac + done +fi
\ No newline at end of file diff --git a/tools/performance/cloud/stale_outputs_checked b/tools/performance/cloud/stale_outputs_checked deleted file mode 100644 index e69de29b..00000000 --- a/tools/performance/cloud/stale_outputs_checked +++ /dev/null |