diff options
14 files changed, 569 insertions, 198 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..2f3470a4 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,15 @@ 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)) + .maximumExpectedValue(MAX_BUCKET_DURATION) .publishPercentileHistogram(true) .register(registry) private val totalLatency = Timer.builder(name(MESSAGES, LATENCY)) + .maximumExpectedValue(MAX_BUCKET_DURATION) .publishPercentileHistogram(true) .register(registry) @@ -65,12 +71,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 +99,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()) @@ -128,7 +136,7 @@ class MicrometerMetrics internal constructor( } companion object { - val INSTANCE = MicrometerMetrics() + val INSTANCE by lazy { MicrometerMetrics() } internal const val PREFIX = "hvves" internal const val MESSAGES = "messages" internal const val RECEIVED = "received" @@ -147,6 +155,9 @@ class MicrometerMetrics internal constructor( internal const val TIME = "time" internal const val LATENCY = "latency" internal const val PAYLOAD = "payload" + internal const val WITHOUT = "without" + internal const val ROUTING = "routing" + internal val MAX_BUCKET_DURATION = Duration.ofSeconds(300L) 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 f4445844..1b5f1f90 100755 --- a/tools/performance/cloud/cloud-based-performance-test.sh +++ b/tools/performance/cloud/cloud-based-performance-test.sh @@ -19,6 +19,9 @@ 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 @@ -31,15 +34,22 @@ GRAFANA_DASHBOARD_PROVIDERS=grafana-dashboards-providers ONAP_NAMESPACE=onap MAXIMUM_BACK_OFF_CHECK_ITERATIONS=30 CHECK_NUMBER=0 +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} @@ -50,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" @@ -68,6 +78,9 @@ function clean() { echo "Attempting to delete client certs secret" kubectl delete secret cert -n ${ONAP_NAMESPACE} + echo "Attempting to turn off SSL" + ./configure-consul.sh true + echo "Environment clean up finished!" } @@ -82,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}); @@ -101,6 +114,36 @@ function generate_certs() { ./gen-certs.sh } +function handle_backoffs() { + IMAGE_PULL_BACK_OFFS=$(kubectl get pods -l app=${PRODUCER_APPS_LABEL} -n ${ONAP_NAMESPACE} -o ${NAME_REASON_PATTERN} | grep -c "ImagePullBackOff \| ErrImagePull") + if [[ ${IMAGE_PULL_BACK_OFFS} -gt 0 ]]; then + CHECK_NUMBER=$((CHECK_NUMBER + 1)) + if [[ ${CHECK_NUMBER} -gt ${MAXIMUM_BACK_OFF_CHECK_ITERATIONS} ]]; then + echo "Error: Image pull problem" + exit 1 + fi + fi +} + +function handle_key_interrupt() { + trap SIGINT + echo "Script interrupted, attempt to delete producers" + echo "Wait with patience" + COMPLETED_PRODUCERS_SUM=$(($(kubectl delete pods -l app=${PRODUCER_APPS_LABEL} -n ${ONAP_NAMESPACE} | grep producer | wc -l) + COMPLETED_PRODUCERS_SUM)) + echo "Total number of completed producers: ${COMPLETED_PRODUCERS_SUM}" + exit 0 +} + +function print_test_setup_info() { + echo "Starting cloud based performance tests" + echo "________________________________________" + 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 "________________________________________" +} + function usage() { echo "" echo "Run cloud based HV-VES performance test" @@ -109,8 +152,10 @@ function usage() { echo " setup : set up ConfigMap and consumers" echo " start : create producers - start the performance test" echo " Optional parameters:" - 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:" @@ -118,6 +163,8 @@ function usage() { echo "./cloud-based-performance-test.sh setup" 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 @@ -125,6 +172,7 @@ function usage() { function setup_environment() { echo "Setting up environment" + echo "Copying certs to hv-ves pod" copy_certs_to_hvves @@ -132,8 +180,11 @@ function setup_environment() { kubectl create secret generic cert --from-file=./client.p12 --from-file=./client.pass -n ${ONAP_NAMESPACE} cd ${SCRIPT_DIRECTORY} + echo "Turning on SSL" + ./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 @@ -169,28 +220,50 @@ function setup_environment() { echo "Setting up environment finished!" } +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 + + trap "handle_key_interrupt" INT + + echo "Constant producer number keeper started working" + while :; do + PRODUCERS_TO_RECREATE=$((CONTAINERS_COUNT-$(kubectl get pods -l app=${PRODUCER_APPS_LABEL} -n ${ONAP_NAMESPACE} | grep -c "Running"))) + handle_backoffs + + set -e + for i in $(seq 1 ${PRODUCERS_TO_RECREATE}); + do + echo "Recreating ${i}/${PRODUCERS_TO_RECREATE} producer" + kubectl create -f producer-pod.yaml -n ${ONAP_NAMESPACE} + done + set +e + COMPLETED_PRODUCERS_SUM=$((COMPLETED_PRODUCERS_SUM + PRODUCERS_TO_RECREATE)) + echo "Attempting to clear completed producers" + kubectl delete pod --field-selector=status.phase==Succeeded -l app=${PRODUCER_APPS_LABEL} -n ${ONAP_NAMESPACE} + + [[ ${CHECK_NUMBER} -gt ${MAXIMUM_BACK_OFF_CHECK_ITERATIONS} ]] && break + sleep 1 + done + + trap SIGINT + exit 0 +} + function start_performance_test() { - echo "Starting cloud based performance tests" - echo "________________________________________" - echo "Test configuration:" - echo "Producer containers count: ${CONTAINERS_COUNT}" - echo "Properties file path: ${PROPERTIES_FILE}" - echo "________________________________________" + print_test_setup_info create_producers echo "Waiting for producers completion" while :; do COMPLETED_PRODUCERS=$(kubectl get pods -l app=${PRODUCER_APPS_LABEL} -n ${ONAP_NAMESPACE} | grep -c "Completed") - IMAGE_PULL_BACK_OFFS=$(kubectl get pods -l app=${PRODUCER_APPS_LABEL} -n ${ONAP_NAMESPACE} -o ${NAME_REASON_PATTERN} | grep -c "ImagePullBackOff \| ErrImagePull") - - if [[ ${IMAGE_PULL_BACK_OFFS} -gt 0 ]]; then - CHECK_NUMBER=$((CHECK_NUMBER + 1)) - if [[ ${CHECK_NUMBER} -gt ${MAXIMUM_BACK_OFF_CHECK_ITERATIONS} ]]; then - echo "Error: Image pull problem" - exit 1 - fi - fi + handle_backoffs [[ ${COMPLETED_PRODUCERS} -eq ${CONTAINERS_COUNT} || ${CHECK_NUMBER} -gt ${MAXIMUM_BACK_OFF_CHECK_ITERATIONS} ]] && break sleep 1 @@ -220,12 +293,18 @@ else shift 1 while [[ $(($#)) -gt 0 ]]; do case "${1}" in + --load) + LOAD_TEST=${2} + ;; --containers) CONTAINERS_COUNT=${2} ;; --properties-file) PROPERTIES_FILE=${2} ;; + --retention-time-minutes) + KAFKA_RETENTION_TIME_MINUTES=${2} + ;; *) echo "Unknown option: ${1}" usage @@ -233,7 +312,11 @@ else esac shift 2 done - start_performance_test + if [ ${LOAD_TEST} == "true" ] ; then + start_load_tests + else + start_performance_test + fi ;; clean) clean diff --git a/tools/performance/cloud/configure-consul.sh b/tools/performance/cloud/configure-consul.sh new file mode 100755 index 00000000..3ab298e8 --- /dev/null +++ b/tools/performance/cloud/configure-consul.sh @@ -0,0 +1,61 @@ +#!/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========================================================= + +SSL=$1 +if [ "$SSL" != "true" ] && [ "$SSL" != "false" ] ; then + SSL=true +fi +IP=$(kubectl config view -o jsonpath='{.clusters[].cluster.server}') +HTTPS="https://" +IP=${IP[@]//${HTTPS}} +IP=${IP[@]//:*} + +STATUS=$(curl -s --header "Content-Type: application/json" \ + --request PUT \ + --data '{"security.sslDisable": '${SSL}', +"logLevel": "INFO", +"server.listenPort": 6061, +"server.idleTimeoutSec": 300, +"cbs.requestIntervalSec": 5, +"streams_publishes": { + "perf3gpp": { + "type": "kafka", + "aaf_credentials": { + "username": "admin", + "password": "admin_secret" + }, + "kafka_info": { + "bootstrap_servers": "message-router-kafka:9092", + "topic_name": "HV_VES_PERF3GPP" + } + } + }, +"security.keys.trustStoreFile": "/etc/ves-hv/ssl/trust.p12", +"security.keys.keyStoreFile": "/etc/ves-hv/ssl/server.p12", +"security.keys.trustStorePasswordFile":"/etc/ves-hv/ssl/trust.pass", +"security.keys.keyStorePasswordFile": "/etc/ves-hv/ssl/server.pass"}' \ +${IP}:30270/v1/kv/dcae-hv-ves-collector?dc=dc1&token=) + +if [ "$STATUS" = "true" ] ; then + if [ "$SSL" = "true" ] ; then + echo "SSL turned off" + else + echo "SSL turned on" + fi +fi
\ No newline at end of file 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/performance-tests.yaml b/tools/performance/cloud/grafana/dashboards/performance-tests.yaml index b1a5d526..97ca71b1 100644 --- a/tools/performance/cloud/grafana/dashboards/performance-tests.yaml +++ b/tools/performance/cloud/grafana/dashboards/performance-tests.yaml @@ -42,8 +42,7 @@ data: "editable": true, "gnetId": null, "graphTooltip": 0, - "id": 2, - "iteration": 1571311867605, + "iteration": 1571408429976, "links": [], "panels": [ { @@ -55,8 +54,8 @@ data: "fill": 1, "fillGradient": 0, "gridPos": { - "h": 10, - "w": 7, + "h": 11, + "w": 12, "x": 0, "y": 0 }, @@ -87,14 +86,15 @@ data: "targets": [ { "expr": "irate(hvves_data_received_bytes_total[30s])", + "legendFormat": "Incoming data rate", "refId": "A" } ], "thresholds": [], - "timeFrom": "30m", + "timeFrom": "20m", "timeRegions": [], "timeShift": null, - "title": "HV-VES incomming data rate", + "title": "HV-VES incoming data rate", "tooltip": { "shared": true, "sort": 0, @@ -114,92 +114,7 @@ data: "label": null, "logBase": 1, "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": null, - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 10, - "w": 8, - "x": 7, - "y": 0 - }, - "id": 14, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "hvves_messages_received_total - hvves_messages_sent_total", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": "30m", - "timeRegions": [], - "timeShift": null, - "title": "HV-VES processing message queue", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, + "min": "0", "show": true }, { @@ -225,23 +140,23 @@ data: "fill": 0, "fillGradient": 0, "gridPos": { - "h": 10, - "w": 9, - "x": 15, + "h": 11, + "w": 12, + "x": 12, "y": 0 }, "id": 6, "legend": { - "alignAsTable": true, - "avg": true, - "current": true, + "alignAsTable": false, + "avg": false, + "current": false, "hideEmpty": true, "hideZero": true, "max": false, "min": false, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, @@ -260,40 +175,41 @@ data: "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.90, sum( rate(hv_kafka_consumer_travel_time_seconds_bucket[$time_range]) )by(le) )", + "expr": "rate(hv_kafka_consumer_travel_time_seconds_sum{instance=\"kafka-processing-consumer.onap:6063\"}[1h])/rate(hv_kafka_consumer_travel_time_seconds_count{instance=\"kafka-processing-consumer.onap:6063\"}[1h])", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "90th percentile of avg message travel time", + "legendFormat": "average", "refId": "A" }, { - "expr": "histogram_quantile(0.95, sum( rate(hv_kafka_consumer_travel_time_seconds_bucket[$time_range]) )by(le) )", + "expr": "histogram_quantile(0.90, sum( rate(hv_kafka_consumer_travel_time_seconds_bucket[1h]) )by(le) )", "format": "time_series", "intervalFactor": 1, - "legendFormat": "95th percentile of avg message travel time", + "legendFormat": "90th percentile", "refId": "B" }, { - "expr": "histogram_quantile(0.99, sum( rate(hv_kafka_consumer_travel_time_seconds_bucket[$time_range]) )by(le) )", + "expr": "histogram_quantile(0.95, sum( rate(hv_kafka_consumer_travel_time_seconds_bucket[1h]) )by(le) )", "format": "time_series", "intervalFactor": 1, - "legendFormat": "99th percentile of avg message travel time", + "legendFormat": "95th percentile", "refId": "C" }, { - "expr": "rate(hv_kafka_consumer_travel_time_seconds_sum[$time_range])/rate(hv_kafka_consumer_travel_time_seconds_count[$time_range])", + "expr": "histogram_quantile(0.99, sum( rate(hv_kafka_consumer_travel_time_seconds_bucket[1h]) )by(le) )", "format": "time_series", + "hide": false, "intervalFactor": 1, - "legendFormat": "avg message travel time", + "legendFormat": "95th percentile", "refId": "D" } ], "thresholds": [], - "timeFrom": "30m", + "timeFrom": "20m", "timeRegions": [], "timeShift": null, - "title": "Avg message travel time", + "title": "End to end message travel time", "tooltip": { "shared": true, "sort": 0, @@ -309,11 +225,11 @@ data: }, "yaxes": [ { - "format": "ns", + "format": "s", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -333,7 +249,6 @@ data: { "aliasColors": {}, "bars": false, - "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": null, @@ -343,9 +258,9 @@ data: "h": 11, "w": 12, "x": 0, - "y": 10 + "y": 11 }, - "id": 8, + "id": 14, "legend": { "avg": false, "current": false, @@ -357,7 +272,6 @@ data: }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null", "options": { "dataLinks": [] @@ -372,17 +286,16 @@ data: "steppedLine": false, "targets": [ { - "expr": "hv_kafka_consumer_offset_partition", - "format": "time_series", - "intervalFactor": 1, + "expr": "hvves_messages_received_total - hvves_messages_sent_total", + "legendFormat": "Processing message queue", "refId": "A" } ], "thresholds": [], - "timeFrom": "30m", + "timeFrom": "20m", "timeRegions": [], "timeShift": null, - "title": "Current offset on partitions", + "title": "HV-VES processing message queue", "tooltip": { "shared": true, "sort": 0, @@ -402,7 +315,7 @@ data: "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -432,7 +345,7 @@ data: "h": 11, "w": 12, "x": 12, - "y": 10 + "y": 11 }, "id": 10, "legend": { @@ -463,11 +376,12 @@ data: { "expr": "sum(hv_kafka_consumer_offset_partition)", "intervalFactor": 1, + "legendFormat": "Kafka consumer offset", "refId": "A" } ], "thresholds": [], - "timeFrom": "30m", + "timeFrom": "20m", "timeRegions": [], "timeShift": null, "title": "Total number of messages on topic", @@ -490,7 +404,7 @@ data: "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -517,9 +431,9 @@ data: { "allValue": null, "current": { - "tags": [], - "text": "15m", - "value": "15m" + "selected": false, + "text": "1h", + "value": "1h" }, "hide": 0, "includeAll": false, @@ -538,7 +452,7 @@ data: "value": "10m" }, { - "selected": true, + "selected": false, "text": "15m", "value": "15m" }, @@ -548,7 +462,7 @@ data: "value": "30m" }, { - "selected": false, + "selected": true, "text": "1h", "value": "1h" } diff --git a/tools/performance/cloud/grafana/dashboards/processing.yaml b/tools/performance/cloud/grafana/dashboards/processing.yaml index 959cfb9d..27abba13 100644 --- a/tools/performance/cloud/grafana/dashboards/processing.yaml +++ b/tools/performance/cloud/grafana/dashboards/processing.yaml @@ -300,15 +300,15 @@ data: "fillGradient": 0, "gridPos": { "h": 12, - "w": 15, + "w": 7, "x": 9, "y": 0 }, "id": 9, "legend": { - "alignAsTable": true, - "avg": true, - "current": true, + "alignAsTable": false, + "avg": false, + "current": false, "hideEmpty": true, "hideZero": true, "max": false, @@ -317,7 +317,7 @@ data: "show": true, "sideWidth": 350, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, @@ -336,30 +336,156 @@ data: "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.90, sum( rate(hvves_messages_processing_time_seconds_bucket[15m]) )by(le) )", + "expr": "rate(hvves_messages_processing_time_seconds_sum[1h])/rate(hvves_messages_processing_time_seconds_count[1h])", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "average", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.90, sum( rate(hvves_messages_processing_time_seconds_bucket[1h]) )by(le) )", "format": "time_series", "hide": false, "instant": false, "intervalFactor": 1, - "legendFormat": "90th percentile of processing time on pods", + "legendFormat": "90th percentile", "refId": "B" }, { - "expr": "rate(hvves_messages_processing_time_seconds_sum[15m])/rate(hvves_messages_processing_time_seconds_count[15m])", + "expr": "histogram_quantile(0.95, sum( rate(hvves_messages_processing_time_seconds_bucket[1h]) )by(le) )", + "legendFormat": "95th percentile", + "refId": "C" + }, + { + "expr": "histogram_quantile(0.99, sum( rate(hvves_messages_processing_time_seconds_bucket[1h]) )by(le) )", + "legendFormat": "99th percentile", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": "20m", + "timeRegions": [], + "timeShift": null, + "title": "Message processing time in HV-VES", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": null, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 12, + "w": 8, + "x": 16, + "y": 0 + }, + "id": 23, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "hideEmpty": true, + "hideZero": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "sideWidth": 350, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "rate(hvves_messages_processing_time_without_routing_seconds_sum[1h])/rate(hvves_messages_processing_time_without_routing_seconds_count[1h])", "format": "time_series", "hide": false, "instant": false, "interval": "", "intervalFactor": 1, - "legendFormat": "{{ kubernetes_pod_name }}", + "legendFormat": "average", "refId": "A" + }, + { + "expr": "histogram_quantile(0.90, sum( rate(hvves_messages_processing_time_without_routing_seconds_bucket[1h]) )by(le) )", + "format": "time_series", + "hide": false, + "instant": false, + "intervalFactor": 1, + "legendFormat": "90th percentile", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.95, sum( rate(hvves_messages_processing_time_without_routing_seconds_bucket[1h]) )by(le) )", + "legendFormat": "95th percentile", + "refId": "C" + }, + { + "expr": "histogram_quantile(0.99, sum( rate(hvves_messages_processing_time_without_routing_seconds_bucket[1h]) )by(le) )", + "legendFormat": "99th percentile", + "refId": "D" } ], "thresholds": [], - "timeFrom": null, + "timeFrom": "20m", "timeRegions": [], "timeShift": null, - "title": "Single message processing time average", + "title": "Message processing time in HV-VES without routing", "tooltip": { "shared": true, "sort": 0, @@ -411,7 +537,9 @@ data: "y": 3 }, "id": 8, + "interval": "", "legend": { + "alignAsTable": false, "avg": false, "current": false, "hideEmpty": true, @@ -444,15 +572,30 @@ data: "hide": false, "instant": false, "intervalFactor": 1, - "legendFormat": "{{ kubernetes_pod_name }}", + "legendFormat": "average", "refId": "A" + }, + { + "expr": "histogram_quantile(0.90, sum( rate(hvves_messages_latency_seconds_bucket[1h]) )by(le) )", + "legendFormat": "90th percentile", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.95, sum( rate(hvves_messages_latency_seconds_bucket[1h]) )by(le) )", + "legendFormat": "95th percentile", + "refId": "C" + }, + { + "expr": "histogram_quantile(0.99, sum( rate(hvves_messages_latency_seconds_bucket[1h]) )by(le) )", + "legendFormat": "99th precentile", + "refId": "D" } ], "thresholds": [], - "timeFrom": null, + "timeFrom": "20m", "timeRegions": [], "timeShift": null, - "title": "Latency [s]", + "title": "Message travel time from Producer to HV-VES output", "tooltip": { "shared": true, "sort": 0, @@ -469,11 +612,11 @@ data: "yaxes": [ { "decimals": 2, - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -619,15 +762,15 @@ data: "expr": "sum(process_cpu_usage) by (kubernetes_pod_name)*100", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{kubernetes_pod_name}}", + "legendFormat": "CPU usage", "refId": "A" } ], "thresholds": [], - "timeFrom": "20 m", + "timeFrom": "20m", "timeRegions": [], "timeShift": null, - "title": "System cpu usage", + "title": "HV-VES process cpu usage", "tooltip": { "shared": true, "sort": 0, @@ -714,13 +857,13 @@ data: "hide": false, "instant": false, "intervalFactor": 1, - "legendFormat": "jvm_memory_used_bytes", + "legendFormat": "Memory used", "refId": "A" }, { "expr": "sum(jvm_memory_committed_bytes)/1024/1024/1024", "instant": false, - "legendFormat": "jvm_memory_committed_bytes", + "legendFormat": "Memory commited", "refId": "B" } ], @@ -728,7 +871,7 @@ data: "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Memory usage per pod [GB]", + "title": "Memory usage of HV-VES", "tooltip": { "shared": true, "sort": 0, @@ -748,7 +891,7 @@ data: "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -977,7 +1120,7 @@ data: "list": [] }, "time": { - "from": "now-30m", + "from": "now-1h", "to": "now" }, "timepicker": { 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 |