From 8c20cd8fde34e11c429e32fac72e923737458b5e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Piotr=20R=C5=BCysko?= Date: Thu, 17 Aug 2023 11:18:59 +0200 Subject: [PATCH 1/6] Added support for micrometer in hermes-consumers (#1699) --- .../hermes/common/metric/ConsumerMetrics.java | 76 +++++++- .../common/metric/ConsumerSenderMetrics.java | 69 +++++++ .../tech/hermes/common/metric/Counters.java | 1 - .../hermes/common/metric/GaugeRegistrar.java | 3 +- .../tech/hermes/common/metric/Gauges.java | 1 + .../hermes/common/metric/HermesMetrics.java | 30 +-- .../hermes/common/metric/MaxRateMetrics.java | 63 ++++++ .../hermes/common/metric/MetricsFacade.java | 89 ++++++++- .../common/metric/OffsetCommitsMetrics.java | 52 +++++ .../common/metric/SubscriptionMetrics.java | 130 +++++++++++-- .../metric/SubscriptionTagsFactory.java | 17 ++ .../hermes/common/metric/TopicMetrics.java | 13 +- .../metric/UndeliveredMessagesMetrics.java | 3 +- .../hermes/common/metric/WorkloadMetrics.java | 176 +++++++++++++++++ .../consumers/config/CommonConfiguration.java | 4 +- .../config/ConsumerConfiguration.java | 11 +- .../config/ConsumerReceiverConfiguration.java | 6 +- .../config/ConsumerSenderConfiguration.java | 6 +- .../consumers/config/OAuthConfiguration.java | 4 +- .../config/SupervisorConfiguration.java | 38 ++-- .../consumers/consumer/BatchConsumer.java | 21 +- .../consumer/BatchConsumerMetrics.java | 102 ++++++++++ .../consumer/ConsumerMessageSender.java | 23 ++- .../ConsumerMessageSenderFactory.java | 16 +- .../consumers/consumer/SerialConsumer.java | 8 +- .../consumer/SubscriptionMetrics.java | 180 ------------------ .../batch/ByteBufferMessageBatchFactory.java | 9 +- .../consumer/batch/MessageBatchReceiver.java | 6 +- .../filtering/FilteredMessageHandler.java | 13 +- .../oauth/OAuthAccessTokensLoader.java | 12 +- .../consumer/offset/OffsetCommitter.java | 22 ++- .../consumer/offset/OffsetQueue.java | 4 +- .../rate/SerialConsumerRateLimiter.java | 8 +- .../OutputRateCalculatorFactory.java | 4 +- .../rate/maxrate/MaxRateCalculator.java | 10 +- .../rate/maxrate/MaxRateCalculatorJob.java | 4 +- .../rate/maxrate/MaxRateProviderFactory.java | 6 +- .../rate/maxrate/MaxRateSupervisor.java | 4 +- .../maxrate/NegotiatedMaxRateProvider.java | 19 +- .../consumer/receiver/ReceiverFactory.java | 4 +- .../receiver/ThrottlingMessageReceiver.java | 15 +- .../kafka/KafkaMessageReceiverFactory.java | 27 +-- .../KafkaSingleThreadedMessageReceiver.java | 15 +- .../consumer/result/DefaultErrorHandler.java | 50 ++++- .../result/DefaultSuccessHandler.java | 40 +++- .../http/HttpClientsWorkloadReporter.java | 86 ++++++--- .../consumers/queue/MonitoredMpscQueue.java | 13 +- .../consumers/supervisor/ConsumerFactory.java | 12 +- .../supervisor/ConsumersExecutorService.java | 6 +- .../NonblockingConsumersSupervisor.java | 7 +- .../monitor/ConsumersRuntimeMonitor.java | 12 +- .../process/ConsumerProcessSupervisor.java | 27 ++- .../supervisor/workload/BalancingJob.java | 34 +--- .../workload/WorkloadSupervisor.java | 4 +- .../weighted/AvgTargetWeightCalculator.java | 4 +- .../ScoringTargetWeightCalculator.java | 7 +- .../WeightedWorkBalancingListener.java | 4 +- .../weighted/WeightedWorkloadMetrics.java | 102 ---------- .../WeightedWorkloadMetricsReporter.java | 92 +++++++++ .../ZookeeperConsumerNodeLoadRegistry.java | 9 +- .../offset/OffsetCommitterTest.groovy | 17 +- .../NegotiatedMaxRateProviderTest.groovy | 15 +- ...kaSingleThreadedMessageReceiverTest.groovy | 9 +- .../result/DefaultErrorHandlerTest.groovy | 13 +- .../result/DefaultSuccessHandlerTest.groovy | 13 +- .../HttpClientConnectionMonitoringTest.groovy | 21 +- .../HttpClientsWorkloadReporterTest.groovy | 18 +- .../ConsumerProcessSupervisorTest.groovy | 21 +- .../ScoringTargetWeightCalculatorTest.groovy | 7 +- .../weighted/WeightedWorkBalancerTest.groovy | 9 +- .../WeightedWorkBalancingListenerTest.groovy | 14 +- .../consumer/ConsumerMessageSenderTest.java | 39 ++-- .../sender/http/JettyMessageSenderTest.java | 16 +- .../ConsumerTestRuntimeEnvironment.java | 15 +- ...ZookeeperConsumerNodeLoadRegistryTest.java | 6 +- .../metrics/DefaultHermesHistogram.java | 23 +++ .../tech/hermes/metrics/HermesHistogram.java | 20 +- .../metrics/TestMetricsFacadeFactory.java | 17 ++ 78 files changed, 1391 insertions(+), 735 deletions(-) create mode 100644 hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsumerSenderMetrics.java create mode 100644 hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MaxRateMetrics.java create mode 100644 hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/OffsetCommitsMetrics.java create mode 100644 hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionTagsFactory.java create mode 100644 hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/WorkloadMetrics.java create mode 100644 hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/BatchConsumerMetrics.java delete mode 100644 hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/SubscriptionMetrics.java delete mode 100644 hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkloadMetrics.java create mode 100644 hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkloadMetricsReporter.java create mode 100644 hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/DefaultHermesHistogram.java create mode 100644 hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/metrics/TestMetricsFacadeFactory.java diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsumerMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsumerMetrics.java index 7435c87c34..8ef5a5d086 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsumerMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsumerMetrics.java @@ -1,21 +1,91 @@ package pl.allegro.tech.hermes.common.metric; import io.micrometer.core.instrument.MeterRegistry; +import io.micrometer.core.instrument.Tags; +import pl.allegro.tech.hermes.api.Subscription; +import pl.allegro.tech.hermes.metrics.HermesCounter; import pl.allegro.tech.hermes.metrics.HermesTimer; +import pl.allegro.tech.hermes.metrics.counters.HermesCounters; + +import java.util.function.ToDoubleFunction; + +import static pl.allegro.tech.hermes.common.metric.Gauges.BATCH_BUFFER_AVAILABLE_BYTES; +import static pl.allegro.tech.hermes.common.metric.Gauges.BATCH_BUFFER_TOTAL_BYTES; +import static pl.allegro.tech.hermes.common.metric.Gauges.THREADS; +import static pl.allegro.tech.hermes.common.metric.SubscriptionTagsFactory.subscriptionTags; public class ConsumerMetrics { private final HermesMetrics hermesMetrics; private final MeterRegistry meterRegistry; + private final GaugeRegistrar gaugeRegistrar; public ConsumerMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { this.hermesMetrics = hermesMetrics; this.meterRegistry = meterRegistry; + this.gaugeRegistrar = new GaugeRegistrar(meterRegistry, hermesMetrics); + } + + public void registerQueueUtilizationGauge(T obj, String queueName, ToDoubleFunction f) { + gaugeRegistrar.registerGauge("queue." + queueName + ".utilization", obj, f); + } + + public HermesCounter queueFailuresCounter(String name) { + return HermesCounters.from( + meterRegistry.counter("queue." + name + ".failures"), + hermesMetrics.counter("queue." + name + ".failures") + ); + } + + public void registerConsumerProcessesThreadsGauge(T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge(THREADS, "consumer-processes.threads", obj, f); + } + + public void registerRunningConsumerProcessesGauge(T obj, ToDoubleFunction f) { + hermesMetrics.registerRunningConsumerProcessesCountGauge(() -> (int) f.applyAsDouble(obj)); + meterRegistry.gauge("consumer-processes.running", obj, f); + } + + public void registerDyingConsumerProcessesGauge(T obj, ToDoubleFunction f) { + hermesMetrics.registerDyingConsumerProcessesCountGauge(() -> (int) f.applyAsDouble(obj)); + meterRegistry.gauge("consumer-processes.dying", obj, f); } - public HermesTimer offsetCommitterDuration() { + public void registerBatchBufferTotalBytesGauge(T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge(BATCH_BUFFER_TOTAL_BYTES, "batch-buffer.total-bytes", obj, f); + } + + public void registerBatchBufferAvailableBytesGauge(T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge(BATCH_BUFFER_AVAILABLE_BYTES, "batch-buffer.available-bytes", obj, f); + } + + public HermesCounter oAuthSubscriptionTokenRequestCounter(Subscription subscription, String providerName) { + return HermesCounters.from( + meterRegistry.counter("oauth.token-requests", Tags.concat( + subscriptionTags(subscription.getQualifiedName()), + "provider", providerName + )), + hermesMetrics.oAuthSubscriptionTokenRequestMeter(subscription, providerName) + ); + } + + public HermesTimer oAuthProviderLatencyTimer(String providerName) { return HermesTimer.from( - meterRegistry.timer("offset-committer.duration"), - hermesMetrics.timer("offset-committer.duration") + meterRegistry.timer("oauth.token-request-latency", Tags.of("provider", providerName)), + hermesMetrics.oAuthProviderLatencyTimer(providerName) + ); + } + + public HermesCounter processedSignalsCounter(String name) { + return HermesCounters.from( + meterRegistry.counter("signals.processed", Tags.of("signal", name)), + hermesMetrics.counter("supervisor.signal." + name) + ); + } + + public HermesCounter droppedSignalsCounter(String name) { + return HermesCounters.from( + meterRegistry.counter("signals.dropped", Tags.of("signal", name)), + hermesMetrics.counter("supervisor.signal.dropped." + name) ); } } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsumerSenderMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsumerSenderMetrics.java new file mode 100644 index 0000000000..76bba6dde7 --- /dev/null +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/ConsumerSenderMetrics.java @@ -0,0 +1,69 @@ +package pl.allegro.tech.hermes.common.metric; + +import io.micrometer.core.instrument.MeterRegistry; + +import java.util.function.ToDoubleFunction; + +import static pl.allegro.tech.hermes.common.metric.Gauges.CONSUMER_SENDER_HTTP_1_BATCH_CLIENT_ACTIVE_CONNECTIONS; +import static pl.allegro.tech.hermes.common.metric.Gauges.CONSUMER_SENDER_HTTP_1_BATCH_CLIENT_IDLE_CONNECTIONS; +import static pl.allegro.tech.hermes.common.metric.Gauges.CONSUMER_SENDER_HTTP_1_SERIAL_CLIENT_ACTIVE_CONNECTIONS; +import static pl.allegro.tech.hermes.common.metric.Gauges.CONSUMER_SENDER_HTTP_1_SERIAL_CLIENT_IDLE_CONNECTIONS; +import static pl.allegro.tech.hermes.common.metric.Gauges.CONSUMER_SENDER_HTTP_2_SERIAL_CLIENT_CONNECTIONS; +import static pl.allegro.tech.hermes.common.metric.Gauges.CONSUMER_SENDER_HTTP_2_SERIAL_CLIENT_PENDING_CONNECTIONS; + +public class ConsumerSenderMetrics { + + private final HermesMetrics hermesMetrics; + private final MeterRegistry meterRegistry; + private final GaugeRegistrar gaugeRegistrar; + + ConsumerSenderMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { + this.hermesMetrics = hermesMetrics; + this.meterRegistry = meterRegistry; + this.gaugeRegistrar = new GaugeRegistrar(meterRegistry, hermesMetrics); + } + + public void registerRequestQueueSizeGauge(T obj, ToDoubleFunction f) { + hermesMetrics.registerConsumerSenderRequestQueueSize(() -> (int) f.applyAsDouble(obj)); + meterRegistry.gauge("http-clients.request-queue-size", obj, f); + } + + public void registerHttp1SerialClientRequestQueueSizeGauge(T obj, ToDoubleFunction f) { + hermesMetrics.registerConsumerSenderHttp1SerialClientRequestQueueSize(() -> (int) f.applyAsDouble(obj)); + meterRegistry.gauge("http-clients.serial.http1.request-queue-size", obj, f); + } + + public void registerHttp1BatchClientRequestQueueSizeGauge(T obj, ToDoubleFunction f) { + hermesMetrics.registerConsumerSenderHttp1BatchClientRequestQueueSize(() -> (int) f.applyAsDouble(obj)); + meterRegistry.gauge("http-clients.batch.http1.request-queue-size", obj, f); + } + + public void registerHttp2RequestQueueSizeGauge(T obj, ToDoubleFunction f) { + hermesMetrics.registerConsumerSenderHttp2RequestQueueSize(() -> (int) f.applyAsDouble(obj)); + meterRegistry.gauge("http-clients.serial.http2.request-queue-size", obj, f); + } + + public void registerHttp1SerialClientActiveConnectionsGauge(T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge(CONSUMER_SENDER_HTTP_1_SERIAL_CLIENT_ACTIVE_CONNECTIONS, obj, f); + } + + public void registerHttp1SerialClientIdleConnectionsGauge(T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge(CONSUMER_SENDER_HTTP_1_SERIAL_CLIENT_IDLE_CONNECTIONS, obj, f); + } + + public void registerHttp1BatchClientActiveConnectionsGauge(T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge(CONSUMER_SENDER_HTTP_1_BATCH_CLIENT_ACTIVE_CONNECTIONS, obj, f); + } + + public void registerHttp1BatchClientIdleConnectionsGauge(T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge(CONSUMER_SENDER_HTTP_1_BATCH_CLIENT_IDLE_CONNECTIONS, obj, f); + } + + public void registerHttp2SerialClientConnectionsGauge(T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge(CONSUMER_SENDER_HTTP_2_SERIAL_CLIENT_CONNECTIONS, obj, f); + } + + public void registerHttp2SerialClientPendingConnectionsGauge(T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge(CONSUMER_SENDER_HTTP_2_SERIAL_CLIENT_PENDING_CONNECTIONS, obj, f); + } +} diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Counters.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Counters.java index 910bdc3632..2b20ee366a 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Counters.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Counters.java @@ -9,7 +9,6 @@ public class Counters { public static final String PUBLISHED = "published." + GROUP + "." + TOPIC; public static final String DELIVERED = "delivered." + GROUP + "." + TOPIC + "." + SUBSCRIPTION; public static final String DISCARDED = "discarded." + GROUP + "." + TOPIC + "." + SUBSCRIPTION; - public static final String INFLIGHT = "inflight." + GROUP + "." + TOPIC + "." + SUBSCRIPTION; public static final String MAXRATE_RATE_HISTORY_FAILURES = "consumers-rate.max-rate.node." + GROUP + "." + TOPIC + "." + SUBSCRIPTION + ".history.failures"; public static final String MAXRATE_FETCH_FAILURES = diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/GaugeRegistrar.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/GaugeRegistrar.java index 10399b5718..3b1acf0486 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/GaugeRegistrar.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/GaugeRegistrar.java @@ -1,6 +1,7 @@ package pl.allegro.tech.hermes.common.metric; import io.micrometer.core.instrument.MeterRegistry; +import io.micrometer.core.instrument.Tag; import io.micrometer.core.instrument.Tags; import java.util.function.ToDoubleFunction; @@ -18,7 +19,7 @@ public void registerGauge(String graphiteName, String prometheusName, T stateObj, ToDoubleFunction f, - Tags tags) { + Iterable tags) { meterRegistry.gauge(prometheusName, tags, stateObj, f); hermesMetrics.registerGauge(graphiteName, () -> f.applyAsDouble(stateObj)); } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Gauges.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Gauges.java index fd4e5d0807..e7f2f1fa8e 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Gauges.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/Gauges.java @@ -42,4 +42,5 @@ public class Gauges { public static final String TASK_QUEUED = EXECUTORS + EXECUTOR_NAME + ".task-queue-size"; public static final String TASKS_QUEUE_UTILIZATION = EXECUTORS + EXECUTOR_NAME + ".task-queue-utilization"; public static final String TASKS_REJECTED_COUNT = EXECUTORS + EXECUTOR_NAME + "task-rejected"; + public static final String INFLIGHT = "inflight." + GROUP + "." + TOPIC + "." + SUBSCRIPTION + ".count"; } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/HermesMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/HermesMetrics.java index 09aea1a123..cb341c0253 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/HermesMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/HermesMetrics.java @@ -92,10 +92,6 @@ public void registerProducerInflightRequest(Gauge gauge) { metricRegistry.register(metricRegistryName(Gauges.INFLIGHT_REQUESTS), gauge); } - public void registerConsumersThreadGauge(Gauge gauge) { - metricRegistry.register(metricRegistryName(Gauges.THREADS), gauge); - } - public void registerMessageRepositorySizeGauge(Gauge gauge) { metricRegistry.register(metricRegistryName(Gauges.BACKUP_STORAGE_SIZE), gauge); } @@ -144,20 +140,12 @@ public void incrementThreadPoolTaskRejectedCount(String executorName) { executorCounter(Gauges.TASKS_REJECTED_COUNT, executorName).inc(); } - public void incrementInflightCounter(SubscriptionName subscription) { - getInflightCounter(subscription).inc(); - } - - public void decrementInflightCounter(SubscriptionName subscription) { - getInflightCounter(subscription).dec(); + public void registerInflightGauge(SubscriptionName subscription, Gauge gauge) { + registerGauge(metricRegistryName(Gauges.INFLIGHT, subscription.getTopicName(), subscription.getName()), gauge); } - public void decrementInflightCounter(SubscriptionName subscription, int size) { - getInflightCounter(subscription).dec(size); - } - - public void unregisterInflightCounter(SubscriptionName subscription) { - unregister(Counters.INFLIGHT, subscription); + public void unregisterInflightGauge(SubscriptionName subscription) { + unregister(Gauges.INFLIGHT, subscription); } public static void close(Timer.Context... timers) { @@ -168,10 +156,6 @@ public static void close(Timer.Context... timers) { } } - private Counter getInflightCounter(SubscriptionName subscription) { - return counter(Counters.INFLIGHT, subscription.getTopicName(), subscription.getName()); - } - public void registerGauge(String name, Gauge gauge) { String path = pathCompiler.compile(name); if (!metricRegistry.getGauges().containsKey(name)) { @@ -248,7 +232,7 @@ public void unregisterInflightTimeHistogram(SubscriptionName subscription) { unregister(INFLIGHT_TIME, subscription); } - public void registerConsumerHttpAnswer(SubscriptionName subscription, int statusCode) { + public void registerConsumerHttpAnswer(SubscriptionName subscription, int statusCode, long count) { PathContext pathContext = pathContext() .withGroup(escapeDots(subscription.getTopicName().getGroupName())) .withTopic(escapeDots(subscription.getTopicName().getName())) @@ -256,8 +240,8 @@ public void registerConsumerHttpAnswer(SubscriptionName subscription, int status .withHttpCode(statusCode) .withHttpCodeFamily(httpStatusFamily(statusCode)) .build(); - metricRegistry.meter(pathCompiler.compile(ERRORS_HTTP_BY_FAMILY, pathContext)).mark(); - metricRegistry.meter(pathCompiler.compile(ERRORS_HTTP_BY_CODE, pathContext)).mark(); + metricRegistry.meter(pathCompiler.compile(ERRORS_HTTP_BY_FAMILY, pathContext)).mark(count); + metricRegistry.meter(pathCompiler.compile(ERRORS_HTTP_BY_CODE, pathContext)).mark(count); } public void unregisterStatusMeters(SubscriptionName subscription) { diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MaxRateMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MaxRateMetrics.java new file mode 100644 index 0000000000..f1a759308d --- /dev/null +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MaxRateMetrics.java @@ -0,0 +1,63 @@ +package pl.allegro.tech.hermes.common.metric; + +import io.micrometer.core.instrument.MeterRegistry; +import pl.allegro.tech.hermes.api.SubscriptionName; +import pl.allegro.tech.hermes.metrics.HermesCounter; +import pl.allegro.tech.hermes.metrics.counters.HermesCounters; + +import java.util.List; +import java.util.function.ToDoubleFunction; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static pl.allegro.tech.hermes.common.metric.Counters.MAXRATE_FETCH_FAILURES; +import static pl.allegro.tech.hermes.common.metric.Counters.MAXRATE_RATE_HISTORY_FAILURES; +import static pl.allegro.tech.hermes.common.metric.Gauges.MAX_RATE_ACTUAL_RATE_VALUE; +import static pl.allegro.tech.hermes.common.metric.Gauges.MAX_RATE_CALCULATION_DURATION; +import static pl.allegro.tech.hermes.common.metric.Gauges.MAX_RATE_VALUE; +import static pl.allegro.tech.hermes.common.metric.Gauges.OUTPUT_RATE; +import static pl.allegro.tech.hermes.common.metric.SubscriptionTagsFactory.subscriptionTags; + +public class MaxRateMetrics { + + private final HermesMetrics hermesMetrics; + private final MeterRegistry meterRegistry; + + MaxRateMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { + this.hermesMetrics = hermesMetrics; + this.meterRegistry = meterRegistry; + } + + public void registerCalculationDurationInMillisGauge(T obj, ToDoubleFunction f) { + hermesMetrics.registerGauge(MAX_RATE_CALCULATION_DURATION, () -> (int) f.applyAsDouble(obj)); + meterRegistry.more().timeGauge("max-rate.calculation.duration", List.of(), obj, MILLISECONDS, f); + } + + public HermesCounter historyUpdateFailuresCounter(SubscriptionName subscription) { + return HermesCounters.from( + meterRegistry.counter("max-rate.history-update.failures", subscriptionTags(subscription)), + hermesMetrics.counter(MAXRATE_RATE_HISTORY_FAILURES, subscription.getTopicName(), subscription.getName()) + ); + } + + public HermesCounter fetchFailuresCounter(SubscriptionName subscription) { + return HermesCounters.from( + meterRegistry.counter("max-rate.fetch.failures", subscriptionTags(subscription)), + hermesMetrics.counter(MAXRATE_FETCH_FAILURES, subscription.getTopicName(), subscription.getName()) + ); + } + + public void registerCalculatedRateGauge(SubscriptionName subscription, T obj, ToDoubleFunction f) { + hermesMetrics.registerGauge(MAX_RATE_VALUE, subscription, () -> (int) f.applyAsDouble(obj)); + meterRegistry.gauge("max-rate.calculated-rate", subscriptionTags(subscription), obj, f); + } + + public void registerActualRateGauge(SubscriptionName subscription, T obj, ToDoubleFunction f) { + hermesMetrics.registerGauge(MAX_RATE_ACTUAL_RATE_VALUE, subscription, () -> (int) f.applyAsDouble(obj)); + meterRegistry.gauge("max-rate.actual-rate", subscriptionTags(subscription), obj, f); + } + + public void registerOutputRateGauge(SubscriptionName subscription, T obj, ToDoubleFunction f) { + hermesMetrics.registerGauge(OUTPUT_RATE, subscription, () -> (int) f.applyAsDouble(obj)); + meterRegistry.gauge("max-rate.output-rate", subscriptionTags(subscription), obj, f); + } +} diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricsFacade.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricsFacade.java index 364503137e..43ace4994b 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricsFacade.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/MetricsFacade.java @@ -1,9 +1,33 @@ package pl.allegro.tech.hermes.common.metric; +import io.micrometer.core.instrument.Meter; import io.micrometer.core.instrument.MeterRegistry; +import io.micrometer.core.instrument.search.Search; +import pl.allegro.tech.hermes.api.SubscriptionName; + +import java.util.Collection; + +import static pl.allegro.tech.hermes.common.metric.Counters.DELIVERED; +import static pl.allegro.tech.hermes.common.metric.Counters.DISCARDED; +import static pl.allegro.tech.hermes.common.metric.Counters.MAXRATE_FETCH_FAILURES; +import static pl.allegro.tech.hermes.common.metric.Counters.MAXRATE_RATE_HISTORY_FAILURES; +import static pl.allegro.tech.hermes.common.metric.Gauges.MAX_RATE_ACTUAL_RATE_VALUE; +import static pl.allegro.tech.hermes.common.metric.Gauges.MAX_RATE_VALUE; +import static pl.allegro.tech.hermes.common.metric.Gauges.OUTPUT_RATE; +import static pl.allegro.tech.hermes.common.metric.Meters.DISCARDED_SUBSCRIPTION_METER; +import static pl.allegro.tech.hermes.common.metric.Meters.FAILED_METER_SUBSCRIPTION; +import static pl.allegro.tech.hermes.common.metric.Meters.FILTERED_METER; +import static pl.allegro.tech.hermes.common.metric.Meters.SUBSCRIPTION_BATCH_METER; +import static pl.allegro.tech.hermes.common.metric.Meters.SUBSCRIPTION_METER; +import static pl.allegro.tech.hermes.common.metric.Meters.SUBSCRIPTION_THROUGHPUT_BYTES; +import static pl.allegro.tech.hermes.common.metric.SubscriptionTagsFactory.subscriptionTags; +import static pl.allegro.tech.hermes.common.metric.Timers.CONSUMER_IDLE_TIME; +import static pl.allegro.tech.hermes.common.metric.Timers.SUBSCRIPTION_LATENCY; public class MetricsFacade { + private final MeterRegistry meterRegistry; + private final HermesMetrics hermesMetrics; private final TopicMetrics topicMetrics; private final SubscriptionMetrics subscriptionMetrics; private final ConsumerMetrics consumerMetrics; @@ -14,9 +38,14 @@ public class MetricsFacade { private final SchemaClientMetrics schemaClientMetrics; private final UndeliveredMessagesMetrics undeliveredMessagesMetrics; private final DeserializationMetrics deserializationMetrics; + private final WorkloadMetrics workloadMetrics; + private final ConsumerSenderMetrics consumerSenderMetrics; + private final OffsetCommitsMetrics offsetCommitsMetrics; + private final MaxRateMetrics maxRateMetrics; - public MetricsFacade(MeterRegistry meterRegistry, - HermesMetrics hermesMetrics) { + public MetricsFacade(MeterRegistry meterRegistry, HermesMetrics hermesMetrics) { + this.meterRegistry = meterRegistry; + this.hermesMetrics = hermesMetrics; this.topicMetrics = new TopicMetrics(hermesMetrics, meterRegistry); this.subscriptionMetrics = new SubscriptionMetrics(hermesMetrics, meterRegistry); this.consumerMetrics = new ConsumerMetrics(hermesMetrics, meterRegistry); @@ -26,18 +55,22 @@ public MetricsFacade(MeterRegistry meterRegistry, this.executorMetrics = new ExecutorMetrics(hermesMetrics, meterRegistry); this.schemaClientMetrics = new SchemaClientMetrics(hermesMetrics, meterRegistry); this.undeliveredMessagesMetrics = new UndeliveredMessagesMetrics(hermesMetrics, meterRegistry); - this.deserializationMetrics = new DeserializationMetrics(hermesMetrics, meterRegistry); + this.deserializationMetrics = new DeserializationMetrics(hermesMetrics, meterRegistry); + this.workloadMetrics = new WorkloadMetrics(hermesMetrics, meterRegistry); + this.consumerSenderMetrics = new ConsumerSenderMetrics(hermesMetrics, meterRegistry); + this.offsetCommitsMetrics = new OffsetCommitsMetrics(hermesMetrics, meterRegistry); + this.maxRateMetrics = new MaxRateMetrics(hermesMetrics, meterRegistry); } public TopicMetrics topics() { return topicMetrics; } - public SubscriptionMetrics subscriptionMetrics() { + public SubscriptionMetrics subscriptions() { return subscriptionMetrics; } - public ConsumerMetrics consumers() { + public ConsumerMetrics consumer() { return consumerMetrics; } @@ -68,5 +101,49 @@ public UndeliveredMessagesMetrics undeliveredMessages() { public DeserializationMetrics deserialization() { return deserializationMetrics; } -} + public WorkloadMetrics workload() { + return workloadMetrics; + } + + public ConsumerSenderMetrics consumerSender() { + return consumerSenderMetrics; + } + + public OffsetCommitsMetrics offsetCommits() { + return offsetCommitsMetrics; + } + + public MaxRateMetrics maxRate() { + return maxRateMetrics; + } + + public void unregisterAllMetricsRelatedTo(SubscriptionName subscription) { + Collection meters = Search.in(meterRegistry) + .tags(subscriptionTags(subscription)) + .meters(); + for (Meter meter : meters) { + meterRegistry.remove(meter); + } + hermesMetrics.unregister(DISCARDED_SUBSCRIPTION_METER, subscription); + hermesMetrics.unregister(FAILED_METER_SUBSCRIPTION, subscription); + hermesMetrics.unregister(SUBSCRIPTION_BATCH_METER, subscription); + hermesMetrics.unregister(SUBSCRIPTION_METER, subscription); + hermesMetrics.unregister(DELIVERED, subscription); + hermesMetrics.unregister(DISCARDED, subscription); + hermesMetrics.unregisterInflightGauge(subscription); + hermesMetrics.unregisterInflightTimeHistogram(subscription); + hermesMetrics.unregisterConsumerErrorsTimeoutMeter(subscription); + hermesMetrics.unregisterConsumerErrorsOtherMeter(subscription); + hermesMetrics.unregisterStatusMeters(subscription); + hermesMetrics.unregister(OUTPUT_RATE, subscription); + hermesMetrics.unregister(MAX_RATE_ACTUAL_RATE_VALUE, subscription); + hermesMetrics.unregister(MAX_RATE_VALUE, subscription); + hermesMetrics.unregister(MAXRATE_FETCH_FAILURES, subscription); + hermesMetrics.unregister(MAXRATE_RATE_HISTORY_FAILURES, subscription); + hermesMetrics.unregister(CONSUMER_IDLE_TIME, subscription); + hermesMetrics.unregister(FILTERED_METER, subscription); + hermesMetrics.unregister(SUBSCRIPTION_LATENCY, subscription); + hermesMetrics.unregister(SUBSCRIPTION_THROUGHPUT_BYTES, subscription); + } +} diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/OffsetCommitsMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/OffsetCommitsMetrics.java new file mode 100644 index 0000000000..84527282a4 --- /dev/null +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/OffsetCommitsMetrics.java @@ -0,0 +1,52 @@ +package pl.allegro.tech.hermes.common.metric; + +import io.micrometer.core.instrument.MeterRegistry; +import pl.allegro.tech.hermes.metrics.HermesCounter; +import pl.allegro.tech.hermes.metrics.HermesTimer; +import pl.allegro.tech.hermes.metrics.counters.HermesCounters; + +public class OffsetCommitsMetrics { + + private final HermesMetrics hermesMetrics; + private final MeterRegistry meterRegistry; + + OffsetCommitsMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { + this.hermesMetrics = hermesMetrics; + this.meterRegistry = meterRegistry; + } + + public HermesCounter skippedCounter() { + return HermesCounters.from( + meterRegistry.counter("offset-commits.skipped"), + hermesMetrics.counter("offset-committer.skipped") + ); + } + + public HermesCounter obsoleteCounter() { + return HermesCounters.from( + meterRegistry.counter("offset-commits.obsolete"), + hermesMetrics.counter("offset-committer.obsolete") + ); + } + + public HermesCounter committedCounter() { + return HermesCounters.from( + meterRegistry.counter("offset-commits.committed"), + hermesMetrics.counter("offset-committer.committed") + ); + } + + public HermesTimer duration() { + return HermesTimer.from( + meterRegistry.timer("offset-commits.duration"), + hermesMetrics.timer("offset-committer.duration") + ); + } + + public HermesCounter failuresCounter() { + return HermesCounters.from( + meterRegistry.counter("offset-commits.failures"), + hermesMetrics.counter("offset-committer.failed") + ); + } +} diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionMetrics.java index 80c800a981..a4db4f5efb 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionMetrics.java @@ -1,13 +1,34 @@ package pl.allegro.tech.hermes.common.metric; import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.DistributionSummary; import io.micrometer.core.instrument.MeterRegistry; -import io.micrometer.core.instrument.Tag; +import io.micrometer.core.instrument.Tags; import pl.allegro.tech.hermes.api.SubscriptionName; +import pl.allegro.tech.hermes.metrics.DefaultHermesHistogram; +import pl.allegro.tech.hermes.metrics.HermesCounter; +import pl.allegro.tech.hermes.metrics.HermesHistogram; +import pl.allegro.tech.hermes.metrics.HermesTimer; +import pl.allegro.tech.hermes.metrics.counters.HermesCounters; -import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.function.ToDoubleFunction; +import static pl.allegro.tech.hermes.common.metric.Counters.DELIVERED; +import static pl.allegro.tech.hermes.common.metric.Counters.DISCARDED; +import static pl.allegro.tech.hermes.common.metric.Meters.DISCARDED_METER; +import static pl.allegro.tech.hermes.common.metric.Meters.DISCARDED_SUBSCRIPTION_METER; +import static pl.allegro.tech.hermes.common.metric.Meters.DISCARDED_TOPIC_METER; +import static pl.allegro.tech.hermes.common.metric.Meters.FAILED_METER_SUBSCRIPTION; +import static pl.allegro.tech.hermes.common.metric.Meters.FILTERED_METER; +import static pl.allegro.tech.hermes.common.metric.Meters.METER; +import static pl.allegro.tech.hermes.common.metric.Meters.SUBSCRIPTION_BATCH_METER; +import static pl.allegro.tech.hermes.common.metric.Meters.SUBSCRIPTION_METER; import static pl.allegro.tech.hermes.common.metric.Meters.SUBSCRIPTION_THROUGHPUT_BYTES; +import static pl.allegro.tech.hermes.common.metric.Meters.TOPIC_METER; +import static pl.allegro.tech.hermes.common.metric.SubscriptionTagsFactory.subscriptionTags; +import static pl.allegro.tech.hermes.common.metric.Timers.CONSUMER_IDLE_TIME; +import static pl.allegro.tech.hermes.common.metric.Timers.SUBSCRIPTION_LATENCY; public class SubscriptionMetrics { private final HermesMetrics hermesMetrics; @@ -18,27 +39,108 @@ public SubscriptionMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegis this.meterRegistry = meterRegistry; } - public SubscriptionHermesCounter subscriptionThroughputBytes(SubscriptionName subscription) { + public SubscriptionHermesCounter throughputInBytes(SubscriptionName subscription) { return SubscriptionHermesCounter.from( - micrometerCounter("subscription-throughput", subscription), + micrometerCounter("subscription-throughput-bytes", subscription), hermesMetrics.meter(SUBSCRIPTION_THROUGHPUT_BYTES, subscription.getTopicName(), subscription.getName()), SUBSCRIPTION_THROUGHPUT_BYTES, subscription); } - public void unregister(SubscriptionHermesCounter hermesCounter) { - meterRegistry.remove(hermesCounter.getMicrometerCounter()); - hermesMetrics.unregister(hermesCounter.getGraphiteName(), hermesCounter.getSubscription()); + public HermesCounter successes(SubscriptionName subscription) { + return size -> { + hermesMetrics.meter(METER).mark(size); + hermesMetrics.meter(TOPIC_METER, subscription.getTopicName()).mark(size); + hermesMetrics.meter(SUBSCRIPTION_METER, subscription.getTopicName(), subscription.getName()).mark(size); + hermesMetrics.counter(DELIVERED, subscription.getTopicName(), subscription.getName()).inc(size); + micrometerCounter("subscription.delivered", subscription).increment(size); + }; } - private Counter micrometerCounter(String metricName, SubscriptionName subscription) { - return meterRegistry.counter(metricName, subscriptionTags(subscription)); + public HermesCounter batchSuccesses(SubscriptionName subscription) { + return HermesCounters.from( + micrometerCounter("subscription.batches", subscription), + hermesMetrics.meter(SUBSCRIPTION_BATCH_METER, subscription.getTopicName(), subscription.getName()) + ); + } + + public HermesCounter discarded(SubscriptionName subscription) { + return size -> { + hermesMetrics.meter(DISCARDED_METER).mark(size); + hermesMetrics.meter(DISCARDED_TOPIC_METER, subscription.getTopicName()).mark(size); + hermesMetrics.meter(DISCARDED_SUBSCRIPTION_METER, subscription.getTopicName(), subscription.getName()).mark(size); + hermesMetrics.counter(DISCARDED, subscription.getTopicName(), subscription.getName()).inc(size); + micrometerCounter("subscription.discarded", subscription).increment(size); + }; + } + + public HermesTimer latency(SubscriptionName subscription) { + return HermesTimer.from( + meterRegistry.timer("subscription.latency", subscriptionTags(subscription)), + hermesMetrics.timer(SUBSCRIPTION_LATENCY, subscription.getTopicName(), subscription.getName()) + ); + } + + public void registerInflightGauge(SubscriptionName subscription, T obj, ToDoubleFunction f) { + hermesMetrics.registerInflightGauge(subscription, () -> (int) f.applyAsDouble(obj)); + meterRegistry.gauge("subscription.inflight", subscriptionTags(subscription), obj, f); + } + + public HermesTimer consumerIdleTimer(SubscriptionName subscription) { + return HermesTimer.from( + meterRegistry.timer("subscription.idle-duration", subscriptionTags(subscription)), + hermesMetrics.timer(CONSUMER_IDLE_TIME, subscription.getTopicName(), subscription.getName()) + ); + } + + public HermesCounter filteredOutCounter(SubscriptionName subscription) { + return HermesCounters.from( + micrometerCounter("subscription.filtered-out", subscription), + hermesMetrics.meter(FILTERED_METER, subscription.getTopicName(), subscription.getName()) + ); + } + + public HermesCounter httpAnswerCounter(SubscriptionName subscription, int statusCode) { + return size -> { + meterRegistry.counter( + "subscription.http-status-codes", + Tags.concat(subscriptionTags(subscription), "status_code", String.valueOf(statusCode)) + ).increment(size); + hermesMetrics.registerConsumerHttpAnswer(subscription, statusCode, size); + }; + } + + public HermesCounter timeoutsCounter(SubscriptionName subscription) { + return HermesCounters.from( + micrometerCounter("subscription.timeouts", subscription), + hermesMetrics.consumerErrorsTimeoutMeter(subscription) + ); } - private Iterable subscriptionTags(SubscriptionName subscriptionName) { - return List.of( - Tag.of("group", subscriptionName.getTopicName().getGroupName()), - Tag.of("topic", subscriptionName.getTopicName().getName()), - Tag.of("subscription", subscriptionName.getName()) + public HermesCounter otherErrorsCounter(SubscriptionName subscription) { + return HermesCounters.from( + micrometerCounter("subscription.other-errors", subscription), + hermesMetrics.consumerErrorsOtherMeter(subscription) ); } + + public HermesCounter failuresCounter(SubscriptionName subscription) { + return HermesCounters.from( + micrometerCounter("subscription.failures", subscription), + hermesMetrics.meter(FAILED_METER_SUBSCRIPTION, subscription.getTopicName(), subscription.getName()) + ); + } + + public HermesHistogram inflightTimeInMillisHistogram(SubscriptionName subscriptionName) { + return value -> { + DistributionSummary.builder("subscription.inflight-time-seconds") + .tags(subscriptionTags(subscriptionName)) + .register(meterRegistry) + .record(value / 1000d); + hermesMetrics.inflightTimeHistogram(subscriptionName).update(value); + }; + } + + private Counter micrometerCounter(String metricName, SubscriptionName subscription) { + return meterRegistry.counter(metricName, subscriptionTags(subscription)); + } } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionTagsFactory.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionTagsFactory.java new file mode 100644 index 0000000000..931e21dc5f --- /dev/null +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionTagsFactory.java @@ -0,0 +1,17 @@ +package pl.allegro.tech.hermes.common.metric; + +import io.micrometer.core.instrument.Tag; +import pl.allegro.tech.hermes.api.SubscriptionName; + +import java.util.Set; + +class SubscriptionTagsFactory { + + static Set subscriptionTags(SubscriptionName subscriptionName) { + return Set.of( + Tag.of("group", subscriptionName.getTopicName().getGroupName()), + Tag.of("topic", subscriptionName.getTopicName().getName()), + Tag.of("subscription", subscriptionName.getName()) + ); + } +} diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TopicMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TopicMetrics.java index d2dc0a511f..331ea9b35d 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TopicMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TopicMetrics.java @@ -7,6 +7,7 @@ import io.micrometer.core.instrument.Tags; import io.micrometer.core.instrument.Timer; import pl.allegro.tech.hermes.api.TopicName; +import pl.allegro.tech.hermes.metrics.DefaultHermesHistogram; import pl.allegro.tech.hermes.metrics.HermesCounter; import pl.allegro.tech.hermes.metrics.HermesHistogram; import pl.allegro.tech.hermes.metrics.HermesTimer; @@ -68,14 +69,14 @@ public HermesTimer ackLeaderBrokerLatency() { public MeterBackedHermesCounter topicThroughputBytes(TopicName topicName) { return HermesCounters.from( - micrometerCounter("topic-throughput", topicName), + micrometerCounter("topic-throughput-bytes", topicName), hermesMetrics.meter(TOPIC_THROUGHPUT_BYTES, topicName) ); } public MeterBackedHermesCounter topicGlobalThroughputBytes() { return HermesCounters.from( - meterRegistry.counter("topic-global-throughput"), + meterRegistry.counter("topic-global-throughput-bytes"), hermesMetrics.meter(THROUGHPUT_BYTES) ); } @@ -97,7 +98,7 @@ public HermesCounter topicGlobalRequestCounter() { public HermesCounter topicRequestCounter(TopicName topicName) { return HermesCounters.from( micrometerCounter("topic-requests", topicName), - hermesMetrics.meter(TOPIC_METER) + hermesMetrics.meter(TOPIC_METER, topicName) ); } @@ -111,7 +112,7 @@ public HermesCounter topicGlobalDelayedProcessingCounter() { public HermesCounter topicDelayedProcessingCounter(TopicName topicName) { return HermesCounters.from( micrometerCounter("topic-delayed-processing", topicName), - hermesMetrics.meter(TOPIC_DELAYED_PROCESSING) + hermesMetrics.meter(TOPIC_DELAYED_PROCESSING, topicName) ); } @@ -130,7 +131,7 @@ public HermesCounter topicHttpStatusCodeCounter(TopicName topicName, int statusC } public HermesHistogram topicGlobalMessageContentSizeHistogram() { - return HermesHistogram.of( + return DefaultHermesHistogram.of( DistributionSummary.builder("topic-global-message-size-bytes") .register(meterRegistry), hermesMetrics.messageContentSizeHistogram() @@ -138,7 +139,7 @@ public HermesHistogram topicGlobalMessageContentSizeHistogram() { } public HermesHistogram topicMessageContentSizeHistogram(TopicName topicName) { - return HermesHistogram.of( + return DefaultHermesHistogram.of( DistributionSummary.builder("topic-message-size-bytes") .tags(topicTags(topicName)) .register(meterRegistry), diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/UndeliveredMessagesMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/UndeliveredMessagesMetrics.java index 8885d4e953..ca4edaa24e 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/UndeliveredMessagesMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/UndeliveredMessagesMetrics.java @@ -2,6 +2,7 @@ import io.micrometer.core.instrument.DistributionSummary; import io.micrometer.core.instrument.MeterRegistry; +import pl.allegro.tech.hermes.metrics.DefaultHermesHistogram; import pl.allegro.tech.hermes.metrics.HermesCounter; import pl.allegro.tech.hermes.metrics.HermesHistogram; import pl.allegro.tech.hermes.metrics.counters.HermesCounters; @@ -26,7 +27,7 @@ public HermesCounter undeliveredMessagesCounter() { } public HermesHistogram undeliveredMessagesSizeHistogram() { - return HermesHistogram.of( + return DefaultHermesHistogram.of( DistributionSummary.builder("undelivered-messages.persisted.message-size.bytes") .register(meterRegistry), hermesMetrics.histogram(PERSISTED_UNDELIVERED_MESSAGE_SIZE) diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/WorkloadMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/WorkloadMetrics.java new file mode 100644 index 0000000000..7c2e840d0f --- /dev/null +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/WorkloadMetrics.java @@ -0,0 +1,176 @@ +package pl.allegro.tech.hermes.common.metric; + +import io.micrometer.core.instrument.Gauge; +import io.micrometer.core.instrument.MeterRegistry; +import io.micrometer.core.instrument.Tags; +import io.micrometer.core.instrument.search.Search; +import pl.allegro.tech.hermes.metrics.HermesTimer; + +import java.util.Collection; +import java.util.Set; +import java.util.function.ToDoubleFunction; + +public class WorkloadMetrics { + + private static final String CONSUMER_ID_TAG = "consumer-id"; + private static final String KAFKA_CLUSTER_TAG = "kafka-cluster"; + private static final String METRICS_PREFIX = "consumer-workload.weighted."; + private static final String CONSUMER_ID_PLACEHOLDER = "$consumerId"; + private static final String CURRENT_SCORE = METRICS_PREFIX + CONSUMER_ID_PLACEHOLDER + ".current-score"; + private static final String PROPOSED_SCORE = METRICS_PREFIX + CONSUMER_ID_PLACEHOLDER + ".proposed-score"; + private static final String SCORING_ERROR = METRICS_PREFIX + CONSUMER_ID_PLACEHOLDER + ".error"; + private static final String CURRENT_WEIGHT_OPS = METRICS_PREFIX + CONSUMER_ID_PLACEHOLDER + ".current-weight.ops"; + private static final String PROPOSED_WEIGHT_OPS = METRICS_PREFIX + CONSUMER_ID_PLACEHOLDER + ".proposed-weight.ops"; + + private final HermesMetrics hermesMetrics; + private final MeterRegistry meterRegistry; + private final GaugeRegistrar gaugeRegistrar; + + WorkloadMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { + this.hermesMetrics = hermesMetrics; + this.meterRegistry = meterRegistry; + this.gaugeRegistrar = new GaugeRegistrar(meterRegistry, hermesMetrics); + } + + public void registerAllAssignmentsGauge(T obj, String kafkaCluster, ToDoubleFunction f) { + gaugeRegistrar.registerGauge( + "consumers-workload." + kafkaCluster + ".all-assignments", + "workload.all-assignments", + obj, + f, + Tags.of(KAFKA_CLUSTER_TAG, kafkaCluster) + ); + } + + public void registerMissingResourcesGauge(T obj, String kafkaCluster, ToDoubleFunction f) { + gaugeRegistrar.registerGauge( + "consumers-workload." + kafkaCluster + ".missing-resources", + "workload.missing-resources", + obj, + f, + Tags.of(KAFKA_CLUSTER_TAG, kafkaCluster) + ); + } + + public void registerDeletedAssignmentsGauge(T obj, String kafkaCluster, ToDoubleFunction f) { + gaugeRegistrar.registerGauge( + "consumers-workload." + kafkaCluster + ".deleted-assignments", + "workload.deleted-assignments", + obj, + f, + Tags.of(KAFKA_CLUSTER_TAG, kafkaCluster) + ); + } + + public void registerCreatedAssignmentsGauge(T obj, String kafkaCluster, ToDoubleFunction f) { + gaugeRegistrar.registerGauge( + "consumers-workload." + kafkaCluster + ".created-assignments", + "workload.created-assignments", + obj, + f, + Tags.of(KAFKA_CLUSTER_TAG, kafkaCluster) + ); + } + + public HermesTimer rebalanceDurationTimer(String kafkaCluster) { + return HermesTimer.from( + meterRegistry.timer("workload.rebalance-duration", Tags.of(KAFKA_CLUSTER_TAG, kafkaCluster)), + hermesMetrics.consumersWorkloadRebalanceDurationTimer(kafkaCluster) + ); + } + + public void registerRunningSubscriptionsGauge(T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge("consumers-workload.monitor.running", "workload.subscriptions.running", obj, f); + } + + public void registerAssignedSubscriptionsGauge(T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge("consumers-workload.monitor.assigned", "workload.subscriptions.assigned", obj, f); + } + + public void registerMissingSubscriptionsGauge(T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge("consumers-workload.monitor.missing", "workload.subscriptions.missing", obj, f); + } + + public void registerOversubscribedGauge(T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge("consumers-workload.monitor.oversubscribed", "workload.subscriptions.oversubscribed", obj, f); + } + + public void registerOperationsPerSecondGauge(T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge("consumer-workload.weighted.load.ops", "workload.weighted.ops", obj, f); + } + + public void registerCpuUtilizationGauge(T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge("consumer-workload.weighted.load.cpu-utilization", "workload.weighted.cpu-utilization", obj, f); + } + + public void registerCurrentScoreGauge(String consumerId, T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge( + buildFullGraphiteMetricPath(CURRENT_SCORE, consumerId), + "workload.weighted.current-score", + obj, + f, + Tags.of(CONSUMER_ID_TAG, consumerId) + ); + } + + public void registerProposedErrorGauge(String consumerId, T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge( + buildFullGraphiteMetricPath(PROPOSED_SCORE, consumerId), + "workload.weighted.proposed-error", + obj, + f, + Tags.of(CONSUMER_ID_TAG, consumerId) + ); + } + + public void registerScoringErrorGauge(String consumerId, T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge( + buildFullGraphiteMetricPath(SCORING_ERROR, consumerId), + "workload.weighted.scoring-error", + obj, + f, + Tags.of(CONSUMER_ID_TAG, consumerId) + ); + } + + public void registerCurrentWeightGauge(String consumerId, T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge( + buildFullGraphiteMetricPath(CURRENT_WEIGHT_OPS, consumerId), + "workload.weighted.current-weight.ops", + obj, + f, + Tags.of(CONSUMER_ID_TAG, consumerId) + ); + } + + public void registerProposedWeightGauge(String consumerId, T obj, ToDoubleFunction f) { + gaugeRegistrar.registerGauge( + buildFullGraphiteMetricPath(PROPOSED_WEIGHT_OPS, consumerId), + "workload.weighted.proposed-weight.ops", + obj, + f, + Tags.of(CONSUMER_ID_TAG, consumerId) + ); + } + + public void unregisterAllWorkloadWeightedGaugesForConsumerIds(Set consumerIds) { + Collection gauges = Search.in(meterRegistry) + .tag(CONSUMER_ID_TAG, consumerIds::contains) + .name(s -> s.startsWith("workload.weighted")) + .gauges(); + for (Gauge gauge : gauges) { + meterRegistry.remove(gauge); + } + for (String consumerId : consumerIds) { + hermesMetrics.unregister(buildFullGraphiteMetricPath(CURRENT_SCORE, consumerId)); + hermesMetrics.unregister(buildFullGraphiteMetricPath(PROPOSED_SCORE, consumerId)); + hermesMetrics.unregister(buildFullGraphiteMetricPath(SCORING_ERROR, consumerId)); + hermesMetrics.unregister(buildFullGraphiteMetricPath(CURRENT_WEIGHT_OPS, consumerId)); + hermesMetrics.unregister(buildFullGraphiteMetricPath(PROPOSED_WEIGHT_OPS, consumerId)); + } + } + + private String buildFullGraphiteMetricPath(String metric, String consumerId) { + return metric.replace(CONSUMER_ID_PLACEHOLDER, HermesMetrics.escapeDots(consumerId)); + } +} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java index cf79dd3703..6fdd451dad 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java @@ -244,8 +244,8 @@ public HermesMetrics hermesMetrics(MetricRegistry metricRegistry, } @Bean - public MetricsFacade micrometerHermesMetrics(MeterRegistry metricRegistry, HermesMetrics hermesMetrics) { - return new MetricsFacade(metricRegistry, hermesMetrics); + public MetricsFacade metricsFacade(MeterRegistry meterRegistry, HermesMetrics hermesMetrics) { + return new MetricsFacade(meterRegistry, hermesMetrics); } @Bean diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerConfiguration.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerConfiguration.java index 27c6735bb9..5d912ef791 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerConfiguration.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerConfiguration.java @@ -5,7 +5,7 @@ import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; import pl.allegro.tech.hermes.common.message.undelivered.UndeliveredMessageLog; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.common.metric.executor.InstrumentedExecutorServiceFactory; import pl.allegro.tech.hermes.consumers.consumer.ConsumerAuthorizationHandler; import pl.allegro.tech.hermes.consumers.consumer.ConsumerMessageSenderFactory; @@ -26,7 +26,6 @@ import pl.allegro.tech.hermes.consumers.consumer.rate.maxrate.MaxRateRegistry; import pl.allegro.tech.hermes.consumers.consumer.rate.maxrate.MaxRateSupervisor; import pl.allegro.tech.hermes.consumers.consumer.sender.MessageSenderFactory; -import pl.allegro.tech.hermes.consumers.consumer.sender.MessageSendingResult; import pl.allegro.tech.hermes.consumers.consumer.sender.timeout.FutureAsyncTimeout; import pl.allegro.tech.hermes.consumers.registry.ConsumerNodesRegistry; import pl.allegro.tech.hermes.consumers.subscription.cache.SubscriptionsCache; @@ -98,7 +97,7 @@ public MaxRateSupervisor maxRateSupervisor(MaxRateProperties maxRateProperties, MaxRateRegistry maxRateRegistry, ConsumerNodesRegistry consumerNodesRegistry, SubscriptionsCache subscriptionsCache, - HermesMetrics metrics, + MetricsFacade metrics, Clock clock) { return new MaxRateSupervisor( maxRateProperties, @@ -112,7 +111,7 @@ public MaxRateSupervisor maxRateSupervisor(MaxRateProperties maxRateProperties, } @Bean - public OffsetQueue offsetQueue(HermesMetrics metrics, + public OffsetQueue offsetQueue(MetricsFacade metrics, CommitOffsetProperties commitOffsetProperties) { return new OffsetQueue(metrics, commitOffsetProperties.getQueuesSize()); } @@ -142,10 +141,10 @@ public OutputRateCalculatorFactory outputRateCalculatorFactory(RateProperties ra } @Bean - public MessageBatchFactory messageBatchFactory(HermesMetrics hermesMetrics, + public MessageBatchFactory messageBatchFactory(MetricsFacade metrics, Clock clock, BatchProperties batchProperties) { - return new ByteBufferMessageBatchFactory(batchProperties.getPoolableSize(), batchProperties.getMaxPoolSize(), clock, hermesMetrics); + return new ByteBufferMessageBatchFactory(batchProperties.getPoolableSize(), batchProperties.getMaxPoolSize(), clock, metrics); } @Bean diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerReceiverConfiguration.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerReceiverConfiguration.java index 2dabfe1c40..3915bdd64a 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerReceiverConfiguration.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerReceiverConfiguration.java @@ -5,7 +5,7 @@ import org.springframework.context.annotation.Configuration; import pl.allegro.tech.hermes.common.kafka.KafkaNamesMapper; import pl.allegro.tech.hermes.common.message.wrapper.CompositeMessageContentWrapper; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.offset.ConsumerPartitionAssignmentState; import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue; import pl.allegro.tech.hermes.consumers.consumer.receiver.ReceiverFactory; @@ -37,7 +37,7 @@ public ReceiverFactory kafkaMessageReceiverFactory(CommonConsumerProperties comm KafkaConsumerProperties kafkaConsumerProperties, KafkaClustersProperties kafkaClustersProperties, KafkaConsumerRecordToMessageConverterFactory messageConverterFactory, - HermesMetrics hermesMetrics, + MetricsFacade metrics, OffsetQueue offsetQueue, KafkaNamesMapper kafkaNamesMapper, FilterChainFactory filterChainFactory, @@ -53,7 +53,7 @@ public ReceiverFactory kafkaMessageReceiverFactory(CommonConsumerProperties comm kafkaConsumerProperties, kafkaProperties, messageConverterFactory, - hermesMetrics, + metrics, offsetQueue, kafkaNamesMapper, filterChainFactory, diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerSenderConfiguration.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerSenderConfiguration.java index 51057e3cab..1958e4bff0 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerSenderConfiguration.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/ConsumerSenderConfiguration.java @@ -18,7 +18,7 @@ import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Conditional; import org.springframework.context.annotation.Configuration; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.common.metric.executor.InstrumentedExecutorServiceFactory; import pl.allegro.tech.hermes.common.ssl.SslContextFactory; import pl.allegro.tech.hermes.consumers.consumer.interpolation.UriInterpolator; @@ -133,7 +133,7 @@ public MessageBatchSenderFactory httpMessageBatchSenderFactory(SendingResultHand } @Bean(initMethod = "start") - public HttpClientsWorkloadReporter httpClientsWorkloadReporter(HermesMetrics metrics, + public HttpClientsWorkloadReporter httpClientsWorkloadReporter(MetricsFacade metrics, @Named("http1-serial-client") HttpClient http1SerialClient, @Named("http1-batch-client") HttpClient http1BatchClient, Http2ClientHolder http2ClientHolder, @@ -261,7 +261,7 @@ public EndpointAddressResolver interpolatingEndpointAddressResolver(UriInterpola @Bean public FutureAsyncTimeout futureAsyncTimeoutFactory(InstrumentedExecutorServiceFactory executorFactory, - SenderAsyncTimeoutProperties senderAsyncTimeoutProperties) { + SenderAsyncTimeoutProperties senderAsyncTimeoutProperties) { ScheduledExecutorService timeoutExecutorService = executorFactory.getScheduledExecutorService( "async-timeout", senderAsyncTimeoutProperties.getThreadPoolSize(), diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/OAuthConfiguration.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/OAuthConfiguration.java index a31f027fe1..82c55cea79 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/OAuthConfiguration.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/OAuthConfiguration.java @@ -8,7 +8,7 @@ import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.ConsumerAuthorizationHandler; import pl.allegro.tech.hermes.consumers.consumer.oauth.OAuthAccessTokens; import pl.allegro.tech.hermes.consumers.consumer.oauth.OAuthAccessTokensLoader; @@ -61,7 +61,7 @@ public OAuthSubscriptionHandlerFactory oAuthSubscriptionHandlerFactory(Subscript public OAuthAccessTokensLoader oAuthAccessTokensLoader(SubscriptionRepository subscriptionRepository, OAuthProviderRepository oAuthProviderRepository, OAuthClient oAuthClient, - HermesMetrics metrics) { + MetricsFacade metrics) { return new OAuthAccessTokensLoader(subscriptionRepository, oAuthProviderRepository, oAuthClient, metrics); } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/SupervisorConfiguration.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/SupervisorConfiguration.java index b5b4695ce7..055f3d34d3 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/SupervisorConfiguration.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/SupervisorConfiguration.java @@ -10,7 +10,6 @@ import pl.allegro.tech.hermes.common.concurrent.ExecutorServiceFactory; import pl.allegro.tech.hermes.common.kafka.offset.SubscriptionOffsetChangeIndicator; import pl.allegro.tech.hermes.common.message.wrapper.CompositeMessageContentWrapper; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.config.WorkloadProperties.TargetWeightCalculationStrategy.UnknownTargetWeightCalculationStrategyException; import pl.allegro.tech.hermes.consumers.config.WorkloadProperties.WeightedWorkBalancingProperties; @@ -54,7 +53,7 @@ import pl.allegro.tech.hermes.consumers.supervisor.workload.weighted.TargetWeightCalculator; import pl.allegro.tech.hermes.consumers.supervisor.workload.weighted.WeightedWorkBalancer; import pl.allegro.tech.hermes.consumers.supervisor.workload.weighted.WeightedWorkBalancingListener; -import pl.allegro.tech.hermes.consumers.supervisor.workload.weighted.WeightedWorkloadMetrics; +import pl.allegro.tech.hermes.consumers.supervisor.workload.weighted.WeightedWorkloadMetricsReporter; import pl.allegro.tech.hermes.consumers.supervisor.workload.weighted.ZookeeperConsumerNodeLoadRegistry; import pl.allegro.tech.hermes.consumers.supervisor.workload.weighted.ZookeeperSubscriptionProfileRegistry; import pl.allegro.tech.hermes.domain.notifications.InternalNotificationsBus; @@ -91,7 +90,7 @@ public WorkloadSupervisor workloadSupervisor(InternalNotificationsBus notificati SubscriptionsCache subscriptionsCache, ConsumersSupervisor supervisor, ZookeeperAdminCache adminCache, - HermesMetrics metrics, + MetricsFacade metrics, WorkloadProperties workloadProperties, KafkaClustersProperties kafkaClustersProperties, WorkloadConstraintsRepository workloadConstraintsRepository, @@ -153,7 +152,7 @@ public ConsumerNodeLoadRegistry consumerNodeLoadRegistry(CuratorFramework curato DatacenterNameProvider datacenterNameProvider, ExecutorServiceFactory executorServiceFactory, Clock clock, - HermesMetrics metrics) { + MetricsFacade metrics) { switch (workloadProperties.getWorkBalancingStrategy()) { case SELECTIVE: return new NoOpConsumerNodeLoadRegistry(); @@ -179,15 +178,15 @@ public ConsumerNodeLoadRegistry consumerNodeLoadRegistry(CuratorFramework curato @Bean public TargetWeightCalculator targetWeightCalculator(WorkloadProperties workloadProperties, - WeightedWorkloadMetrics weightedWorkloadMetrics, + WeightedWorkloadMetricsReporter metricsReporter, Clock clock) { WeightedWorkBalancingProperties weightedWorkBalancing = workloadProperties.getWeightedWorkBalancing(); switch (weightedWorkBalancing.getTargetWeightCalculationStrategy()) { case AVG: - return new AvgTargetWeightCalculator(weightedWorkloadMetrics); + return new AvgTargetWeightCalculator(metricsReporter); case SCORING: return new ScoringTargetWeightCalculator( - weightedWorkloadMetrics, + metricsReporter, clock, weightedWorkBalancing.getWeightWindowSize(), weightedWorkBalancing.getScoringGain() @@ -202,7 +201,7 @@ public BalancingListener balancingListener(ConsumerNodeLoadRegistry consumerNode SubscriptionProfileRegistry subscriptionProfileRegistry, WorkloadProperties workloadProperties, CurrentLoadProvider currentLoadProvider, - WeightedWorkloadMetrics weightedWorkloadMetrics, + WeightedWorkloadMetricsReporter weightedWorkloadMetrics, Clock clock) { switch (workloadProperties.getWorkBalancingStrategy()) { case SELECTIVE: @@ -227,8 +226,8 @@ public CurrentLoadProvider currentLoadProvider() { } @Bean - public WeightedWorkloadMetrics weightedWorkloadMetrics(HermesMetrics hermesMetrics) { - return new WeightedWorkloadMetrics(hermesMetrics); + public WeightedWorkloadMetricsReporter weightedWorkloadMetrics(MetricsFacade metrics) { + return new WeightedWorkloadMetricsReporter(metrics); } @Bean @@ -260,8 +259,7 @@ public Retransmitter retransmitter(SubscriptionOffsetChangeIndicator subscriptio @Bean public ConsumerFactory consumerFactory(ReceiverFactory messageReceiverFactory, - HermesMetrics hermesMetrics, - MetricsFacade metricsFacade, + MetricsFacade metrics, CommonConsumerProperties commonConsumerProperties, ConsumerRateLimitSupervisor consumerRateLimitSupervisor, OutputRateCalculatorFactory outputRateCalculatorFactory, @@ -278,8 +276,7 @@ public ConsumerFactory consumerFactory(ReceiverFactory messageReceiverFactory, SubscriptionLoadRecordersRegistry subscriptionLoadRecordersRegistry) { return new ConsumerFactory( messageReceiverFactory, - hermesMetrics, - metricsFacade, + metrics, commonConsumerProperties, consumerRateLimitSupervisor, outputRateCalculatorFactory, @@ -299,8 +296,8 @@ public ConsumerFactory consumerFactory(ReceiverFactory messageReceiverFactory, @Bean public ConsumersExecutorService consumersExecutorService(CommonConsumerProperties commonConsumerProperties, - HermesMetrics hermesMetrics) { - return new ConsumersExecutorService(commonConsumerProperties.getThreadPoolSize(), hermesMetrics); + MetricsFacade metrics) { + return new ConsumersExecutorService(commonConsumerProperties.getThreadPoolSize(), metrics); } @Bean @@ -312,26 +309,25 @@ public ConsumersSupervisor nonblockingConsumersSupervisor(CommonConsumerProperti Retransmitter retransmitter, UndeliveredMessageLogPersister undeliveredMessageLogPersister, SubscriptionRepository subscriptionRepository, - HermesMetrics metrics, - MetricsFacade metricsFacade, + MetricsFacade metrics, ConsumerMonitor monitor, Clock clock, CommitOffsetProperties commitOffsetProperties) { return new NonblockingConsumersSupervisor(commonConsumerProperties, executor, consumerFactory, offsetQueue, consumerPartitionAssignmentState, retransmitter, undeliveredMessageLogPersister, - subscriptionRepository, metrics, metricsFacade, monitor, clock, commitOffsetProperties.getPeriod()); + subscriptionRepository, metrics, monitor, clock, commitOffsetProperties.getPeriod()); } @Bean(initMethod = "start", destroyMethod = "shutdown") public ConsumersRuntimeMonitor consumersRuntimeMonitor(ConsumersSupervisor consumerSupervisor, WorkloadSupervisor workloadSupervisor, - HermesMetrics hermesMetrics, + MetricsFacade metrics, SubscriptionsCache subscriptionsCache, WorkloadProperties workloadProperties) { return new ConsumersRuntimeMonitor( consumerSupervisor, workloadSupervisor, - hermesMetrics, + metrics, subscriptionsCache, workloadProperties.getMonitorScanInterval() ); diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/BatchConsumer.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/BatchConsumer.java index 759fb853df..737ae0f099 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/BatchConsumer.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/BatchConsumer.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.consumers.consumer; -import com.codahale.metrics.Timer; import com.github.rholder.retry.Attempt; import com.github.rholder.retry.RetryListener; import com.github.rholder.retry.Retryer; @@ -12,6 +11,7 @@ import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffset; import pl.allegro.tech.hermes.common.message.wrapper.CompositeMessageContentWrapper; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.batch.MessageBatch; import pl.allegro.tech.hermes.consumers.consumer.batch.MessageBatchFactory; import pl.allegro.tech.hermes.consumers.consumer.batch.MessageBatchReceiver; @@ -56,7 +56,8 @@ public class BatchConsumer implements Consumer { private volatile boolean consuming = true; - private final SubscriptionMetrics metrics; + private final MetricsFacade metricsFacade; + private final BatchConsumerMetrics metrics; private MessageBatchReceiver receiver; public BatchConsumer(ReceiverFactory messageReceiverFactory, @@ -65,7 +66,7 @@ public BatchConsumer(ReceiverFactory messageReceiverFactory, OffsetQueue offsetQueue, MessageConverterResolver messageConverterResolver, CompositeMessageContentWrapper compositeMessageContentWrapper, - SubscriptionMetrics metrics, + MetricsFacade metricsFacade, Trackers trackers, Subscription subscription, Topic topic, @@ -78,7 +79,8 @@ public BatchConsumer(ReceiverFactory messageReceiverFactory, this.subscription = subscription; this.useTopicMessageSize = useTopicMessageSize; this.loadRecorder = loadRecorder; - this.metrics = metrics; + this.metricsFacade = metricsFacade; + this.metrics = new BatchConsumerMetrics(metricsFacade, subscription.getQualifiedName()); this.messageConverterResolver = messageConverterResolver; this.compositeMessageContentWrapper = compositeMessageContentWrapper; this.topic = topic; @@ -107,7 +109,7 @@ public void consume(Runnable signalsInterrupt) { }); result.getDiscarded().forEach(m -> { - metrics.markDiscarded(m); + metrics.markDiscarded(); trackers.get(subscription).logDiscarded(m, "too large"); }); } finally { @@ -133,20 +135,20 @@ public void initialize() { subscription, new BatchConsumerRateLimiter(), loadRecorder, - metrics + metricsFacade ); logger.debug("Consumer: preparing batch receiver for subscription {}", subscription.getQualifiedName()); this.receiver = new MessageBatchReceiver( receiver, batchFactory, - metrics, messageConverterResolver, compositeMessageContentWrapper, topic, trackers, loadRecorder ); + metrics.initialize(); } @Override @@ -227,6 +229,7 @@ private Retryer createRetryer(final MessageBatch batch, private void markSendingResult(MessageBatch batch, MessageSendingResult result) { if (result.succeeded()) { + metrics.recordAttemptAsFinished(batch.getMessageCount()); metrics.markSuccess(batch, result); batch.getMessagesMetadata().forEach( m -> trackers.get(subscription).logSent(m, result.getHostname()) @@ -244,7 +247,8 @@ private boolean shouldRetryOnClientError(boolean retryClientErrors, MessageSendi } private void deliver(Runnable signalsInterrupt, MessageBatch batch, Retryer retryer) { - try (Timer.Context timer = metrics.subscriptionLatencyTimer().time()) { + metrics.recordAttempt(batch.getMessageCount()); + try (HermesTimerContext ignored = metrics.latencyTimer().time()) { retryer.call(() -> { loadRecorder.recordSingleOperation(); signalsInterrupt.run(); @@ -257,6 +261,7 @@ private void deliver(Runnable signalsInterrupt, MessageBatch batch, Retryer trackers.get(subscription).logDiscarded(m, e.getMessage())); } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/BatchConsumerMetrics.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/BatchConsumerMetrics.java new file mode 100644 index 0000000000..d75c9418bf --- /dev/null +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/BatchConsumerMetrics.java @@ -0,0 +1,102 @@ +package pl.allegro.tech.hermes.consumers.consumer; + +import pl.allegro.tech.hermes.api.SubscriptionName; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; +import pl.allegro.tech.hermes.consumers.consumer.batch.MessageBatch; +import pl.allegro.tech.hermes.consumers.consumer.sender.MessageSendingResult; +import pl.allegro.tech.hermes.metrics.HermesCounter; +import pl.allegro.tech.hermes.metrics.HermesHistogram; +import pl.allegro.tech.hermes.metrics.HermesTimer; +import pl.allegro.tech.hermes.tracker.consumers.MessageMetadata; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.LongAdder; + +class BatchConsumerMetrics { + + private final MetricsFacade metrics; + private final SubscriptionName subscriptionName; + private final LongAdder inflightCount = new LongAdder(); + private final HermesCounter failures; + private final HermesCounter timeouts; + private final HermesCounter otherErrors; + private final HermesCounter discarded; + private final HermesHistogram inflightTime; + private final HermesCounter throughputInBytes; + private final HermesCounter successes; + private final HermesCounter batchSuccesses; + private final HermesTimer latency; + private final Map httpStatusCodes = new ConcurrentHashMap<>(); + + BatchConsumerMetrics(MetricsFacade metrics, SubscriptionName subscriptionName) { + this.metrics = metrics; + this.subscriptionName = subscriptionName; + this.failures = metrics.subscriptions().failuresCounter(subscriptionName); + this.timeouts = metrics.subscriptions().timeoutsCounter(subscriptionName); + this.otherErrors = metrics.subscriptions().otherErrorsCounter(subscriptionName); + this.discarded = metrics.subscriptions().discarded(subscriptionName); + this.inflightTime = metrics.subscriptions().inflightTimeInMillisHistogram(subscriptionName); + this.throughputInBytes = metrics.subscriptions().throughputInBytes(subscriptionName); + this.successes = metrics.subscriptions().successes(subscriptionName); + this.batchSuccesses = metrics.subscriptions().batchSuccesses(subscriptionName); + this.latency = metrics.subscriptions().latency(subscriptionName); + } + + void recordAttempt(int messageCount) { + inflightCount.add(messageCount); + } + + void recordAttemptAsFinished(int messageCount) { + inflightCount.add(-1 * messageCount); + } + + void markFailure(MessageBatch batch, MessageSendingResult result) { + failures.increment(); + if (result.hasHttpAnswer()) { + markHttpStatusCode(result.getStatusCode()); + } else if (result.isTimeout()) { + timeouts.increment(); + } else { + otherErrors.increment(); + } + throughputInBytes.increment(batch.getSize()); + } + + void markSuccess(MessageBatch batch, MessageSendingResult result) { + successes.increment(batch.getMessageCount()); + batchSuccesses.increment(); + throughputInBytes.increment(batch.getSize()); + markHttpStatusCode(result.getStatusCode()); + inflightTime.record(batch.getLifetime()); + } + + private void markHttpStatusCode(int statusCode) { + httpStatusCodes.computeIfAbsent( + statusCode, + integer -> metrics.subscriptions().httpAnswerCounter(subscriptionName, statusCode) + ).increment(); + } + + void shutdown() { + metrics.unregisterAllMetricsRelatedTo(subscriptionName); + } + + void initialize() { + metrics.subscriptions() + .registerInflightGauge(subscriptionName, this, metrics -> metrics.inflightCount.doubleValue()); + } + + void markDiscarded() { + discarded.increment(); + } + + void markDiscarded(MessageBatch batch) { + discarded.increment(batch.getMessageCount()); + inflightTime.record(batch.getLifetime()); + } + + HermesTimer latencyTimer() { + return latency; + } +} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java index f2b64dce7d..7172e944fd 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSender.java @@ -1,11 +1,11 @@ package pl.allegro.tech.hermes.consumers.consumer; -import com.codahale.metrics.Timer; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.eclipse.jetty.http.HttpStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.api.Subscription; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder; import pl.allegro.tech.hermes.consumers.consumer.rate.InflightsPool; import pl.allegro.tech.hermes.consumers.consumer.rate.SerialConsumerRateLimiter; @@ -16,6 +16,8 @@ import pl.allegro.tech.hermes.consumers.consumer.sender.MessageSendingResult; import pl.allegro.tech.hermes.consumers.consumer.sender.MessageSendingResultLogInfo; import pl.allegro.tech.hermes.consumers.consumer.sender.timeout.FutureAsyncTimeout; +import pl.allegro.tech.hermes.metrics.HermesTimer; +import pl.allegro.tech.hermes.metrics.HermesTimerContext; import java.net.URI; import java.time.Clock; @@ -27,6 +29,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.LongAdder; import static java.lang.String.format; import static org.apache.commons.lang3.math.NumberUtils.INTEGER_ZERO; @@ -41,10 +44,11 @@ public class ConsumerMessageSender { private final Clock clock; private final InflightsPool inflight; private final SubscriptionLoadRecorder loadRecorder; - private final Timer consumerLatencyTimer; + private final HermesTimer consumerLatencyTimer; private final SerialConsumerRateLimiter rateLimiter; private final FutureAsyncTimeout async; private final int asyncTimeoutMs; + private final LongAdder inflightCount = new LongAdder(); private MessageSender messageSender; private Subscription subscription; @@ -52,7 +56,6 @@ public class ConsumerMessageSender { private ScheduledExecutorService retrySingleThreadExecutor; private volatile boolean running = true; - public ConsumerMessageSender(Subscription subscription, MessageSenderFactory messageSenderFactory, List successHandlers, @@ -60,7 +63,7 @@ public ConsumerMessageSender(Subscription subscription, SerialConsumerRateLimiter rateLimiter, ExecutorService deliveryReportingExecutor, InflightsPool inflight, - SubscriptionMetrics metrics, + MetricsFacade metrics, int asyncTimeoutMs, FutureAsyncTimeout futureAsyncTimeout, Clock clock, @@ -77,7 +80,8 @@ public ConsumerMessageSender(Subscription subscription, this.messageSender = messageSender(subscription); this.subscription = subscription; this.inflight = inflight; - this.consumerLatencyTimer = metrics.subscriptionLatencyTimer(); + this.consumerLatencyTimer = metrics.subscriptions().latency(subscription.getQualifiedName()); + metrics.subscriptions().registerInflightGauge(subscription.getQualifiedName(), this, sender -> sender.inflightCount.doubleValue()); } public void initialize() { @@ -99,6 +103,7 @@ public void shutdown() { } public void sendAsync(Message message) { + inflightCount.increment(); sendAsync(message, calculateMessageDelay(message.getPublishingTimestamp())); } @@ -126,7 +131,7 @@ private int calculateMessageDelay(long publishingMessageTimestamp) { */ private void sendMessage(final Message message) { loadRecorder.recordSingleOperation(); - Timer.Context timer = consumerLatencyTimer.time(); + HermesTimerContext timer = consumerLatencyTimer.time(); CompletableFuture response = messageSender.send(message); response.thenAcceptAsync(new ResponseHandlingListener(message, timer), deliveryReportingExecutor) @@ -228,11 +233,13 @@ private void logResultInfo(Message message, MessageSendingResultLogInfo logInfo) private void handleMessageDiscarding(Message message, MessageSendingResult result) { inflight.release(); + inflightCount.decrement(); errorHandlers.forEach(h -> h.handleDiscarded(message, subscription, result)); } private void handleMessageSendingSuccess(Message message, MessageSendingResult result) { inflight.release(); + inflightCount.decrement(); successHandlers.forEach(h -> h.handleSuccess(message, subscription, result)); } @@ -256,9 +263,9 @@ private boolean isUnauthorizedForOAuthSecuredSubscription(MessageSendingResult r class ResponseHandlingListener implements java.util.function.Consumer { private final Message message; - private final Timer.Context timer; + private final HermesTimerContext timer; - public ResponseHandlingListener(Message message, Timer.Context timer) { + public ResponseHandlingListener(Message message, HermesTimerContext timer) { this.message = message; this.timer = timer; } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderFactory.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderFactory.java index ce97ec9813..c47557bcf7 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderFactory.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderFactory.java @@ -2,6 +2,7 @@ import pl.allegro.tech.hermes.api.Subscription; import pl.allegro.tech.hermes.common.message.undelivered.UndeliveredMessageLog; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.common.metric.executor.InstrumentedExecutorServiceFactory; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder; import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue; @@ -59,15 +60,24 @@ public ConsumerMessageSender create(Subscription subscription, OffsetQueue offsetQueue, InflightsPool inflight, SubscriptionLoadRecorder subscriptionLoadRecorder, - SubscriptionMetrics metrics) { + MetricsFacade metrics) { List successHandlers = Arrays.asList( consumerAuthorizationHandler, - new DefaultSuccessHandler(offsetQueue, metrics, trackers)); + new DefaultSuccessHandler(offsetQueue, metrics, trackers, subscription.getQualifiedName())); List errorHandlers = Arrays.asList( consumerAuthorizationHandler, - new DefaultErrorHandler(offsetQueue, metrics, undeliveredMessageLog, clock, trackers, kafkaClusterName)); + new DefaultErrorHandler( + offsetQueue, + metrics, + undeliveredMessageLog, + clock, + trackers, + kafkaClusterName, + subscription.getQualifiedName() + ) + ); return new ConsumerMessageSender(subscription, messageSenderFactory, diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/SerialConsumer.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/SerialConsumer.java index 9d89846bb5..7fffaff85a 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/SerialConsumer.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/SerialConsumer.java @@ -5,6 +5,7 @@ import pl.allegro.tech.hermes.api.Subscription; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffset; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.CommonConsumerParameters; import pl.allegro.tech.hermes.consumers.consumer.converter.MessageConverterResolver; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder; @@ -30,7 +31,7 @@ public class SerialConsumer implements Consumer { private static final Logger logger = LoggerFactory.getLogger(SerialConsumer.class); private final ReceiverFactory messageReceiverFactory; - private final SubscriptionMetrics metrics; + private final MetricsFacade metrics; private final SerialConsumerRateLimiter rateLimiter; private final Trackers trackers; private final MessageConverterResolver messageConverterResolver; @@ -50,7 +51,7 @@ public class SerialConsumer implements Consumer { private MessageReceiver messageReceiver; public SerialConsumer(ReceiverFactory messageReceiverFactory, - SubscriptionMetrics metrics, + MetricsFacade metrics, Subscription subscription, SerialConsumerRateLimiter rateLimiter, ConsumerMessageSenderFactory consumerMessageSenderFactory, @@ -129,7 +130,6 @@ private void sendMessage(Message message) { message.getPartitionAssignmentTerm()) ); - metrics.markAttempt(); trackers.get(subscription).logInflight(toMessageMetadata(message, subscription)); sender.sendAsync(message); @@ -159,7 +159,7 @@ public void tearDown() { rateLimiter.shutdown(); loadRecorder.shutdown(); consumerAuthorizationHandler.removeSubscriptionHandler(subscription.getQualifiedName()); - metrics.shutdown(); + metrics.unregisterAllMetricsRelatedTo(subscription.getQualifiedName()); } @Override diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/SubscriptionMetrics.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/SubscriptionMetrics.java deleted file mode 100644 index c8b18340bc..0000000000 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/SubscriptionMetrics.java +++ /dev/null @@ -1,180 +0,0 @@ -package pl.allegro.tech.hermes.consumers.consumer; - -import com.codahale.metrics.Counter; -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Timer; -import pl.allegro.tech.hermes.api.SubscriptionName; -import pl.allegro.tech.hermes.api.TopicName; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; -import pl.allegro.tech.hermes.common.metric.MetricsFacade; -import pl.allegro.tech.hermes.common.metric.SubscriptionHermesCounter; -import pl.allegro.tech.hermes.consumers.consumer.batch.MessageBatch; -import pl.allegro.tech.hermes.consumers.consumer.sender.MessageSendingResult; -import pl.allegro.tech.hermes.tracker.consumers.MessageMetadata; - -import static pl.allegro.tech.hermes.api.TopicName.fromQualifiedName; -import static pl.allegro.tech.hermes.common.metric.Counters.DELIVERED; -import static pl.allegro.tech.hermes.common.metric.Counters.DISCARDED; -import static pl.allegro.tech.hermes.common.metric.Counters.MAXRATE_FETCH_FAILURES; -import static pl.allegro.tech.hermes.common.metric.Counters.MAXRATE_RATE_HISTORY_FAILURES; -import static pl.allegro.tech.hermes.common.metric.Gauges.MAX_RATE_ACTUAL_RATE_VALUE; -import static pl.allegro.tech.hermes.common.metric.Gauges.MAX_RATE_VALUE; -import static pl.allegro.tech.hermes.common.metric.Gauges.OUTPUT_RATE; -import static pl.allegro.tech.hermes.common.metric.Meters.DISCARDED_METER; -import static pl.allegro.tech.hermes.common.metric.Meters.DISCARDED_SUBSCRIPTION_METER; -import static pl.allegro.tech.hermes.common.metric.Meters.DISCARDED_TOPIC_METER; -import static pl.allegro.tech.hermes.common.metric.Meters.FAILED_METER_SUBSCRIPTION; -import static pl.allegro.tech.hermes.common.metric.Meters.FILTERED_METER; -import static pl.allegro.tech.hermes.common.metric.Meters.METER; -import static pl.allegro.tech.hermes.common.metric.Meters.SUBSCRIPTION_BATCH_METER; -import static pl.allegro.tech.hermes.common.metric.Meters.SUBSCRIPTION_METER; -import static pl.allegro.tech.hermes.common.metric.Meters.TOPIC_METER; -import static pl.allegro.tech.hermes.common.metric.Timers.CONSUMER_IDLE_TIME; -import static pl.allegro.tech.hermes.common.metric.Timers.SUBSCRIPTION_LATENCY; - -public class SubscriptionMetrics { - - private final HermesMetrics metrics; - private final SubscriptionName subscription; - private final MetricsFacade metricsFacade; - - private final SubscriptionHermesCounter subscriptionThroughputCounter; - - public SubscriptionMetrics(HermesMetrics metrics, SubscriptionName subscription, - MetricsFacade metricsFacade) { - this.metrics = metrics; - this.subscription = subscription; - this.metricsFacade = metricsFacade; - this.subscriptionThroughputCounter = metricsFacade.subscriptionMetrics() - .subscriptionThroughputBytes(subscription); - } - - public void markAttempt() { - metrics.incrementInflightCounter(subscription); - } - - public void markSuccess(MessageBatch batch, MessageSendingResult result) { - metrics.meter(METER).mark(batch.getMessageCount()); - metrics.meter(TOPIC_METER, subscription.getTopicName()).mark(batch.getMessageCount()); - metrics.meter(SUBSCRIPTION_METER, subscription.getTopicName(), subscription.getName()).mark(batch.getMessageCount()); - metrics.meter(SUBSCRIPTION_BATCH_METER, subscription.getTopicName(), subscription.getName()).mark(); - subscriptionThroughputCounter.increment(batch.getSize()); - metrics.registerConsumerHttpAnswer(subscription, result.getStatusCode()); - metrics.counter(DELIVERED, subscription.getTopicName(), subscription.getName()).inc(batch.getMessageCount()); - metrics.decrementInflightCounter(subscription, batch.getMessageCount()); - metrics.inflightTimeHistogram(subscription).update(batch.getLifetime()); - } - - public void markSuccess(Message message, MessageSendingResult result) { - metrics.meter(METER).mark(); - metrics.meter(TOPIC_METER, subscription.getTopicName()).mark(); - metrics.meter(SUBSCRIPTION_METER, subscription.getTopicName(), subscription.getName()).mark(); - subscriptionThroughputCounter.increment(message.getSize()); - metrics.registerConsumerHttpAnswer(subscription, result.getStatusCode()); - metrics.counter(DELIVERED, subscription.getTopicName(), subscription.getName()).inc(); - metrics.decrementInflightCounter(subscription); - metrics.inflightTimeHistogram(subscription).update(System.currentTimeMillis() - message.getReadingTimestamp()); - } - - public void markFailure(MessageBatch batch, MessageSendingResult result) { - registerFailureMetrics(result, batch.getSize()); - } - - public void markFailure(Message message, MessageSendingResult result) { - registerFailureMetrics(result, message.getSize()); - } - - private void registerFailureMetrics(MessageSendingResult result, long messageSize) { - metrics.meter(FAILED_METER_SUBSCRIPTION, subscription.getTopicName(), subscription.getName()).mark(); - if (result.hasHttpAnswer()) { - metrics.registerConsumerHttpAnswer(subscription, result.getStatusCode()); - } else if (result.isTimeout()) { - metrics.consumerErrorsTimeoutMeter(subscription).mark(); - } else { - metrics.consumerErrorsOtherMeter(subscription).mark(); - } - subscriptionThroughputCounter.increment(messageSize); - } - - public void markDiscarded(Message message) { - metrics.meter(DISCARDED_METER).mark(); - metrics.meter(DISCARDED_TOPIC_METER, subscription.getTopicName()).mark(); - metrics.meter(DISCARDED_SUBSCRIPTION_METER, subscription.getTopicName(), subscription.getName()).mark(); - metrics.counter(DISCARDED, subscription.getTopicName(), subscription.getName()).inc(); - metrics.decrementInflightCounter(subscription); - metrics.inflightTimeHistogram(subscription).update(System.currentTimeMillis() - message.getReadingTimestamp()); - } - - public void markDiscarded(MessageBatch batch) { - metrics.meter(DISCARDED_METER).mark(batch.getMessageCount()); - metrics.meter(DISCARDED_TOPIC_METER, subscription.getTopicName()).mark(batch.getMessageCount()); - metrics.meter(DISCARDED_SUBSCRIPTION_METER, subscription.getTopicName(), subscription.getName()).mark(batch.getMessageCount()); - metrics.counter(DISCARDED, subscription.getTopicName(), subscription.getName()).inc(batch.getMessageCount()); - metrics.decrementInflightCounter(subscription, batch.getMessageCount()); - metrics.inflightTimeHistogram(subscription).update(batch.getLifetime()); - } - - public void markDiscarded(MessageMetadata messageMetadata) { - TopicName topicName = fromQualifiedName(messageMetadata.getTopic()); - metrics.counter(DISCARDED, topicName, messageMetadata.getSubscription()).inc(); - metrics.meter(DISCARDED_METER).mark(); - metrics.meter(DISCARDED_TOPIC_METER, topicName).mark(); - metrics.meter(DISCARDED_SUBSCRIPTION_METER, topicName, messageMetadata.getSubscription()).mark(); - metrics.decrementInflightCounter(subscription); - } - - public Timer subscriptionLatencyTimer() { - return metrics.timer(SUBSCRIPTION_LATENCY, subscription.getTopicName(), subscription.getName()); - } - - public void markFilteredOut() { - metrics.meter(FILTERED_METER, subscription.getTopicName(), subscription.getName()).mark(); - } - - public Timer consumerIdleTimer() { - return metrics.timer(CONSUMER_IDLE_TIME, subscription.getTopicName(), subscription.getName()); - } - - public Counter rateHistoryFailuresCounter() { - return metrics.counter(MAXRATE_RATE_HISTORY_FAILURES, subscription.getTopicName(), subscription.getName()); - } - - public Counter maxRateFetchFailuresCounter() { - return metrics.counter(MAXRATE_FETCH_FAILURES, subscription.getTopicName(), subscription.getName()); - } - - public void registerMaxRateGauge(Gauge gauge) { - metrics.registerGauge(MAX_RATE_VALUE, subscription, gauge); - } - - public void registerRateGauge(Gauge gauge) { - metrics.registerGauge(MAX_RATE_ACTUAL_RATE_VALUE, subscription, gauge); - } - - public void registerOutputRateGauge(Gauge gauge) { - metrics.registerGauge(OUTPUT_RATE, subscription, gauge); - } - - public void shutdown() { - metrics.unregister(DISCARDED_SUBSCRIPTION_METER, subscription); - metrics.unregister(FAILED_METER_SUBSCRIPTION, subscription); - metrics.unregister(SUBSCRIPTION_BATCH_METER, subscription); - metrics.unregister(SUBSCRIPTION_METER, subscription); - metrics.unregister(DELIVERED, subscription); - metrics.unregister(DISCARDED, subscription); - metrics.unregisterInflightCounter(subscription); - metrics.unregisterInflightTimeHistogram(subscription); - metrics.unregisterConsumerErrorsTimeoutMeter(subscription); - metrics.unregisterConsumerErrorsOtherMeter(subscription); - metrics.unregisterStatusMeters(subscription); - metrics.unregister(OUTPUT_RATE, subscription); - metrics.unregister(MAX_RATE_ACTUAL_RATE_VALUE, subscription); - metrics.unregister(MAX_RATE_VALUE, subscription); - metrics.unregister(MAXRATE_FETCH_FAILURES, subscription); - metrics.unregister(MAXRATE_RATE_HISTORY_FAILURES, subscription); - metrics.unregister(CONSUMER_IDLE_TIME, subscription); - metrics.unregister(FILTERED_METER, subscription); - metrics.unregister(SUBSCRIPTION_LATENCY, subscription); - metricsFacade.subscriptionMetrics().unregister(subscriptionThroughputCounter); - } -} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/batch/ByteBufferMessageBatchFactory.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/batch/ByteBufferMessageBatchFactory.java index 115227bdf6..09ecbfc037 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/batch/ByteBufferMessageBatchFactory.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/batch/ByteBufferMessageBatchFactory.java @@ -2,8 +2,7 @@ import pl.allegro.tech.hermes.api.Subscription; import pl.allegro.tech.hermes.common.exception.InternalProcessingException; -import pl.allegro.tech.hermes.common.metric.Gauges; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import java.nio.ByteBuffer; import java.time.Clock; @@ -15,11 +14,11 @@ public class ByteBufferMessageBatchFactory implements MessageBatchFactory { private final DirectBufferPool bufferPool; private final Clock clock; - public ByteBufferMessageBatchFactory(int poolableSize, int maxPoolSize, Clock clock, HermesMetrics hermesMetrics) { + public ByteBufferMessageBatchFactory(int poolableSize, int maxPoolSize, Clock clock, MetricsFacade metrics) { this.clock = clock; this.bufferPool = new DirectBufferPool(maxPoolSize, poolableSize, true); - hermesMetrics.registerGauge(Gauges.BATCH_BUFFER_TOTAL_BYTES, bufferPool::totalMemory); - hermesMetrics.registerGauge(Gauges.BATCH_BUFFER_AVAILABLE_BYTES, bufferPool::availableMemory); + metrics.consumer().registerBatchBufferTotalBytesGauge(bufferPool, DirectBufferPool::totalMemory); + metrics.consumer().registerBatchBufferAvailableBytesGauge(bufferPool, DirectBufferPool::availableMemory); } @Override diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/batch/MessageBatchReceiver.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/batch/MessageBatchReceiver.java index 3d4c187e77..efbdc09f77 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/batch/MessageBatchReceiver.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/batch/MessageBatchReceiver.java @@ -9,7 +9,6 @@ import pl.allegro.tech.hermes.common.message.wrapper.CompositeMessageContentWrapper; import pl.allegro.tech.hermes.common.message.wrapper.UnsupportedContentTypeException; import pl.allegro.tech.hermes.consumers.consumer.Message; -import pl.allegro.tech.hermes.consumers.consumer.SubscriptionMetrics; import pl.allegro.tech.hermes.consumers.consumer.converter.MessageConverterResolver; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder; import pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset; @@ -38,7 +37,6 @@ public class MessageBatchReceiver { private final MessageBatchFactory batchFactory; private final MessageConverterResolver messageConverterResolver; private final CompositeMessageContentWrapper compositeMessageContentWrapper; - private final SubscriptionMetrics metrics; private final Trackers trackers; private final Queue inflight; private final Topic topic; @@ -47,7 +45,6 @@ public class MessageBatchReceiver { public MessageBatchReceiver(MessageReceiver receiver, MessageBatchFactory batchFactory, - SubscriptionMetrics metrics, MessageConverterResolver messageConverterResolver, CompositeMessageContentWrapper compositeMessageContentWrapper, Topic topic, @@ -55,7 +52,6 @@ public MessageBatchReceiver(MessageReceiver receiver, SubscriptionLoadRecorder loadRecorder) { this.receiver = receiver; this.batchFactory = batchFactory; - this.metrics = metrics; this.messageConverterResolver = messageConverterResolver; this.compositeMessageContentWrapper = compositeMessageContentWrapper; this.topic = topic; @@ -115,7 +111,7 @@ private Optional readAndTransform(Subscription subscription, String bat Message transformed = messageConverterResolver.converterFor(message, subscription).convert(message, topic); transformed = message().fromMessage(transformed).withData(wrap(subscription, transformed)).build(); - metrics.markAttempt(); + trackers.get(subscription).logInflight(toMessageMetadata(transformed, subscription, batchId)); return Optional.of(transformed); diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/filtering/FilteredMessageHandler.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/filtering/FilteredMessageHandler.java index 1d4b5daba8..12063eb085 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/filtering/FilteredMessageHandler.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/filtering/FilteredMessageHandler.java @@ -3,11 +3,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.api.Subscription; +import pl.allegro.tech.hermes.api.SubscriptionName; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.Message; -import pl.allegro.tech.hermes.consumers.consumer.SubscriptionMetrics; import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue; import pl.allegro.tech.hermes.consumers.consumer.rate.ConsumerRateLimiter; import pl.allegro.tech.hermes.domain.filtering.chain.FilterResult; +import pl.allegro.tech.hermes.metrics.HermesCounter; import pl.allegro.tech.hermes.tracker.consumers.Trackers; import java.util.Optional; @@ -20,18 +22,19 @@ public class FilteredMessageHandler { private final OffsetQueue offsetQueue; private final Optional consumerRateLimiter; private final Trackers trackers; - private final SubscriptionMetrics metrics; + private final HermesCounter filteredOutCounter; private static final Logger logger = LoggerFactory.getLogger(FilteredMessageHandler.class); public FilteredMessageHandler(OffsetQueue offsetQueue, ConsumerRateLimiter consumerRateLimiter, Trackers trackers, - SubscriptionMetrics metrics) { + MetricsFacade metrics, + SubscriptionName subscriptionName) { this.offsetQueue = offsetQueue; this.consumerRateLimiter = Optional.ofNullable(consumerRateLimiter); this.trackers = trackers; - this.metrics = metrics; + this.filteredOutCounter = metrics.subscriptions().filteredOutCounter(subscriptionName); } public void handle(FilterResult result, Message message, Subscription subscription) { @@ -43,7 +46,7 @@ public void handle(FilterResult result, Message message, Subscription subscripti offsetQueue.offerCommittedOffset(subscriptionPartitionOffset(subscription.getQualifiedName(), message.getPartitionOffset(), message.getPartitionAssignmentTerm())); - metrics.markFilteredOut(); + filteredOutCounter.increment(); if (subscription.isTrackingEnabled()) { trackers.get(subscription).logFiltered(toMessageMetadata(message, subscription), result.getFilterType().get()); diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/oauth/OAuthAccessTokensLoader.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/oauth/OAuthAccessTokensLoader.java index ab61fc7cdf..40fd06cd0a 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/oauth/OAuthAccessTokensLoader.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/oauth/OAuthAccessTokensLoader.java @@ -1,16 +1,16 @@ package pl.allegro.tech.hermes.consumers.consumer.oauth; -import com.codahale.metrics.Timer; import com.google.common.cache.CacheLoader; import pl.allegro.tech.hermes.api.OAuthProvider; import pl.allegro.tech.hermes.api.Subscription; import pl.allegro.tech.hermes.api.SubscriptionName; import pl.allegro.tech.hermes.api.SubscriptionOAuthPolicy; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.oauth.client.OAuthClient; import pl.allegro.tech.hermes.consumers.consumer.oauth.client.OAuthTokenRequest; import pl.allegro.tech.hermes.domain.oauth.OAuthProviderRepository; import pl.allegro.tech.hermes.domain.subscription.SubscriptionRepository; +import pl.allegro.tech.hermes.metrics.HermesTimerContext; import static pl.allegro.tech.hermes.api.SubscriptionOAuthPolicy.GrantType.USERNAME_PASSWORD; import static pl.allegro.tech.hermes.consumers.consumer.oauth.client.OAuthTokenRequest.oAuthTokenRequest; @@ -23,12 +23,12 @@ public class OAuthAccessTokensLoader extends CacheLoader inflightOffsets = new HashSet<>(); private final Map maxCommittedOffsets = new HashMap<>(); @@ -95,20 +97,20 @@ public OffsetCommitter( ConsumerPartitionAssignmentState partitionAssignmentState, MessageCommitter messageCommitter, int offsetCommitPeriodSeconds, - HermesMetrics oldMetrics, - MetricsFacade metricsFacade + MetricsFacade metrics ) { this.offsetQueue = offsetQueue; this.partitionAssignmentState = partitionAssignmentState; this.messageCommitter = messageCommitter; this.offsetCommitPeriodSeconds = offsetCommitPeriodSeconds; - this.oldMetrics = oldMetrics; - this.metricsFacade = metricsFacade; + this.obsoleteCounter = metrics.offsetCommits().obsoleteCounter(); + this.committedCounter = metrics.offsetCommits().committedCounter(); + this.timer = metrics.offsetCommits().duration(); } @Override public void run() { - try (HermesTimerContext c = metricsFacade.consumers().offsetCommitterDuration().time()) { + try (HermesTimerContext ignored = timer.time()) { // committed offsets need to be drained first so that there is no possibility of new committed offsets // showing up after inflight queue is drained - this would lead to stall in committing offsets ReducingConsumer committedOffsetsReducer = processCommittedOffsets(); @@ -151,8 +153,8 @@ public void run() { committedOffsetToBeRemoved.forEach(maxCommittedOffsets::remove); messageCommitter.commitOffsets(offsetsToCommit); - oldMetrics.counter("offset-committer.obsolete").inc(obsoleteCount); - oldMetrics.counter("offset-committer.committed").inc(scheduledToCommitCount); + obsoleteCounter.increment(obsoleteCount); + committedCounter.increment(scheduledToCommitCount); cleanupStoredOffsetsWithObsoleteTerms(); } catch (Exception exception) { diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetQueue.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetQueue.java index bde3d9a50d..aee72eb210 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetQueue.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetQueue.java @@ -1,7 +1,7 @@ package pl.allegro.tech.hermes.consumers.consumer.offset; import org.jctools.queues.MessagePassingQueue; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.queue.FullDrainMpscQueue; import pl.allegro.tech.hermes.consumers.queue.MonitoredMpscQueue; import pl.allegro.tech.hermes.consumers.queue.MpscQueue; @@ -13,7 +13,7 @@ public class OffsetQueue { private final MpscQueue commitOffsetsQueue; - public OffsetQueue(HermesMetrics metrics, int commitOffsetQueuesSize) { + public OffsetQueue(MetricsFacade metrics, int commitOffsetQueuesSize) { this.inflightOffsetsQueue = new MonitoredMpscQueue<>(new FullDrainMpscQueue<>(commitOffsetQueuesSize), metrics, "inflightOffsets"); this.commitOffsetsQueue = diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/SerialConsumerRateLimiter.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/SerialConsumerRateLimiter.java index 62de4420e2..c2bf40d93f 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/SerialConsumerRateLimiter.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/SerialConsumerRateLimiter.java @@ -2,7 +2,7 @@ import com.google.common.util.concurrent.RateLimiter; import pl.allegro.tech.hermes.api.Subscription; -import pl.allegro.tech.hermes.consumers.consumer.SubscriptionMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.rate.calculator.OutputRateCalculationResult; import pl.allegro.tech.hermes.consumers.consumer.rate.calculator.OutputRateCalculator; import pl.allegro.tech.hermes.consumers.consumer.rate.calculator.OutputRateCalculatorFactory; @@ -14,7 +14,7 @@ public class SerialConsumerRateLimiter implements ConsumerRateLimiter { private Subscription subscription; - private final SubscriptionMetrics metrics; + private final MetricsFacade metrics; private final ConsumerRateLimitSupervisor rateLimitSupervisor; @@ -30,7 +30,7 @@ public class SerialConsumerRateLimiter implements ConsumerRateLimiter { public SerialConsumerRateLimiter(Subscription subscription, OutputRateCalculatorFactory outputRateCalculatorFactory, - SubscriptionMetrics metrics, + MetricsFacade metrics, ConsumerRateLimitSupervisor rateLimitSupervisor, Clock clock) { this.subscription = subscription; @@ -47,7 +47,7 @@ public SerialConsumerRateLimiter(Subscription subscription, public void initialize() { outputRateCalculator.start(); adjustConsumerRate(); - metrics.registerOutputRateGauge(rateLimiter::getRate); + metrics.maxRate().registerOutputRateGauge(subscription.getQualifiedName(), rateLimiter, RateLimiter::getRate); rateLimitSupervisor.register(this); } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/calculator/OutputRateCalculatorFactory.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/calculator/OutputRateCalculatorFactory.java index cfb133f235..52d4892b08 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/calculator/OutputRateCalculatorFactory.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/calculator/OutputRateCalculatorFactory.java @@ -1,7 +1,7 @@ package pl.allegro.tech.hermes.consumers.consumer.rate.calculator; import pl.allegro.tech.hermes.api.Subscription; -import pl.allegro.tech.hermes.consumers.consumer.SubscriptionMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.rate.SendCounters; import pl.allegro.tech.hermes.consumers.consumer.rate.maxrate.MaxRateProvider; import pl.allegro.tech.hermes.consumers.consumer.rate.maxrate.MaxRateProviderFactory; @@ -19,7 +19,7 @@ public OutputRateCalculatorFactory(RateCalculatorParameters rateCalculatorParame public OutputRateCalculator createCalculator(Subscription subscription, SendCounters sendCounters, - SubscriptionMetrics metrics) { + MetricsFacade metrics) { MaxRateProvider maxRateProvider = maxRateProviderFactory.create(subscription, sendCounters, metrics); return new OutputRateCalculator(rateCalculatorParameters, maxRateProvider); } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateCalculator.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateCalculator.java index 30e2ffd63b..536f815fbc 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateCalculator.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateCalculator.java @@ -3,8 +3,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.api.Subscription; -import pl.allegro.tech.hermes.common.metric.Gauges; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.subscription.cache.SubscriptionsCache; import pl.allegro.tech.hermes.consumers.supervisor.workload.ClusterAssignmentCache; @@ -21,7 +20,6 @@ class MaxRateCalculator { private final SubscriptionsCache subscriptionsCache; private final MaxRateBalancer balancer; private final MaxRateRegistry maxRateRegistry; - private final HermesMetrics metrics; private final Clock clock; private volatile long lastUpdateDurationMillis = 0; @@ -30,16 +28,14 @@ class MaxRateCalculator { SubscriptionsCache subscriptionsCache, MaxRateBalancer balancer, MaxRateRegistry maxRateRegistry, - HermesMetrics metrics, + MetricsFacade metrics, Clock clock) { this.clusterAssignmentCache = clusterAssignmentCache; this.subscriptionsCache = subscriptionsCache; this.balancer = balancer; this.maxRateRegistry = maxRateRegistry; - this.metrics = metrics; this.clock = clock; - - metrics.registerGauge(Gauges.MAX_RATE_CALCULATION_DURATION, () -> lastUpdateDurationMillis); + metrics.maxRate().registerCalculationDurationInMillisGauge(this, calculator -> calculator.lastUpdateDurationMillis); } void calculate() { diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateCalculatorJob.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateCalculatorJob.java index 12ce0a42c5..0ceebc5814 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateCalculatorJob.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateCalculatorJob.java @@ -2,7 +2,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.curator.framework.recipes.leader.LeaderLatchListener; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.registry.ConsumerNodesRegistry; import pl.allegro.tech.hermes.consumers.subscription.cache.SubscriptionsCache; import pl.allegro.tech.hermes.consumers.supervisor.workload.ClusterAssignmentCache; @@ -29,7 +29,7 @@ class MaxRateCalculatorJob implements LeaderLatchListener, Runnable { MaxRateBalancer balancer, MaxRateRegistry maxRateRegistry, SubscriptionsCache subscriptionsCache, - HermesMetrics metrics, + MetricsFacade metrics, Clock clock) { this.consumerNodesRegistry = consumerNodesRegistry; this.interval = internal; diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateProviderFactory.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateProviderFactory.java index bbf853a5ce..40fed0de87 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateProviderFactory.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateProviderFactory.java @@ -2,7 +2,7 @@ import com.google.common.base.Preconditions; import pl.allegro.tech.hermes.api.Subscription; -import pl.allegro.tech.hermes.consumers.consumer.SubscriptionMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.rate.SendCounters; public class MaxRateProviderFactory { @@ -24,12 +24,12 @@ public MaxRateProviderFactory(MaxRateParameters maxRateParameters, }; } - public MaxRateProvider create(Subscription subscription, SendCounters sendCounters, SubscriptionMetrics metrics) { + public MaxRateProvider create(Subscription subscription, SendCounters sendCounters, MetricsFacade metrics) { return providerCreator.create(subscription, sendCounters, metrics); } private interface Creator { - MaxRateProvider create(Subscription subscription, SendCounters sendCounters, SubscriptionMetrics metrics); + MaxRateProvider create(Subscription subscription, SendCounters sendCounters, MetricsFacade metrics); } private void checkNegotiatedSettings(double minSignificantChange, double busyTolerance) { diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateSupervisor.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateSupervisor.java index fd3b69b640..069356e9dd 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateSupervisor.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/MaxRateSupervisor.java @@ -1,7 +1,7 @@ package pl.allegro.tech.hermes.consumers.consumer.rate.maxrate; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.registry.ConsumerNodesRegistry; import pl.allegro.tech.hermes.consumers.subscription.cache.SubscriptionsCache; import pl.allegro.tech.hermes.consumers.supervisor.workload.ClusterAssignmentCache; @@ -30,7 +30,7 @@ public MaxRateSupervisor(MaxRateParameters maxRateParameters, MaxRateRegistry maxRateRegistry, ConsumerNodesRegistry consumerNodesRegistry, SubscriptionsCache subscriptionsCache, - HermesMetrics metrics, + MetricsFacade metrics, Clock clock) { this.maxRateRegistry = maxRateRegistry; this.selfUpdateInterval = maxRateParameters.getUpdateInterval(); diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/NegotiatedMaxRateProvider.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/NegotiatedMaxRateProvider.java index 312ff1d42e..705c2c1e2c 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/NegotiatedMaxRateProvider.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/NegotiatedMaxRateProvider.java @@ -3,8 +3,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.api.Subscription; -import pl.allegro.tech.hermes.consumers.consumer.SubscriptionMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.rate.SendCounters; +import pl.allegro.tech.hermes.metrics.HermesCounter; import java.util.Optional; @@ -16,7 +17,9 @@ public class NegotiatedMaxRateProvider implements MaxRateProvider { private final MaxRateRegistry registry; private final MaxRateSupervisor maxRateSupervisor; private final SendCounters sendCounters; - private final SubscriptionMetrics metrics; + private final MetricsFacade metrics; + private final HermesCounter fetchFailuresCounter; + private final HermesCounter historyUpdateFailuresCounter; private final double minSignificantChange; private final int historyLimit; private volatile double maxRate; @@ -27,7 +30,7 @@ public class NegotiatedMaxRateProvider implements MaxRateProvider { MaxRateSupervisor maxRateSupervisor, Subscription subscription, SendCounters sendCounters, - SubscriptionMetrics metrics, + MetricsFacade metrics, double initialMaxRate, double minSignificantChange, int historyLimit) { @@ -36,6 +39,8 @@ public class NegotiatedMaxRateProvider implements MaxRateProvider { this.maxRateSupervisor = maxRateSupervisor; this.sendCounters = sendCounters; this.metrics = metrics; + this.fetchFailuresCounter = metrics.maxRate().fetchFailuresCounter(subscription.getQualifiedName()); + this.historyUpdateFailuresCounter = metrics.maxRate().historyUpdateFailuresCounter(subscription.getQualifiedName()); this.minSignificantChange = minSignificantChange; this.historyLimit = historyLimit; this.maxRate = initialMaxRate; @@ -61,7 +66,7 @@ private void recordCurrentRate(double actualRate) { previousRecordedRate = usedRate; } catch (Exception e) { logger.warn("Encountered problem updating max rate for {}", consumer, e); - metrics.rateHistoryFailuresCounter().inc(); + historyUpdateFailuresCounter.increment(); } } } @@ -75,15 +80,15 @@ private Optional fetchCurrentMaxRate() { return registry.getMaxRate(consumer); } catch (Exception e) { logger.warn("Encountered problem fetching max rate for {}", consumer); - metrics.maxRateFetchFailuresCounter().inc(); + fetchFailuresCounter.increment(); return Optional.empty(); } } public void start() { maxRateSupervisor.register(this); - metrics.registerMaxRateGauge(this::get); - metrics.registerRateGauge(sendCounters::getRate); + metrics.maxRate().registerCalculatedRateGauge(consumer.getSubscription(), this, NegotiatedMaxRateProvider::get); + metrics.maxRate().registerActualRateGauge(consumer.getSubscription(), sendCounters, SendCounters::getRate); } public void shutdown() { diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/ReceiverFactory.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/ReceiverFactory.java index ba355c555f..4077ad22bf 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/ReceiverFactory.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/ReceiverFactory.java @@ -2,7 +2,7 @@ import pl.allegro.tech.hermes.api.Subscription; import pl.allegro.tech.hermes.api.Topic; -import pl.allegro.tech.hermes.consumers.consumer.SubscriptionMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder; import pl.allegro.tech.hermes.consumers.consumer.rate.ConsumerRateLimiter; @@ -12,6 +12,6 @@ MessageReceiver createMessageReceiver(Topic receivingTopic, Subscription subscription, ConsumerRateLimiter consumerRateLimiter, SubscriptionLoadRecorder subscriptionLoadRecorder, - SubscriptionMetrics metrics); + MetricsFacade metrics); } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/ThrottlingMessageReceiver.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/ThrottlingMessageReceiver.java index 4849a5c67d..41d959780f 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/ThrottlingMessageReceiver.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/ThrottlingMessageReceiver.java @@ -1,12 +1,14 @@ package pl.allegro.tech.hermes.consumers.consumer.receiver; -import com.codahale.metrics.Timer; import pl.allegro.tech.hermes.api.Subscription; +import pl.allegro.tech.hermes.api.SubscriptionName; import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffset; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.Message; -import pl.allegro.tech.hermes.consumers.consumer.SubscriptionMetrics; import pl.allegro.tech.hermes.consumers.consumer.idletime.IdleTimeCalculator; import pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset; +import pl.allegro.tech.hermes.metrics.HermesTimer; +import pl.allegro.tech.hermes.metrics.HermesTimerContext; import java.util.Optional; import java.util.Set; @@ -16,14 +18,15 @@ public class ThrottlingMessageReceiver implements MessageReceiver { private final MessageReceiver receiver; private final IdleTimeCalculator idleTimeCalculator; - private final SubscriptionMetrics metrics; + private final HermesTimer idleTimer; public ThrottlingMessageReceiver(MessageReceiver receiver, IdleTimeCalculator idleTimeCalculator, - SubscriptionMetrics metrics) { + SubscriptionName subscriptionName, + MetricsFacade metrics) { this.receiver = receiver; this.idleTimeCalculator = idleTimeCalculator; - this.metrics = metrics; + this.idleTimer = metrics.subscriptions().consumerIdleTimer(subscriptionName); } @Override @@ -38,7 +41,7 @@ public Optional next() { } private void awaitUntilNextPoll() { - try (Timer.Context ctx = metrics.consumerIdleTimer().time()) { + try (HermesTimerContext ignored = idleTimer.time()) { Thread.sleep(idleTimeCalculator.increaseIdleTime()); } catch (InterruptedException ex) { Thread.currentThread().interrupt(); diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaMessageReceiverFactory.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaMessageReceiverFactory.java index eda7c4445a..e412be416a 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaMessageReceiverFactory.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaMessageReceiverFactory.java @@ -6,9 +6,8 @@ import pl.allegro.tech.hermes.common.kafka.ConsumerGroupId; import pl.allegro.tech.hermes.common.kafka.KafkaNamesMapper; import pl.allegro.tech.hermes.common.kafka.KafkaParameters; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.CommonConsumerParameters; -import pl.allegro.tech.hermes.consumers.consumer.SubscriptionMetrics; import pl.allegro.tech.hermes.consumers.consumer.filtering.FilteredMessageHandler; import pl.allegro.tech.hermes.consumers.consumer.idletime.ExponentiallyGrowingIdleTimeCalculator; import pl.allegro.tech.hermes.consumers.consumer.idletime.IdleTimeCalculator; @@ -60,7 +59,7 @@ public class KafkaMessageReceiverFactory implements ReceiverFactory { private final KafkaReceiverParameters consumerReceiverParameters; private final KafkaConsumerParameters kafkaConsumerParameters; private final KafkaConsumerRecordToMessageConverterFactory messageConverterFactory; - private final HermesMetrics hermesMetrics; + private final MetricsFacade metricsFacade; private final OffsetQueue offsetQueue; private final KafkaNamesMapper kafkaNamesMapper; private final FilterChainFactory filterChainFactory; @@ -72,7 +71,7 @@ public KafkaMessageReceiverFactory(CommonConsumerParameters commonConsumerParame KafkaConsumerParameters kafkaConsumerParameters, KafkaParameters kafkaAuthorizationParameters, KafkaConsumerRecordToMessageConverterFactory messageConverterFactory, - HermesMetrics hermesMetrics, + MetricsFacade metricsFacade, OffsetQueue offsetQueue, KafkaNamesMapper kafkaNamesMapper, FilterChainFactory filterChainFactory, @@ -83,7 +82,7 @@ public KafkaMessageReceiverFactory(CommonConsumerParameters commonConsumerParame this.kafkaConsumerParameters = kafkaConsumerParameters; this.kafkaAuthorizationParameters = kafkaAuthorizationParameters; this.messageConverterFactory = messageConverterFactory; - this.hermesMetrics = hermesMetrics; + this.metricsFacade = metricsFacade; this.offsetQueue = offsetQueue; this.kafkaNamesMapper = kafkaNamesMapper; this.filterChainFactory = filterChainFactory; @@ -96,12 +95,12 @@ public MessageReceiver createMessageReceiver(Topic topic, Subscription subscription, ConsumerRateLimiter consumerRateLimiter, SubscriptionLoadRecorder loadReporter, - SubscriptionMetrics metrics) { + MetricsFacade metrics) { MessageReceiver receiver = createKafkaSingleThreadedMessageReceiver(topic, subscription, loadReporter); if (consumerReceiverParameters.isWaitBetweenUnsuccessfulPolls()) { - receiver = createThrottlingMessageReceiver(receiver, metrics); + receiver = createThrottlingMessageReceiver(receiver, subscription, metrics); } if (consumerReceiverParameters.isFilteringEnabled()) { @@ -117,7 +116,7 @@ private MessageReceiver createKafkaSingleThreadedMessageReceiver(Topic topic, return new KafkaSingleThreadedMessageReceiver( createKafkaConsumer(topic, subscription), messageConverterFactory, - hermesMetrics, + metricsFacade, kafkaNamesMapper, topic, subscription, @@ -128,24 +127,28 @@ private MessageReceiver createKafkaSingleThreadedMessageReceiver(Topic topic, ); } - private MessageReceiver createThrottlingMessageReceiver(MessageReceiver receiver, SubscriptionMetrics metrics) { + private MessageReceiver createThrottlingMessageReceiver(MessageReceiver receiver, + Subscription subscription, + MetricsFacade metrics) { IdleTimeCalculator idleTimeCalculator = new ExponentiallyGrowingIdleTimeCalculator( consumerReceiverParameters.getInitialIdleTime().toMillis(), consumerReceiverParameters.getMaxIdleTime().toMillis()); - return new ThrottlingMessageReceiver(receiver, idleTimeCalculator, metrics); + return new ThrottlingMessageReceiver(receiver, idleTimeCalculator, subscription.getQualifiedName(), metrics); } private MessageReceiver createFilteringMessageReceiver(MessageReceiver receiver, ConsumerRateLimiter consumerRateLimiter, Subscription subscription, - SubscriptionMetrics metrics) { + MetricsFacade metrics) { boolean filteringRateLimitEnabled = consumerReceiverParameters.isFilteringRateLimiterEnabled(); FilteredMessageHandler filteredMessageHandler = new FilteredMessageHandler( offsetQueue, filteringRateLimitEnabled ? consumerRateLimiter : null, trackers, - metrics); + metrics, + subscription.getQualifiedName() + ); return new FilteringMessageReceiver(receiver, filteredMessageHandler, filterChainFactory, subscription); } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaSingleThreadedMessageReceiver.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaSingleThreadedMessageReceiver.java index c96b26c780..cc95936347 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaSingleThreadedMessageReceiver.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/receiver/kafka/KafkaSingleThreadedMessageReceiver.java @@ -16,7 +16,7 @@ import pl.allegro.tech.hermes.common.kafka.KafkaTopic; import pl.allegro.tech.hermes.common.kafka.KafkaTopics; import pl.allegro.tech.hermes.common.kafka.offset.PartitionOffset; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.Message; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder; import pl.allegro.tech.hermes.consumers.consumer.offset.ConsumerPartitionAssignmentState; @@ -25,6 +25,7 @@ import pl.allegro.tech.hermes.consumers.consumer.offset.kafka.broker.KafkaConsumerOffsetMover; import pl.allegro.tech.hermes.consumers.consumer.receiver.MessageReceiver; import pl.allegro.tech.hermes.consumers.consumer.receiver.RetryableReceiverError; +import pl.allegro.tech.hermes.metrics.HermesCounter; import java.time.Duration; import java.util.Collection; @@ -46,7 +47,8 @@ public class KafkaSingleThreadedMessageReceiver implements MessageReceiver { private final BlockingQueue> readQueue; private final KafkaConsumerOffsetMover offsetMover; - private final HermesMetrics metrics; + private final HermesCounter skippedCounter; + private final HermesCounter failuresCounter; private final SubscriptionLoadRecorder loadReporter; private volatile Subscription subscription; @@ -55,7 +57,7 @@ public class KafkaSingleThreadedMessageReceiver implements MessageReceiver { public KafkaSingleThreadedMessageReceiver(KafkaConsumer consumer, KafkaConsumerRecordToMessageConverterFactory messageConverterFactory, - HermesMetrics metrics, + MetricsFacade metrics, KafkaNamesMapper kafkaNamesMapper, Topic topic, Subscription subscription, @@ -63,7 +65,8 @@ public KafkaSingleThreadedMessageReceiver(KafkaConsumer consumer int readQueueCapacity, SubscriptionLoadRecorder loadReporter, ConsumerPartitionAssignmentState partitionAssignmentState) { - this.metrics = metrics; + this.skippedCounter = metrics.offsetCommits().skippedCounter(); + this.failuresCounter = metrics.offsetCommits().failuresCounter(); this.subscription = subscription; this.poolTimeout = poolTimeout; this.loadReporter = loadReporter; @@ -176,7 +179,7 @@ public void commit(Set offsets) { Thread.currentThread().interrupt(); } catch (Exception ex) { logger.error("Error while committing offset for subscription {}", subscription.getQualifiedName(), ex); - metrics.counter("offset-committer.failed").inc(); + failuresCounter.increment(); } } @@ -191,7 +194,7 @@ private Map createOffset(Set= partitionOffset.getOffset()) { offsetsData.put(topicAndPartition, new OffsetAndMetadata(partitionOffset.getOffset())); } else { - metrics.counter("offset-committer.skipped").inc(); + skippedCounter.increment(); } } else { logger.warn( diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/result/DefaultErrorHandler.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/result/DefaultErrorHandler.java index 082b349a71..ea7747d698 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/result/DefaultErrorHandler.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/result/DefaultErrorHandler.java @@ -3,14 +3,19 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.api.Subscription; +import pl.allegro.tech.hermes.api.SubscriptionName; import pl.allegro.tech.hermes.common.message.undelivered.UndeliveredMessageLog; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.Message; -import pl.allegro.tech.hermes.consumers.consumer.SubscriptionMetrics; import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue; import pl.allegro.tech.hermes.consumers.consumer.sender.MessageSendingResult; +import pl.allegro.tech.hermes.metrics.HermesCounter; +import pl.allegro.tech.hermes.metrics.HermesHistogram; import pl.allegro.tech.hermes.tracker.consumers.Trackers; import java.time.Clock; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import static pl.allegro.tech.hermes.api.SentMessageTrace.Builder.undeliveredMessage; import static pl.allegro.tech.hermes.consumers.consumer.message.MessageConverter.toMessageMetadata; @@ -21,24 +26,41 @@ public class DefaultErrorHandler implements ErrorHandler { private static final Logger logger = LoggerFactory.getLogger(DefaultErrorHandler.class); private final OffsetQueue offsetQueue; - private final SubscriptionMetrics metrics; + private final MetricsFacade metrics; private final UndeliveredMessageLog undeliveredMessageLog; private final Clock clock; private final Trackers trackers; private final String cluster; + private final SubscriptionName subscriptionName; + private final HermesCounter failures; + private final HermesCounter timeouts; + private final HermesCounter otherErrors; + private final HermesCounter discarded; + private final HermesHistogram inflightTime; + private final HermesCounter throughputInBytes; + private final Map httpStatusCodes = new ConcurrentHashMap<>(); + public DefaultErrorHandler(OffsetQueue offsetQueue, - SubscriptionMetrics metrics, + MetricsFacade metrics, UndeliveredMessageLog undeliveredMessageLog, Clock clock, Trackers trackers, - String cluster) { + String cluster, + SubscriptionName subscriptionName) { this.offsetQueue = offsetQueue; this.metrics = metrics; this.undeliveredMessageLog = undeliveredMessageLog; this.clock = clock; this.trackers = trackers; this.cluster = cluster; + this.subscriptionName = subscriptionName; + this.failures = metrics.subscriptions().failuresCounter(subscriptionName); + this.timeouts = metrics.subscriptions().timeoutsCounter(subscriptionName); + this.otherErrors = metrics.subscriptions().otherErrorsCounter(subscriptionName); + this.discarded = metrics.subscriptions().discarded(subscriptionName); + this.inflightTime = metrics.subscriptions().inflightTimeInMillisHistogram(subscriptionName); + this.throughputInBytes = metrics.subscriptions().throughputInBytes(subscriptionName); } @Override @@ -48,7 +70,8 @@ public void handleDiscarded(Message message, Subscription subscription, MessageS offsetQueue.offerCommittedOffset(subscriptionPartitionOffset(subscription.getQualifiedName(), message.getPartitionOffset(), message.getPartitionAssignmentTerm())); - metrics.markDiscarded(message); + discarded.increment(); + inflightTime.record(System.currentTimeMillis() - message.getReadingTimestamp()); addToMessageLog(message, subscription, result); @@ -85,7 +108,22 @@ private void logResult(Message message, Subscription subscription, MessageSendin @Override public void handleFailed(Message message, Subscription subscription, MessageSendingResult result) { - metrics.markFailure(message, result); + failures.increment(); + if (result.hasHttpAnswer()) { + markHttpStatusCode(result.getStatusCode()); + } else if (result.isTimeout()) { + timeouts.increment(); + } else { + otherErrors.increment(); + } + throughputInBytes.increment(message.getSize()); trackers.get(subscription).logFailed(toMessageMetadata(message, subscription), result.getRootCause(), result.getHostname()); } + + private void markHttpStatusCode(int statusCode) { + httpStatusCodes.computeIfAbsent( + statusCode, + integer -> metrics.subscriptions().httpAnswerCounter(subscriptionName, statusCode) + ).increment(); + } } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/result/DefaultSuccessHandler.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/result/DefaultSuccessHandler.java index b502bdf13d..acd52b773e 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/result/DefaultSuccessHandler.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/result/DefaultSuccessHandler.java @@ -1,32 +1,64 @@ package pl.allegro.tech.hermes.consumers.consumer.result; import pl.allegro.tech.hermes.api.Subscription; +import pl.allegro.tech.hermes.api.SubscriptionName; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.Message; -import pl.allegro.tech.hermes.consumers.consumer.SubscriptionMetrics; import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue; import pl.allegro.tech.hermes.consumers.consumer.sender.MessageSendingResult; +import pl.allegro.tech.hermes.metrics.HermesCounter; +import pl.allegro.tech.hermes.metrics.HermesHistogram; import pl.allegro.tech.hermes.tracker.consumers.Trackers; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + import static pl.allegro.tech.hermes.consumers.consumer.message.MessageConverter.toMessageMetadata; import static pl.allegro.tech.hermes.consumers.consumer.offset.SubscriptionPartitionOffset.subscriptionPartitionOffset; public class DefaultSuccessHandler implements SuccessHandler { private final Trackers trackers; + private final SubscriptionName subscriptionName; private final OffsetQueue offsetQueue; - private final SubscriptionMetrics metrics; + private final MetricsFacade metrics; + private final Map httpStatusCodes = new ConcurrentHashMap<>(); + private final HermesCounter throughputInBytes; + private final HermesCounter successes; + private final HermesHistogram inflightTime; - public DefaultSuccessHandler(OffsetQueue offsetQueue, SubscriptionMetrics metrics, Trackers trackers) { + public DefaultSuccessHandler(OffsetQueue offsetQueue, + MetricsFacade metrics, + Trackers trackers, + SubscriptionName subscriptionName) { this.offsetQueue = offsetQueue; this.metrics = metrics; this.trackers = trackers; + this.subscriptionName = subscriptionName; + this.throughputInBytes = metrics.subscriptions().throughputInBytes(subscriptionName); + this.successes = metrics.subscriptions().successes(subscriptionName); + this.inflightTime = metrics.subscriptions().inflightTimeInMillisHistogram(subscriptionName); } @Override public void handleSuccess(Message message, Subscription subscription, MessageSendingResult result) { offsetQueue.offerCommittedOffset(subscriptionPartitionOffset(subscription.getQualifiedName(), message.getPartitionOffset(), message.getPartitionAssignmentTerm())); - metrics.markSuccess(message, result); + markSuccess(message, result); trackers.get(subscription).logSent(toMessageMetadata(message, subscription), result.getHostname()); } + + private void markSuccess(Message message, MessageSendingResult result) { + successes.increment(); + throughputInBytes.increment(message.getSize()); + markHttpStatusCode(result.getStatusCode()); + inflightTime.record(System.currentTimeMillis() - message.getReadingTimestamp()); + } + + private void markHttpStatusCode(int statusCode) { + httpStatusCodes.computeIfAbsent( + statusCode, + integer -> metrics.subscriptions().httpAnswerCounter(subscriptionName, statusCode) + ).increment(); + } } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientsWorkloadReporter.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientsWorkloadReporter.java index 90bbe304af..f6d36ba6ec 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientsWorkloadReporter.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientsWorkloadReporter.java @@ -5,8 +5,7 @@ import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.client.HttpDestination; import org.eclipse.jetty.client.MultiplexConnectionPool; -import pl.allegro.tech.hermes.common.metric.Gauges; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import java.util.Queue; import java.util.function.Function; @@ -14,7 +13,7 @@ public class HttpClientsWorkloadReporter { - private final HermesMetrics metrics; + private final MetricsFacade metrics; private final HttpClient http1SerialClient; private final HttpClient http1BatchClient; private final Http2ClientHolder http2ClientHolder; @@ -22,7 +21,7 @@ public class HttpClientsWorkloadReporter { private final boolean isConnectionPoolMonitoringEnabled; public HttpClientsWorkloadReporter( - HermesMetrics metrics, + MetricsFacade metrics, HttpClient http1SerialClient, HttpClient http1BatchClient, Http2ClientHolder http2ClientHolder, @@ -47,53 +46,78 @@ public void start() { } private void registerRequestQueueSizeGauges() { - metrics.registerConsumerSenderRequestQueueSize(this::getQueuesSize); - metrics.registerConsumerSenderHttp1SerialClientRequestQueueSize(this::getHttp1SerialClientQueueSize); - metrics.registerConsumerSenderHttp1BatchClientRequestQueueSize(this::getHttp1BatchClientQueueSize); - metrics.registerConsumerSenderHttp2RequestQueueSize(this::getHttp2SerialClientQueueSize); + metrics.consumerSender() + .registerRequestQueueSizeGauge(this, HttpClientsWorkloadReporter::getQueuesSize); + metrics.consumerSender() + .registerHttp1SerialClientRequestQueueSizeGauge(this, HttpClientsWorkloadReporter::getHttp1SerialQueueSize); + metrics.consumerSender() + .registerHttp1BatchClientRequestQueueSizeGauge(this, HttpClientsWorkloadReporter::getHttp1BatchQueueSize); + metrics.consumerSender() + .registerHttp2RequestQueueSizeGauge(this, HttpClientsWorkloadReporter::getHttp2SerialQueueSize); } private void registerConnectionGauges() { - metrics.registerGauge(Gauges.CONSUMER_SENDER_HTTP_1_SERIAL_CLIENT_ACTIVE_CONNECTIONS, () -> - getHttp1ActiveConnectionsCount.apply(http1SerialClient)); - metrics.registerGauge(Gauges.CONSUMER_SENDER_HTTP_1_SERIAL_CLIENT_IDLE_CONNECTIONS, () -> - getHttp1IdleConnectionsCount.apply(http1SerialClient)); - - metrics.registerGauge(Gauges.CONSUMER_SENDER_HTTP_1_BATCH_CLIENT_ACTIVE_CONNECTIONS, () -> - getHttp1ActiveConnectionsCount.apply(http1BatchClient)); - metrics.registerGauge(Gauges.CONSUMER_SENDER_HTTP_1_BATCH_CLIENT_IDLE_CONNECTIONS, () -> - getHttp1IdleConnectionsCount.apply(http1BatchClient)); - - - metrics.registerGauge(Gauges.CONSUMER_SENDER_HTTP_2_SERIAL_CLIENT_CONNECTIONS, () -> - http2ClientHolder.getHttp2Client() - .map(getHttp2ConnectionsCount) - .orElse(0)); - metrics.registerGauge(Gauges.CONSUMER_SENDER_HTTP_2_SERIAL_CLIENT_PENDING_CONNECTIONS, () -> - http2ClientHolder.getHttp2Client() - .map(getHttp2PendingConnectionsCount) - .orElse(0)); + metrics.consumerSender() + .registerHttp1SerialClientActiveConnectionsGauge(this, HttpClientsWorkloadReporter::getHttp1SerialActiveConnections); + metrics.consumerSender() + .registerHttp1SerialClientIdleConnectionsGauge(this, HttpClientsWorkloadReporter::getHttp1SerialIdleConnections); + metrics.consumerSender() + .registerHttp1BatchClientActiveConnectionsGauge(this, HttpClientsWorkloadReporter::getHttp1BatchActiveConnections); + metrics.consumerSender() + .registerHttp1BatchClientIdleConnectionsGauge(this, HttpClientsWorkloadReporter::getHttp1BatchIdleConnections); + metrics.consumerSender() + .registerHttp2SerialClientConnectionsGauge(this, HttpClientsWorkloadReporter::getHttp2SerialConnections); + metrics.consumerSender() + .registerHttp2SerialClientPendingConnectionsGauge(this, HttpClientsWorkloadReporter::getHttp2SerialPendingConnections); } int getQueuesSize() { - return getHttp1SerialClientQueueSize() + getHttp1BatchClientQueueSize() + getHttp2SerialClientQueueSize(); + return getHttp1SerialQueueSize() + getHttp1BatchQueueSize() + getHttp2SerialQueueSize(); } - int getHttp1SerialClientQueueSize() { + int getHttp1SerialQueueSize() { return getQueueSize.apply(http1SerialClient); } - int getHttp1BatchClientQueueSize() { + int getHttp1BatchQueueSize() { return getQueueSize.apply(http1BatchClient); } - int getHttp2SerialClientQueueSize() { + int getHttp2SerialQueueSize() { return http2ClientHolder.getHttp2Client() .map(getQueueSize) .orElse(0); } + private int getHttp1SerialActiveConnections() { + return getHttp1ActiveConnectionsCount.apply(http1SerialClient); + } + + private int getHttp1SerialIdleConnections() { + return getHttp1IdleConnectionsCount.apply(http1SerialClient); + } + + private int getHttp1BatchActiveConnections() { + return getHttp1ActiveConnectionsCount.apply(http1BatchClient); + } + + private int getHttp1BatchIdleConnections() { + return getHttp1IdleConnectionsCount.apply(http1BatchClient); + } + + private int getHttp2SerialConnections() { + return http2ClientHolder.getHttp2Client() + .map(getHttp2ConnectionsCount) + .orElse(0); + } + + private int getHttp2SerialPendingConnections() { + return http2ClientHolder.getHttp2Client() + .map(getHttp2PendingConnectionsCount) + .orElse(0); + } + private final Function getQueueSize = httpClient -> httpClient.getDestinations().stream() .map(HttpDestination.class::cast) diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/queue/MonitoredMpscQueue.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/queue/MonitoredMpscQueue.java index d2bc341d14..fafc33af83 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/queue/MonitoredMpscQueue.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/queue/MonitoredMpscQueue.java @@ -3,7 +3,8 @@ import org.jctools.queues.MessagePassingQueue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; +import pl.allegro.tech.hermes.metrics.HermesCounter; public class MonitoredMpscQueue implements MpscQueue { @@ -13,20 +14,20 @@ public class MonitoredMpscQueue implements MpscQueue { private final String name; - private final HermesMetrics metrics; + private final HermesCounter failuresCounter; - public MonitoredMpscQueue(MpscQueue queue, HermesMetrics metrics, String name) { + public MonitoredMpscQueue(MpscQueue queue, MetricsFacade metrics, String name) { this.queue = queue; this.name = name; - this.metrics = metrics; - metrics.registerGauge("queue." + name + ".utilization", () -> (double) queue.size() / queue.capacity()); + metrics.consumer().registerQueueUtilizationGauge(queue, name, q -> (double) q.size() / q.capacity()); + this.failuresCounter = metrics.consumer().queueFailuresCounter(name); } @Override public boolean offer(T element) { boolean accepted = queue.offer(element); if (!accepted) { - metrics.counter("queue." + name + ".failures").inc(); + failuresCounter.increment(); logger.error("[Queue: {}] Unable to add item: queue is full. Offered item: {}", name, element); } return accepted; diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/ConsumerFactory.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/ConsumerFactory.java index 928a35a420..fc4c0334f1 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/ConsumerFactory.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/ConsumerFactory.java @@ -3,7 +3,6 @@ import pl.allegro.tech.hermes.api.Subscription; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.common.message.wrapper.CompositeMessageContentWrapper; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.CommonConsumerParameters; import pl.allegro.tech.hermes.consumers.consumer.BatchConsumer; @@ -11,7 +10,6 @@ import pl.allegro.tech.hermes.consumers.consumer.ConsumerAuthorizationHandler; import pl.allegro.tech.hermes.consumers.consumer.ConsumerMessageSenderFactory; import pl.allegro.tech.hermes.consumers.consumer.SerialConsumer; -import pl.allegro.tech.hermes.consumers.consumer.SubscriptionMetrics; import pl.allegro.tech.hermes.consumers.consumer.batch.MessageBatchFactory; import pl.allegro.tech.hermes.consumers.consumer.converter.MessageConverterResolver; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder; @@ -32,8 +30,7 @@ public class ConsumerFactory { private final ConsumerRateLimitSupervisor consumerRateLimitSupervisor; private final OutputRateCalculatorFactory outputRateCalculatorFactory; private final ReceiverFactory messageReceiverFactory; - private final HermesMetrics hermesMetrics; - private final MetricsFacade metricsFacade; + private final MetricsFacade metrics; private final CommonConsumerParameters commonConsumerParameters; private final Trackers trackers; private final OffsetQueue offsetQueue; @@ -48,8 +45,7 @@ public class ConsumerFactory { private final SubscriptionLoadRecordersRegistry subscriptionLoadRecordersRegistry; public ConsumerFactory(ReceiverFactory messageReceiverFactory, - HermesMetrics hermesMetrics, - MetricsFacade metricsFacade, + MetricsFacade metrics, CommonConsumerParameters commonConsumerParameters, ConsumerRateLimitSupervisor consumerRateLimitSupervisor, OutputRateCalculatorFactory outputRateCalculatorFactory, @@ -65,8 +61,7 @@ public ConsumerFactory(ReceiverFactory messageReceiverFactory, Clock clock, SubscriptionLoadRecordersRegistry subscriptionLoadRecordersRegistry) { this.messageReceiverFactory = messageReceiverFactory; - this.hermesMetrics = hermesMetrics; - this.metricsFacade = metricsFacade; + this.metrics = metrics; this.commonConsumerParameters = commonConsumerParameters; this.consumerRateLimitSupervisor = consumerRateLimitSupervisor; this.outputRateCalculatorFactory = outputRateCalculatorFactory; @@ -86,7 +81,6 @@ public ConsumerFactory(ReceiverFactory messageReceiverFactory, public Consumer createConsumer(Subscription subscription) { Topic topic = topicRepository.getTopicDetails(subscription.getTopicName()); SubscriptionLoadRecorder loadRecorder = subscriptionLoadRecordersRegistry.register(subscription.getQualifiedName()); - SubscriptionMetrics metrics = new SubscriptionMetrics(hermesMetrics, subscription.getQualifiedName(), metricsFacade); if (subscription.isBatchSubscription()) { return new BatchConsumer(messageReceiverFactory, batchSenderFactory.create(subscription), diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/ConsumersExecutorService.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/ConsumersExecutorService.java index 51ef4ef129..d44c145bce 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/ConsumersExecutorService.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/ConsumersExecutorService.java @@ -3,7 +3,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.supervisor.process.ConsumerProcess; import java.util.concurrent.Executors; @@ -17,14 +17,14 @@ public class ConsumersExecutorService { private static final Logger logger = LoggerFactory.getLogger(ConsumersExecutorService.class); private final ThreadPoolExecutor executor; - public ConsumersExecutorService(int poolSize, HermesMetrics hermesMetrics) { + public ConsumersExecutorService(int poolSize, MetricsFacade metrics) { ThreadFactory threadFactory = new ThreadFactoryBuilder() .setNameFormat("Consumer-%d") .setUncaughtExceptionHandler((t, e) -> logger.error("Exception from consumer with name {}", t.getName(), e)).build(); executor = (ThreadPoolExecutor) Executors.newFixedThreadPool(poolSize, threadFactory); - hermesMetrics.registerConsumersThreadGauge(executor::getActiveCount); + metrics.consumer().registerConsumerProcessesThreadsGauge(executor, ThreadPoolExecutor::getActiveCount); } public Future execute(ConsumerProcess consumer) { diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/NonblockingConsumersSupervisor.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/NonblockingConsumersSupervisor.java index ac98364719..c5dbbf8fbe 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/NonblockingConsumersSupervisor.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/NonblockingConsumersSupervisor.java @@ -6,7 +6,6 @@ import pl.allegro.tech.hermes.api.Subscription; import pl.allegro.tech.hermes.api.SubscriptionName; import pl.allegro.tech.hermes.api.Topic; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.CommonConsumerParameters; import pl.allegro.tech.hermes.consumers.consumer.offset.ConsumerPartitionAssignmentState; @@ -53,8 +52,7 @@ public NonblockingConsumersSupervisor(CommonConsumerParameters commonConsumerPar Retransmitter retransmitter, UndeliveredMessageLogPersister undeliveredMessageLogPersister, SubscriptionRepository subscriptionRepository, - HermesMetrics metrics, - MetricsFacade metricsFacade, + MetricsFacade metrics, ConsumerMonitor monitor, Clock clock, Duration commitOffsetPeriod) { @@ -77,8 +75,7 @@ public NonblockingConsumersSupervisor(CommonConsumerParameters commonConsumerPar backgroundProcess.accept(Signal.of(COMMIT, subscription, offsets.batchFor(subscription))) ), (int) commitOffsetPeriod.toSeconds(), - metrics, - metricsFacade + metrics ); monitor.register(SUBSCRIPTIONS, backgroundProcess::runningSubscriptionsStatus); monitor.register(SUBSCRIPTIONS_COUNT, backgroundProcess::countRunningProcesses); diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/monitor/ConsumersRuntimeMonitor.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/monitor/ConsumersRuntimeMonitor.java index a56963be79..5416ffc344 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/monitor/ConsumersRuntimeMonitor.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/monitor/ConsumersRuntimeMonitor.java @@ -5,7 +5,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.api.SubscriptionName; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.subscription.cache.SubscriptionsCache; import pl.allegro.tech.hermes.consumers.supervisor.ConsumersSupervisor; import pl.allegro.tech.hermes.consumers.supervisor.workload.WorkloadSupervisor; @@ -39,7 +39,7 @@ public class ConsumersRuntimeMonitor implements Runnable { public ConsumersRuntimeMonitor(ConsumersSupervisor consumerSupervisor, WorkloadSupervisor workloadSupervisor, - HermesMetrics hermesMetrics, + MetricsFacade metrics, SubscriptionsCache subscriptionsCache, Duration scanInterval) { this.consumerSupervisor = consumerSupervisor; @@ -47,10 +47,10 @@ public ConsumersRuntimeMonitor(ConsumersSupervisor consumerSupervisor, this.subscriptionsCache = subscriptionsCache; this.scanInterval = scanInterval; - hermesMetrics.registerGauge("consumers-workload.monitor.running", () -> monitorMetrics.running); - hermesMetrics.registerGauge("consumers-workload.monitor.assigned", () -> monitorMetrics.assigned); - hermesMetrics.registerGauge("consumers-workload.monitor.missing", () -> monitorMetrics.missing); - hermesMetrics.registerGauge("consumers-workload.monitor.oversubscribed", () -> monitorMetrics.oversubscribed); + metrics.workload().registerRunningSubscriptionsGauge(monitorMetrics, mm -> mm.running); + metrics.workload().registerAssignedSubscriptionsGauge(monitorMetrics, mm -> mm.assigned); + metrics.workload().registerMissingSubscriptionsGauge(monitorMetrics, mm -> mm.missing); + metrics.workload().registerOversubscribedGauge(monitorMetrics, mm -> mm.oversubscribed); } @Override diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisor.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisor.java index a03771fd19..b62825530a 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisor.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisor.java @@ -4,16 +4,19 @@ import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.api.Subscription; import pl.allegro.tech.hermes.api.SubscriptionName; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.queue.MonitoredMpscQueue; import pl.allegro.tech.hermes.consumers.queue.MpscQueue; import pl.allegro.tech.hermes.consumers.queue.WaitFreeDrainMpscQueue; import pl.allegro.tech.hermes.consumers.supervisor.ConsumersExecutorService; +import pl.allegro.tech.hermes.metrics.HermesCounter; import java.time.Clock; import java.time.Duration; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.Future; @@ -35,15 +38,19 @@ public class ConsumerProcessSupervisor implements Runnable { private final Clock clock; - private final HermesMetrics metrics; + private final MetricsFacade metrics; private final SignalsFilter signalsFilter; private final ConsumerProcessSupplier processFactory; + private final Map processedSignalsCounters = new HashMap<>(); + + private final Map droppedSignalsCounters = new HashMap<>(); + public ConsumerProcessSupervisor(ConsumersExecutorService executor, Clock clock, - HermesMetrics metrics, + MetricsFacade metrics, ConsumerProcessSupplier processFactory, int signalQueueSize, Duration backgroundSupervisorKillAfter) { @@ -56,8 +63,8 @@ public ConsumerProcessSupervisor(ConsumersExecutorService executor, this.processKiller = new ConsumerProcessKiller(backgroundSupervisorKillAfter.toMillis(), clock); this.processFactory = processFactory; - metrics.registerRunningConsumerProcessesCountGauge(runningConsumerProcesses::count); - metrics.registerDyingConsumerProcessesCountGauge(processKiller::countDying); + metrics.consumer().registerRunningConsumerProcessesGauge(runningConsumerProcesses, RunningConsumerProcesses::count); + metrics.consumer().registerDyingConsumerProcessesGauge(processKiller, ConsumerProcessKiller::countDying); } public ConsumerProcessSupervisor accept(Signal signal) { @@ -123,7 +130,10 @@ private void tryToProcessSignal(Signal signal) { private void processSignal(Signal signal) { logger.debug("Processing signal: {}", signal); - metrics.counter("supervisor.signal." + signal.getType().name()).inc(); + processedSignalsCounters.computeIfAbsent( + signal.getType().name(), + name -> metrics.consumer().processedSignalsCounter(name) + ).increment(); switch (signal.getType()) { case START: @@ -187,7 +197,10 @@ private void forRunningConsumerProcess(Signal signal, java.util.function.Consume } private void drop(Signal signal) { - metrics.counter("supervisor.signal.dropped." + signal.getType().name()).inc(); + droppedSignalsCounters.computeIfAbsent( + signal.getType().name(), + name -> metrics.consumer().droppedSignalsCounter(name) + ).increment(); logger.warn("Dropping signal {} as running target consumer process does not exist.", signal); } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/BalancingJob.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/BalancingJob.java index 1a3da3c7dd..b0cdbdb274 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/BalancingJob.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/BalancingJob.java @@ -1,14 +1,14 @@ package pl.allegro.tech.hermes.consumers.supervisor.workload; -import com.codahale.metrics.Timer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.api.SubscriptionName; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.registry.ConsumerNodesRegistry; import pl.allegro.tech.hermes.consumers.subscription.cache.SubscriptionsCache; import pl.allegro.tech.hermes.domain.workload.constraints.ConsumersWorkloadConstraints; import pl.allegro.tech.hermes.domain.workload.constraints.WorkloadConstraintsRepository; +import pl.allegro.tech.hermes.metrics.HermesTimerContext; import java.util.List; @@ -22,7 +22,7 @@ class BalancingJob implements Runnable { private final ClusterAssignmentCache clusterAssignmentCache; private final ConsumerAssignmentRegistry consumerAssignmentRegistry; private final WorkBalancer workBalancer; - private final HermesMetrics metrics; + private final MetricsFacade metrics; private final String kafkaCluster; private final WorkloadConstraintsRepository workloadConstraintsRepository; private final BalancingListener balancingListener; @@ -34,7 +34,7 @@ class BalancingJob implements Runnable { ClusterAssignmentCache clusterAssignmentCache, ConsumerAssignmentRegistry consumerAssignmentRegistry, WorkBalancer workBalancer, - HermesMetrics metrics, + MetricsFacade metrics, String kafkaCluster, WorkloadConstraintsRepository workloadConstraintsRepository, BalancingListener balancingListener) { @@ -48,26 +48,10 @@ class BalancingJob implements Runnable { this.kafkaCluster = kafkaCluster; this.workloadConstraintsRepository = workloadConstraintsRepository; this.balancingListener = balancingListener; - metrics.registerGauge( - gaugeName(kafkaCluster, ".all-assignments"), - () -> balancingMetrics.allAssignments - ); - metrics.registerGauge( - gaugeName(kafkaCluster, ".missing-resources"), - () -> balancingMetrics.missingResources - ); - metrics.registerGauge( - gaugeName(kafkaCluster, ".deleted-assignments"), - () -> balancingMetrics.deletedAssignments - ); - metrics.registerGauge( - gaugeName(kafkaCluster, ".created-assignments"), - () -> balancingMetrics.createdAssignments - ); - } - - private String gaugeName(String kafkaCluster, String name) { - return "consumers-workload." + kafkaCluster + "." + name; + metrics.workload().registerAllAssignmentsGauge(balancingMetrics, kafkaCluster, bm -> bm.allAssignments); + metrics.workload().registerMissingResourcesGauge(balancingMetrics, kafkaCluster, bm -> bm.missingResources); + metrics.workload().registerDeletedAssignmentsGauge(balancingMetrics, kafkaCluster, bm -> bm.deletedAssignments); + metrics.workload().registerCreatedAssignmentsGauge(balancingMetrics, kafkaCluster, bm -> bm.createdAssignments); } @Override @@ -75,7 +59,7 @@ public void run() { try { consumersRegistry.refresh(); if (consumersRegistry.isLeader()) { - try (Timer.Context ctx = metrics.consumersWorkloadRebalanceDurationTimer(kafkaCluster).time()) { + try (HermesTimerContext ignored = metrics.workload().rebalanceDurationTimer(kafkaCluster).time()) { logger.info("Initializing workload balance."); clusterAssignmentCache.refresh(); diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/WorkloadSupervisor.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/WorkloadSupervisor.java index 97961a591e..be74d37319 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/WorkloadSupervisor.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/WorkloadSupervisor.java @@ -8,7 +8,7 @@ import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.common.admin.AdminOperationsCallback; import pl.allegro.tech.hermes.common.admin.zookeeper.ZookeeperAdminCache; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.registry.ConsumerNodesRegistry; import pl.allegro.tech.hermes.consumers.subscription.cache.SubscriptionsCache; import pl.allegro.tech.hermes.consumers.supervisor.ConsumersSupervisor; @@ -53,7 +53,7 @@ public WorkloadSupervisor(ConsumersSupervisor supervisor, ExecutorService assignmentExecutor, WorkBalancingParameters workBalancingParameters, String kafkaClusterName, - HermesMetrics metrics, + MetricsFacade metrics, WorkloadConstraintsRepository workloadConstraintsRepository, WorkBalancer workBalancer, BalancingListener balancingListener) { diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/AvgTargetWeightCalculator.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/AvgTargetWeightCalculator.java index 49c984a2e1..3372d146c1 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/AvgTargetWeightCalculator.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/AvgTargetWeightCalculator.java @@ -7,9 +7,9 @@ public class AvgTargetWeightCalculator implements TargetWeightCalculator { - private final WeightedWorkloadMetrics metrics; + private final WeightedWorkloadMetricsReporter metrics; - public AvgTargetWeightCalculator(WeightedWorkloadMetrics metrics) { + public AvgTargetWeightCalculator(WeightedWorkloadMetricsReporter metrics) { this.metrics = metrics; } diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/ScoringTargetWeightCalculator.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/ScoringTargetWeightCalculator.java index f86fde3597..8ba1210f50 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/ScoringTargetWeightCalculator.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/ScoringTargetWeightCalculator.java @@ -17,13 +17,16 @@ public class ScoringTargetWeightCalculator implements TargetWeightCalculator { private static final double MIN_SCORE = 0.01d; private static final double MAX_SCORE = 1.0d; - private final WeightedWorkloadMetrics metrics; + private final WeightedWorkloadMetricsReporter metrics; private final Clock clock; private final Duration scoringWindowSize; private final double scoringGain; private final Map scores = new HashMap<>(); - public ScoringTargetWeightCalculator(WeightedWorkloadMetrics metrics, Clock clock, Duration scoringWindowSize, double scoringGain) { + public ScoringTargetWeightCalculator(WeightedWorkloadMetricsReporter metrics, + Clock clock, + Duration scoringWindowSize, + double scoringGain) { this.metrics = metrics; this.clock = clock; this.scoringWindowSize = scoringWindowSize; diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkBalancingListener.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkBalancingListener.java index 0d345ba9d1..3cf6c6f745 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkBalancingListener.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkBalancingListener.java @@ -21,14 +21,14 @@ public class WeightedWorkBalancingListener implements BalancingListener { private final ConsumerNodeLoadRegistry consumerNodeLoadRegistry; private final SubscriptionProfileRegistry subscriptionProfileRegistry; private final CurrentLoadProvider currentLoadProvider; - private final WeightedWorkloadMetrics weightedWorkloadMetrics; + private final WeightedWorkloadMetricsReporter weightedWorkloadMetrics; private final SubscriptionProfilesCalculator subscriptionProfilesCalculator; private final Clock clock; public WeightedWorkBalancingListener(ConsumerNodeLoadRegistry consumerNodeLoadRegistry, SubscriptionProfileRegistry subscriptionProfileRegistry, CurrentLoadProvider currentLoadProvider, - WeightedWorkloadMetrics weightedWorkloadMetrics, + WeightedWorkloadMetricsReporter weightedWorkloadMetrics, Clock clock, Duration weightWindowSize) { this.consumerNodeLoadRegistry = consumerNodeLoadRegistry; diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkloadMetrics.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkloadMetrics.java deleted file mode 100644 index 8a70d2be8b..0000000000 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkloadMetrics.java +++ /dev/null @@ -1,102 +0,0 @@ -package pl.allegro.tech.hermes.consumers.supervisor.workload.weighted; - -import com.codahale.metrics.Gauge; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; - -import java.util.Collection; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; - -import static java.util.Collections.emptySet; -import static java.util.stream.Collectors.toSet; - -public class WeightedWorkloadMetrics { - - private static final String METRICS_PREFIX = "consumer-workload.weighted."; - private static final String CONSUMER_ID_PLACEHOLDER = "$consumerId"; - private static final String CURRENT_SCORE = METRICS_PREFIX + CONSUMER_ID_PLACEHOLDER + ".current-score"; - private static final String PROPOSED_SCORE = METRICS_PREFIX + CONSUMER_ID_PLACEHOLDER + ".proposed-score"; - private static final String SCORING_ERROR = METRICS_PREFIX + CONSUMER_ID_PLACEHOLDER + ".error"; - private static final String CURRENT_WEIGHT_OPS = METRICS_PREFIX + CONSUMER_ID_PLACEHOLDER + ".current-weight.ops"; - private static final String PROPOSED_WEIGHT_OPS = METRICS_PREFIX + CONSUMER_ID_PLACEHOLDER + ".proposed-weight.ops"; - - private final HermesMetrics metrics; - - private final Map currentWeights = new ConcurrentHashMap<>(); - private final Map proposedWeights = new ConcurrentHashMap<>(); - private final Map currentScores = new ConcurrentHashMap<>(); - private final Map proposedScores = new ConcurrentHashMap<>(); - private final Map scoringErrors = new ConcurrentHashMap<>(); - - public WeightedWorkloadMetrics(HermesMetrics metrics) { - this.metrics = metrics; - } - - void reportCurrentScore(String consumerId, double score) { - registerGaugeIfNeeded(currentScores, consumerId, CURRENT_SCORE); - currentScores.put(consumerId, score); - } - - void reportProposedScore(String consumerId, double score) { - registerGaugeIfNeeded(proposedScores, consumerId, PROPOSED_SCORE); - proposedScores.put(consumerId, score); - } - - void reportScoringError(String consumerId, double error) { - registerGaugeIfNeeded(scoringErrors, consumerId, SCORING_ERROR); - scoringErrors.put(consumerId, error); - } - - void reportCurrentWeights(Collection consumers) { - for (ConsumerNode consumerNode : consumers) { - String consumerId = consumerNode.getConsumerId(); - registerGaugeIfNeeded(currentWeights, consumerId, CURRENT_WEIGHT_OPS); - currentWeights.put(consumerId, consumerNode.getWeight().getOperationsPerSecond()); - } - } - - void reportProposedWeights(Map newWeights) { - for (Map.Entry entry : newWeights.entrySet()) { - String consumerId = entry.getKey(); - registerGaugeIfNeeded(proposedWeights, consumerId, PROPOSED_WEIGHT_OPS); - proposedWeights.put(consumerId, entry.getValue().getOperationsPerSecond()); - } - } - - private void registerGaugeIfNeeded(Map currentValues, String consumerId, String metric) { - if (!currentValues.containsKey(consumerId)) { - String metricPath = buildFullMetricPath(metric, consumerId); - metrics.registerGauge(metricPath, (Gauge) () -> currentValues.getOrDefault(consumerId, 0d)); - } - } - - void unregisterLeaderMetrics() { - unregisterMetricsForConsumersOtherThan(emptySet()); - } - - void unregisterMetricsForConsumersOtherThan(Set consumerIds) { - unregisterGaugesForConsumersOtherThan(currentWeights, consumerIds, CURRENT_WEIGHT_OPS); - unregisterGaugesForConsumersOtherThan(proposedWeights, consumerIds, PROPOSED_WEIGHT_OPS); - unregisterGaugesForConsumersOtherThan(currentScores, consumerIds, CURRENT_SCORE); - unregisterGaugesForConsumersOtherThan(proposedScores, consumerIds, PROPOSED_SCORE); - unregisterGaugesForConsumersOtherThan(scoringErrors, consumerIds, SCORING_ERROR); - } - - private void unregisterGaugesForConsumersOtherThan(Map currentValues, Set consumerIds, String metric) { - Set consumerIdsToRemove = currentValues.keySet().stream() - .filter(consumerId -> !consumerIds.contains(consumerId)) - .collect(toSet()); - for (String consumerId : consumerIdsToRemove) { - if (!consumerIds.contains(consumerId)) { - String metricPath = buildFullMetricPath(metric, consumerId); - metrics.unregister(metricPath); - currentValues.remove(consumerId); - } - } - } - - private String buildFullMetricPath(String metric, String consumerId) { - return metric.replace(CONSUMER_ID_PLACEHOLDER, HermesMetrics.escapeDots(consumerId)); - } -} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkloadMetricsReporter.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkloadMetricsReporter.java new file mode 100644 index 0000000000..a5138ded40 --- /dev/null +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkloadMetricsReporter.java @@ -0,0 +1,92 @@ +package pl.allegro.tech.hermes.consumers.supervisor.workload.weighted; + +import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; +import pl.allegro.tech.hermes.common.metric.WorkloadMetrics; + +import java.util.Collection; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import static java.util.Collections.emptySet; +import static java.util.stream.Collectors.toSet; + +public class WeightedWorkloadMetricsReporter { + + private final WorkloadMetrics metrics; + private final Map currentWeights = new ConcurrentHashMap<>(); + private final Map proposedWeights = new ConcurrentHashMap<>(); + private final Map currentScores = new ConcurrentHashMap<>(); + private final Map proposedScores = new ConcurrentHashMap<>(); + private final Map scoringErrors = new ConcurrentHashMap<>(); + + public WeightedWorkloadMetricsReporter(MetricsFacade metrics) { + this.metrics = metrics.workload(); + } + + void reportCurrentScore(String consumerId, double score) { + if (!currentScores.containsKey(consumerId)) { + metrics.registerCurrentScoreGauge(consumerId, currentScores, scores -> scores.getOrDefault(consumerId, 0d)); + } + currentScores.put(consumerId, score); + } + + void reportProposedScore(String consumerId, double score) { + if (!currentScores.containsKey(consumerId)) { + metrics.registerProposedErrorGauge(consumerId, proposedScores, scores -> scores.getOrDefault(consumerId, 0d)); + } + proposedScores.put(consumerId, score); + } + + void reportScoringError(String consumerId, double error) { + if (!scoringErrors.containsKey(consumerId)) { + metrics.registerScoringErrorGauge(consumerId, scoringErrors, errors -> errors.getOrDefault(consumerId, 0d)); + } + scoringErrors.put(consumerId, error); + } + + void reportCurrentWeights(Collection consumers) { + for (ConsumerNode consumerNode : consumers) { + String consumerId = consumerNode.getConsumerId(); + if (!currentWeights.containsKey(consumerId)) { + metrics.registerCurrentWeightGauge(consumerId, currentWeights, weights -> weights.getOrDefault(consumerId, 0d)); + } + currentWeights.put(consumerId, consumerNode.getWeight().getOperationsPerSecond()); + } + } + + void reportProposedWeights(Map newWeights) { + for (Map.Entry entry : newWeights.entrySet()) { + String consumerId = entry.getKey(); + if (!proposedWeights.containsKey(consumerId)) { + metrics.registerProposedWeightGauge(consumerId, proposedWeights, weights -> weights.getOrDefault(consumerId, 0d)); + } + proposedWeights.put(consumerId, entry.getValue().getOperationsPerSecond()); + } + } + + void unregisterLeaderMetrics() { + unregisterMetricsForConsumersOtherThan(emptySet()); + } + + void unregisterMetricsForConsumersOtherThan(Set consumerIds) { + metrics.unregisterAllWorkloadWeightedGaugesForConsumerIds(findConsumerIdsToRemove(consumerIds)); + } + + private Set findConsumerIdsToRemove(Set activeIds) { + return Sets.newHashSet( + Iterables.concat( + currentScores.keySet(), + proposedScores.keySet(), + scoringErrors.keySet(), + currentWeights.keySet(), + proposedWeights.keySet() + ) + ) + .stream() + .filter(consumerId -> !activeIds.contains(consumerId)) + .collect(toSet()); + } +} diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/ZookeeperConsumerNodeLoadRegistry.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/ZookeeperConsumerNodeLoadRegistry.java index 15ca3c6b5d..e2740e2f44 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/ZookeeperConsumerNodeLoadRegistry.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/ZookeeperConsumerNodeLoadRegistry.java @@ -1,6 +1,5 @@ package pl.allegro.tech.hermes.consumers.supervisor.workload.weighted; -import com.codahale.metrics.Gauge; import com.sun.management.OperatingSystemMXBean; import org.apache.curator.framework.CuratorFramework; import org.apache.zookeeper.CreateMode; @@ -9,7 +8,7 @@ import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.api.SubscriptionName; import pl.allegro.tech.hermes.common.concurrent.ExecutorServiceFactory; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder; import pl.allegro.tech.hermes.consumers.subscription.id.SubscriptionIds; import pl.allegro.tech.hermes.infrastructure.zookeeper.ZookeeperPaths; @@ -58,7 +57,7 @@ public ZookeeperConsumerNodeLoadRegistry(CuratorFramework curator, Duration interval, ExecutorServiceFactory executorServiceFactory, Clock clock, - HermesMetrics metrics, + MetricsFacade metrics, int consumerLoadEncoderBufferSizeBytes) { this.curator = curator; this.zookeeperPaths = zookeeperPaths; @@ -70,8 +69,8 @@ public ZookeeperConsumerNodeLoadRegistry(CuratorFramework curator, this.decoder = new ConsumerNodeLoadDecoder(subscriptionIds); this.executor = executorServiceFactory.createSingleThreadScheduledExecutor("consumer-node-load-reporter-%d"); this.lastReset = clock.millis(); - metrics.registerGauge("consumer-workload.weighted.load.ops", (Gauge) () -> currentOperationsPerSecond); - metrics.registerGauge("consumer-workload.weighted.load.cpu-utilization", (Gauge) () -> cpuUtilization); + metrics.workload().registerOperationsPerSecondGauge(this, registry -> registry.currentOperationsPerSecond); + metrics.workload().registerCpuUtilizationGauge(this, registry -> registry.cpuUtilization); if (platformMXBean.getProcessCpuLoad() < 0d) { logger.warn("Process CPU load is not available."); } diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetCommitterTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetCommitterTest.groovy index 027a3aecd6..a668980afb 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetCommitterTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/offset/OffsetCommitterTest.groovy @@ -1,12 +1,8 @@ package pl.allegro.tech.hermes.consumers.consumer.offset -import com.codahale.metrics.MetricRegistry -import io.micrometer.core.instrument.simple.SimpleMeterRegistry import pl.allegro.tech.hermes.api.SubscriptionName import pl.allegro.tech.hermes.common.kafka.KafkaTopicName -import pl.allegro.tech.hermes.common.metric.HermesMetrics -import pl.allegro.tech.hermes.common.metric.MetricsFacade -import pl.allegro.tech.hermes.metrics.PathsCompiler +import pl.allegro.tech.hermes.test.helper.metrics.TestMetricsFacadeFactory import spock.lang.Shared import spock.lang.Specification @@ -18,10 +14,7 @@ class OffsetCommitterTest extends Specification { @Shared KafkaTopicName KAFKA_TOPIC_NAME = KafkaTopicName.valueOf("group_topic") - private OffsetQueue queue = new OffsetQueue( - new HermesMetrics(new MetricRegistry(), new PathsCompiler("host")), - 200_000 - ) + private OffsetQueue queue = new OffsetQueue(TestMetricsFacadeFactory.create(), 200_000) private MockMessageCommitter messageCommitter = new MockMessageCommitter() @@ -33,11 +26,7 @@ class OffsetCommitterTest extends Specification { state = new ConsumerPartitionAssignmentState() def commitInterval = 10 - def hermesMetrics = new HermesMetrics(new MetricRegistry(), new PathsCompiler("host")) - committer = new OffsetCommitter(queue, state, messageCommitter, commitInterval, - hermesMetrics, - new MetricsFacade(new SimpleMeterRegistry(), hermesMetrics) - ) + committer = new OffsetCommitter(queue, state, messageCommitter, commitInterval, TestMetricsFacadeFactory.create()) } def "should not commit offsets with negative values"() { diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/NegotiatedMaxRateProviderTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/NegotiatedMaxRateProviderTest.groovy index 50f17268d6..003e605963 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/NegotiatedMaxRateProviderTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/rate/maxrate/NegotiatedMaxRateProviderTest.groovy @@ -1,6 +1,7 @@ package pl.allegro.tech.hermes.consumers.consumer.rate.maxrate -import pl.allegro.tech.hermes.consumers.consumer.SubscriptionMetrics +import pl.allegro.tech.hermes.common.metric.MaxRateMetrics +import pl.allegro.tech.hermes.common.metric.MetricsFacade import pl.allegro.tech.hermes.consumers.consumer.rate.SendCounters import spock.lang.Specification @@ -12,20 +13,24 @@ class NegotiatedMaxRateProviderTest extends Specification { def maxRateRegistry = Mock(MaxRateRegistry) def maxRateSupervisor = Mock(MaxRateSupervisor) - def hermesMetrics = Mock(SubscriptionMetrics) + def metrics = Mock(MetricsFacade) + def maxRateMetrics = Mock(MaxRateMetrics) def sendCounters = Mock(SendCounters) def subscription = subscription("group.topic", "subscription").build() def consumer = new ConsumerInstance("consumer", subscription.getQualifiedName()); - def freshProvider = createProvider() - def initializedProvider = createProvider() + def freshProvider + def initializedProvider def setup() { + metrics.maxRate() >> maxRateMetrics 1 * sendCounters.getRate() >> 0.5 1 * maxRateRegistry.getRateHistory(consumer) >> RateHistory.empty() maxRateRegistry.getMaxRate(consumer) >> Optional.empty() + freshProvider = createProvider() + initializedProvider = createProvider() initializedProvider.tickForHistory() } @@ -96,7 +101,7 @@ class NegotiatedMaxRateProviderTest extends Specification { maxRateSupervisor, subscription, sendCounters, - hermesMetrics, + metrics, 1d, 0.1d, HISTORY_SIZE diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/receiver/KafkaSingleThreadedMessageReceiverTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/receiver/KafkaSingleThreadedMessageReceiverTest.groovy index 6b88695335..c051fd32b1 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/receiver/KafkaSingleThreadedMessageReceiverTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/receiver/KafkaSingleThreadedMessageReceiverTest.groovy @@ -6,7 +6,8 @@ import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.common.TopicPartition import pl.allegro.tech.hermes.common.kafka.KafkaNamesMapper import pl.allegro.tech.hermes.common.kafka.NamespaceKafkaNamesMapper -import pl.allegro.tech.hermes.common.metric.HermesMetrics +import pl.allegro.tech.hermes.common.metric.MetricsFacade +import pl.allegro.tech.hermes.common.metric.OffsetCommitsMetrics import pl.allegro.tech.hermes.consumers.consumer.Message import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder import pl.allegro.tech.hermes.consumers.consumer.offset.ConsumerPartitionAssignmentState @@ -31,10 +32,14 @@ class KafkaSingleThreadedMessageReceiverTest extends Specification { def topic = TopicBuilder.topic("pl.allegro.someTestTopic").build() def subscription = SubscriptionBuilder.subscription(topic, "someSub").build() + MetricsFacade metricsFacade = Mock(MetricsFacade) + OffsetCommitsMetrics offsetCommitsMetrics = Mock(OffsetCommitsMetrics) + def setup() { + metricsFacade.offsetCommits() >> offsetCommitsMetrics converterFactory.create(*_) >> messageConverter receiver = new KafkaSingleThreadedMessageReceiver( - consumer, converterFactory, Mock(HermesMetrics), + consumer, converterFactory, metricsFacade, kafkaNamesMapper, topic, subscription, Duration.ofMillis(10), 10, Mock(SubscriptionLoadRecorder), Mock(ConsumerPartitionAssignmentState) diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/result/DefaultErrorHandlerTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/result/DefaultErrorHandlerTest.groovy index 51ce09ce25..ab5bc5225f 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/result/DefaultErrorHandlerTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/result/DefaultErrorHandlerTest.groovy @@ -1,16 +1,14 @@ package pl.allegro.tech.hermes.consumers.consumer.result -import com.codahale.metrics.MetricRegistry import pl.allegro.tech.hermes.api.Subscription import pl.allegro.tech.hermes.api.TrackingMode import pl.allegro.tech.hermes.common.message.undelivered.UndeliveredMessageLog -import pl.allegro.tech.hermes.common.metric.HermesMetrics +import pl.allegro.tech.hermes.common.metric.MetricsFacade import pl.allegro.tech.hermes.consumers.consumer.Message -import pl.allegro.tech.hermes.consumers.consumer.SubscriptionMetrics import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue import pl.allegro.tech.hermes.consumers.consumer.sender.MessageSendingResult import pl.allegro.tech.hermes.consumers.test.MessageBuilder -import pl.allegro.tech.hermes.metrics.PathsCompiler +import pl.allegro.tech.hermes.test.helper.metrics.TestMetricsFacadeFactory import pl.allegro.tech.hermes.tracker.consumers.Trackers import spock.lang.Specification @@ -20,10 +18,7 @@ import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.sub class DefaultErrorHandlerTest extends Specification { - private OffsetQueue offsetQueue = new OffsetQueue( - new HermesMetrics(new MetricRegistry(), new PathsCompiler("host")), - 200_000 - ) + private OffsetQueue offsetQueue = new OffsetQueue(TestMetricsFacadeFactory.create(), 200_000) private UndeliveredMessageLog undeliveredLog = Mock(UndeliveredMessageLog) @@ -35,7 +30,7 @@ class DefaultErrorHandlerTest extends Specification { .withTrackingMode(TrackingMode.TRACK_ALL).build() private DefaultErrorHandler handler = new DefaultErrorHandler( - offsetQueue, Stub(SubscriptionMetrics), undeliveredLog, Clock.systemUTC(), trackers, "cluster") + offsetQueue, Stub(MetricsFacade), undeliveredLog, Clock.systemUTC(), trackers, "cluster", subscription.qualifiedName) def "should save tracking information on message failure but not commit message"() { given: diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/result/DefaultSuccessHandlerTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/result/DefaultSuccessHandlerTest.groovy index 4d6817118e..68241e20f0 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/result/DefaultSuccessHandlerTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/result/DefaultSuccessHandlerTest.groovy @@ -1,15 +1,13 @@ package pl.allegro.tech.hermes.consumers.consumer.result -import com.codahale.metrics.MetricRegistry import pl.allegro.tech.hermes.api.Subscription import pl.allegro.tech.hermes.api.TrackingMode -import pl.allegro.tech.hermes.common.metric.HermesMetrics +import pl.allegro.tech.hermes.common.metric.MetricsFacade import pl.allegro.tech.hermes.consumers.consumer.Message -import pl.allegro.tech.hermes.consumers.consumer.SubscriptionMetrics import pl.allegro.tech.hermes.consumers.consumer.offset.OffsetQueue import pl.allegro.tech.hermes.consumers.consumer.sender.MessageSendingResult import pl.allegro.tech.hermes.consumers.test.MessageBuilder -import pl.allegro.tech.hermes.metrics.PathsCompiler +import pl.allegro.tech.hermes.test.helper.metrics.TestMetricsFacadeFactory import pl.allegro.tech.hermes.tracker.consumers.Trackers import spock.lang.Specification @@ -17,10 +15,7 @@ import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.sub class DefaultSuccessHandlerTest extends Specification { - private OffsetQueue offsetQueue = new OffsetQueue( - new HermesMetrics(new MetricRegistry(), new PathsCompiler("host")), - 200_000 - ) + private OffsetQueue offsetQueue = new OffsetQueue(TestMetricsFacadeFactory.create(), 200_000) private InMemoryLogRepository sendingTracker = new InMemoryLogRepository() @@ -29,7 +24,7 @@ class DefaultSuccessHandlerTest extends Specification { private Subscription subscription = subscription('group.topic', 'subscription') .withTrackingMode(TrackingMode.TRACK_ALL).build() - private DefaultSuccessHandler handler = new DefaultSuccessHandler(offsetQueue, Stub(SubscriptionMetrics), trackers) + private DefaultSuccessHandler handler = new DefaultSuccessHandler(offsetQueue, Stub(MetricsFacade), trackers, subscription.qualifiedName) def "should commit message and save tracking information on message success"() { given: diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientConnectionMonitoringTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientConnectionMonitoringTest.groovy index 35f4ad98c6..e6ad62185c 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientConnectionMonitoringTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientConnectionMonitoringTest.groovy @@ -2,6 +2,8 @@ package pl.allegro.tech.hermes.consumers.consumer.sender.http import com.codahale.metrics.MetricRegistry import com.github.tomakehurst.wiremock.WireMockServer +import io.micrometer.core.instrument.MeterRegistry +import io.micrometer.core.instrument.search.Search import io.micrometer.core.instrument.simple.SimpleMeterRegistry import org.eclipse.jetty.client.HttpClient import pl.allegro.tech.hermes.common.metric.HermesMetrics @@ -28,8 +30,9 @@ class HttpClientConnectionMonitoringTest extends Specification { HttpClient batchClient MetricRegistry metricRegistry = new MetricRegistry() HermesMetrics hermesMetrics = new HermesMetrics(metricRegistry, new PathsCompiler("localhost")) - MetricsFacade metricsFacade = new MetricsFacade(new SimpleMeterRegistry(), hermesMetrics) - ThreadPoolMetrics threadPoolMetrics = new ThreadPoolMetrics(metricsFacade) + MeterRegistry meterRegistry = new SimpleMeterRegistry() + MetricsFacade metrics = new MetricsFacade(meterRegistry, hermesMetrics) + ThreadPoolMetrics threadPoolMetrics = new ThreadPoolMetrics(metrics) def setupSpec() { port = Ports.nextAvailable() @@ -51,28 +54,32 @@ class HttpClientConnectionMonitoringTest extends Specification { def "should measure http client connections"() { given: - def reporter = new HttpClientsWorkloadReporter(hermesMetrics, client, batchClient, new Http2ClientHolder(null), false, true) + def reporter = new HttpClientsWorkloadReporter(metrics, client, batchClient, new Http2ClientHolder(null), false, true) reporter.start() when: client.POST("http://localhost:${port}/hello").send() and: - def idle = metricRegistry.gauges['http-clients.serial.http1.idle-connections'].value - def active = metricRegistry.gauges['http-clients.serial.http1.active-connections'].value + def idleDropwizard = metricRegistry.gauges['http-clients.serial.http1.idle-connections'].value + def activeDropwizard = metricRegistry.gauges['http-clients.serial.http1.active-connections'].value + def idleMicrometer = Search.in(meterRegistry).name("http-clients.serial.http1.idle-connections").gauge().value() + def activeMicrometer = Search.in(meterRegistry).name("http-clients.serial.http1.active-connections").gauge().value() then: - idle + active > 0 + idleDropwizard + activeDropwizard > 0 + idleMicrometer + activeMicrometer > 0 } def "should not register connection gauges for disabled http connection monitoring"() { given: - def reporter = new HttpClientsWorkloadReporter(hermesMetrics, client, batchClient, new Http2ClientHolder(null), false, false) + def reporter = new HttpClientsWorkloadReporter(metrics, client, batchClient, new Http2ClientHolder(null), false, false) when: reporter.start() then: metricRegistry.gauges.size() == 0 + Search.in(meterRegistry).gauges().size() == 0 } } diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientsWorkloadReporterTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientsWorkloadReporterTest.groovy index 272f4c661c..e573c84281 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientsWorkloadReporterTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/consumer/sender/http/HttpClientsWorkloadReporterTest.groovy @@ -3,7 +3,7 @@ package pl.allegro.tech.hermes.consumers.consumer.sender.http import org.eclipse.jetty.client.HttpClient import org.eclipse.jetty.client.HttpDestination import org.eclipse.jetty.client.HttpExchange -import pl.allegro.tech.hermes.common.metric.HermesMetrics +import pl.allegro.tech.hermes.common.metric.MetricsFacade import spock.lang.Specification import java.util.concurrent.LinkedBlockingQueue @@ -33,7 +33,7 @@ class HttpClientsWorkloadReporterTest extends Specification { http2Client.getDestinations() >> [http2Destination] def reporter = new HttpClientsWorkloadReporter( - Mock(HermesMetrics), + Mock(MetricsFacade), http1Client, http1BatchClient, new Http2ClientHolder(http2Client), @@ -42,9 +42,9 @@ class HttpClientsWorkloadReporterTest extends Specification { expect: reporter.queuesSize == 8 - reporter.http1SerialClientQueueSize == 3 - reporter.http1BatchClientQueueSize == 2 - reporter.http2SerialClientQueueSize == 3 + reporter.http1SerialQueueSize == 3 + reporter.http1BatchQueueSize == 2 + reporter.http2SerialQueueSize == 3 } def "should return sum of http/1 serial client destinations"() { @@ -60,12 +60,12 @@ class HttpClientsWorkloadReporterTest extends Specification { def http1BatchClient = Mock(HttpClient) http1BatchClient.getDestinations() >> [] - def reporter = new HttpClientsWorkloadReporter(Mock(HermesMetrics), http1Client, http1BatchClient, new Http2ClientHolder(null), true, false) + def reporter = new HttpClientsWorkloadReporter(Mock(MetricsFacade), http1Client, http1BatchClient, new Http2ClientHolder(null), true, false) expect: reporter.queuesSize == 3 - reporter.http1SerialClientQueueSize == 3 - reporter.http1BatchClientQueueSize == 0 - reporter.http2SerialClientQueueSize == 0 + reporter.http1SerialQueueSize == 3 + reporter.http1BatchQueueSize == 0 + reporter.http2SerialQueueSize == 0 } } diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisorTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisorTest.groovy index e661d307b6..30df929486 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisorTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/process/ConsumerProcessSupervisorTest.groovy @@ -3,11 +3,15 @@ package pl.allegro.tech.hermes.consumers.supervisor.process import com.codahale.metrics.MetricRegistry import com.jayway.awaitility.Awaitility import com.jayway.awaitility.core.ConditionFactory +import io.micrometer.core.instrument.MeterRegistry +import io.micrometer.core.instrument.search.Search +import io.micrometer.core.instrument.simple.SimpleMeterRegistry import pl.allegro.tech.hermes.api.DeliveryType import pl.allegro.tech.hermes.api.Subscription import pl.allegro.tech.hermes.api.SubscriptionName import pl.allegro.tech.hermes.api.Topic import pl.allegro.tech.hermes.common.metric.HermesMetrics +import pl.allegro.tech.hermes.common.metric.MetricsFacade import pl.allegro.tech.hermes.consumers.config.CommonConsumerProperties import pl.allegro.tech.hermes.consumers.supervisor.ConsumersExecutorService import pl.allegro.tech.hermes.metrics.PathsCompiler @@ -47,7 +51,9 @@ class ConsumerProcessSupervisorTest extends Specification { ] ConsumerProcessSupervisor supervisor - HermesMetrics metrics + MeterRegistry meterRegistry = new SimpleMeterRegistry() + MetricRegistry metricRegistry = new MetricRegistry() + MetricsFacade metrics ConsumerStub consumer Clock clock @@ -64,7 +70,10 @@ class ConsumerProcessSupervisorTest extends Specification { return new ConsumerProcess(startSignal, consumer, Stub(Retransmitter), clock, unhealthyAfter, onConsumerStopped) } - metrics = new HermesMetrics(new MetricRegistry(), new PathsCompiler("localhost")) + metrics = new MetricsFacade( + meterRegistry, + new HermesMetrics(metricRegistry, new PathsCompiler("localhost")) + ) supervisor = new ConsumerProcessSupervisor( new ConsumersExecutorService(new CommonConsumerProperties().getThreadPoolSize(), metrics), @@ -166,7 +175,13 @@ class ConsumerProcessSupervisorTest extends Specification { then: signalsToDrop.forEach { - assert metrics.counter("supervisor.signal.dropped." + it.type.name()).getCount() == 1 + String signal = it.type.name() + assert metricRegistry.counter("supervisor.signal.dropped." + signal).getCount() == 1 + assert Search.in(meterRegistry) + .name {it.startsWith("signals.dropped")} + .tag("signal", signal) + .counters() + .size() == 1 } } diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/ScoringTargetWeightCalculatorTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/ScoringTargetWeightCalculatorTest.groovy index fb7ab66dc5..fba376a1e7 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/ScoringTargetWeightCalculatorTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/ScoringTargetWeightCalculatorTest.groovy @@ -1,9 +1,7 @@ package pl.allegro.tech.hermes.consumers.supervisor.workload.weighted -import com.codahale.metrics.MetricRegistry import pl.allegro.tech.hermes.api.SubscriptionName -import pl.allegro.tech.hermes.common.metric.HermesMetrics -import pl.allegro.tech.hermes.metrics.PathsCompiler +import pl.allegro.tech.hermes.test.helper.metrics.TestMetricsFacadeFactory import spock.lang.Specification import spock.lang.Subject @@ -13,12 +11,11 @@ import java.time.Instant class ScoringTargetWeightCalculatorTest extends Specification { - def hermesMetrics = new HermesMetrics(new MetricRegistry(), new PathsCompiler("host")) def scoringGain = 1.0d @Subject def calculator = new ScoringTargetWeightCalculator( - new WeightedWorkloadMetrics(hermesMetrics), + new WeightedWorkloadMetricsReporter(TestMetricsFacadeFactory.create()), Clock.systemDefaultZone(), Duration.ofMinutes(15), scoringGain diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkBalancerTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkBalancerTest.groovy index 4af44c4073..7c4ed0448e 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkBalancerTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkBalancerTest.groovy @@ -1,13 +1,12 @@ package pl.allegro.tech.hermes.consumers.supervisor.workload.weighted -import com.codahale.metrics.MetricRegistry import pl.allegro.tech.hermes.api.Constraints import pl.allegro.tech.hermes.api.SubscriptionName import pl.allegro.tech.hermes.api.TopicName -import pl.allegro.tech.hermes.common.metric.HermesMetrics +import pl.allegro.tech.hermes.common.metric.MetricsFacade import pl.allegro.tech.hermes.consumers.supervisor.workload.SubscriptionAssignmentViewBuilder import pl.allegro.tech.hermes.consumers.supervisor.workload.WorkloadConstraints -import pl.allegro.tech.hermes.metrics.PathsCompiler +import pl.allegro.tech.hermes.test.helper.metrics.TestMetricsFacadeFactory import pl.allegro.tech.hermes.test.helper.time.ModifiableClock import spock.lang.Specification @@ -633,8 +632,8 @@ class WeightedWorkBalancerTest extends Specification { SubscriptionProfiles subscriptionProfiles) { CurrentLoadProvider currentLoadProvider = new CurrentLoadProvider() currentLoadProvider.updateProfiles(subscriptionProfiles) - HermesMetrics hermesMetrics = new HermesMetrics(new MetricRegistry(), new PathsCompiler("host")) - WeightedWorkloadMetrics workloadMetrics = new WeightedWorkloadMetrics(hermesMetrics) + MetricsFacade metrics = TestMetricsFacadeFactory.create() + WeightedWorkloadMetricsReporter workloadMetrics = new WeightedWorkloadMetricsReporter(metrics) return new WeightedWorkBalancer( clock, stabilizationWindowSize, diff --git a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkBalancingListenerTest.groovy b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkBalancingListenerTest.groovy index f1575e7cb8..ced5eaa583 100644 --- a/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkBalancingListenerTest.groovy +++ b/hermes-consumers/src/test/groovy/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/WeightedWorkBalancingListenerTest.groovy @@ -2,8 +2,11 @@ package pl.allegro.tech.hermes.consumers.supervisor.workload.weighted import com.codahale.metrics.MetricFilter import com.codahale.metrics.MetricRegistry +import io.micrometer.core.instrument.search.Search +import io.micrometer.core.instrument.simple.SimpleMeterRegistry import pl.allegro.tech.hermes.api.SubscriptionName import pl.allegro.tech.hermes.common.metric.HermesMetrics +import pl.allegro.tech.hermes.common.metric.MetricsFacade import pl.allegro.tech.hermes.consumers.supervisor.workload.WorkDistributionChanges import pl.allegro.tech.hermes.metrics.PathsCompiler import pl.allegro.tech.hermes.test.helper.time.ModifiableClock @@ -23,7 +26,13 @@ class WeightedWorkBalancingListenerTest extends Specification { def weightWindowSize = Duration.ofMinutes(1) def currentLoadProvider = new CurrentLoadProvider() def metricsRegistry = new MetricRegistry() - def metrics = new WeightedWorkloadMetrics(new HermesMetrics(metricsRegistry, new PathsCompiler("host"))) + def meterRegistry = new SimpleMeterRegistry() + def metrics = new WeightedWorkloadMetricsReporter( + new MetricsFacade( + meterRegistry, + new HermesMetrics(metricsRegistry, new PathsCompiler("host")) + ) + ) @Subject def listener = new WeightedWorkBalancingListener( @@ -173,6 +182,8 @@ class WeightedWorkBalancingListenerTest extends Specification { then: metricsRegistry.getGauges(MetricFilter.contains(".c2.")).size() == 1 metricsRegistry.getGauges(MetricFilter.contains(".c1.")).size() == 0 + Search.in(meterRegistry).tag("consumer-id", "c2").gauges().size() == 1 + Search.in(meterRegistry).tag("consumer-id", "c1").gauges().size() == 0 } def "should unregister workload metrics when the consumer is no longer a leader"() { @@ -185,6 +196,7 @@ class WeightedWorkBalancingListenerTest extends Specification { then: metricsRegistry.getGauges().size() == 0 + Search.in(meterRegistry).gauges().size() == 0 } private static SubscriptionName subscription(String name) { diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java index 5b98980678..5c1818aacc 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/ConsumerMessageSenderTest.java @@ -1,18 +1,13 @@ package pl.allegro.tech.hermes.consumers.consumer; -import com.codahale.metrics.Meter; -import com.codahale.metrics.Timer; -import io.micrometer.core.instrument.simple.SimpleMeterRegistry; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; import pl.allegro.tech.hermes.api.Subscription; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; -import pl.allegro.tech.hermes.common.metric.Meters; import pl.allegro.tech.hermes.common.metric.MetricsFacade; -import pl.allegro.tech.hermes.common.metric.Timers; +import pl.allegro.tech.hermes.common.metric.SubscriptionMetrics; import pl.allegro.tech.hermes.consumers.consumer.rate.AdjustableSemaphore; import pl.allegro.tech.hermes.consumers.consumer.rate.SerialConsumerRateLimiter; import pl.allegro.tech.hermes.consumers.consumer.result.ErrorHandler; @@ -23,6 +18,9 @@ import pl.allegro.tech.hermes.consumers.consumer.sender.timeout.FutureAsyncTimeout; import pl.allegro.tech.hermes.consumers.supervisor.workload.weighted.NoOpConsumerNodeLoadRegistry; import pl.allegro.tech.hermes.consumers.test.MessageBuilder; +import pl.allegro.tech.hermes.metrics.HermesCounter; +import pl.allegro.tech.hermes.metrics.HermesTimer; +import pl.allegro.tech.hermes.metrics.HermesTimerContext; import pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder; import java.nio.charset.StandardCharsets; @@ -69,29 +67,30 @@ public class ConsumerMessageSenderTest { private SerialConsumerRateLimiter rateLimiter; @Mock - private HermesMetrics hermesMetrics; + private HermesTimer consumerLatencyTimer; @Mock - private Timer consumerLatencyTimer; + private HermesTimerContext consumerLatencyTimerContext; @Mock - private Timer.Context consumerLatencyTimerContext; + private HermesCounter failedMeter; @Mock - private Meter failedMeter; - - @Mock - private Meter errors; + private HermesCounter errors; private AdjustableSemaphore inflightSemaphore; private ConsumerMessageSender sender; + @Mock + private SubscriptionMetrics subscriptionMetrics; + + @Mock private MetricsFacade metricsFacade; @Before public void setUp() { - metricsFacade = new MetricsFacade(new SimpleMeterRegistry(), hermesMetrics); + when(metricsFacade.subscriptions()).thenReturn(subscriptionMetrics); setUpMetrics(subscription); setUpMetrics(subscriptionWith4xxRetry); inflightSemaphore = new AdjustableSemaphore(0); @@ -99,12 +98,10 @@ public void setUp() { } private void setUpMetrics(Subscription subscription) { - when(hermesMetrics.timer(Timers.SUBSCRIPTION_LATENCY, subscription.getTopicName(), subscription.getName())) - .thenReturn(consumerLatencyTimer); - when(hermesMetrics.consumerErrorsOtherMeter(subscription.getQualifiedName())).thenReturn(errors); + when(metricsFacade.subscriptions().latency(subscription.getQualifiedName())).thenReturn(consumerLatencyTimer); + when(metricsFacade.subscriptions().otherErrorsCounter(subscription.getQualifiedName())).thenReturn(errors); when(consumerLatencyTimer.time()).thenReturn(consumerLatencyTimerContext); - when(hermesMetrics.meter(Meters.FAILED_METER_SUBSCRIPTION, subscription.getTopicName(), subscription.getName())) - .thenReturn(failedMeter); + when(metricsFacade.subscriptions().failuresCounter(subscription.getQualifiedName())).thenReturn(failedMeter); } @Test @@ -398,7 +395,7 @@ private ConsumerMessageSender consumerMessageSender(Subscription subscription) { rateLimiter, Executors.newSingleThreadExecutor(), () -> inflightSemaphore.release(), - new SubscriptionMetrics(hermesMetrics, subscription.getQualifiedName(), metricsFacade), + metricsFacade, ASYNC_TIMEOUT_MS, new FutureAsyncTimeout(Executors.newSingleThreadScheduledExecutor()), Clock.systemUTC(), @@ -422,7 +419,7 @@ private void verifyErrorHandlerHandleDiscarded(Message message, Subscription sub } private void verifyLatencyTimersCountedTimes(Subscription subscription, int timeCount, int closeCount) { - verify(hermesMetrics, times(1)).timer(Timers.SUBSCRIPTION_LATENCY, subscription.getTopicName(), subscription.getName()); + verify(metricsFacade.subscriptions(), times(1)).latency(subscription.getQualifiedName()); verify(consumerLatencyTimer, times(timeCount)).time(); verify(consumerLatencyTimerContext, times(closeCount)).close(); } diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyMessageSenderTest.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyMessageSenderTest.java index cda57968c6..c473a4ffb0 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyMessageSenderTest.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/consumer/sender/http/JettyMessageSenderTest.java @@ -1,8 +1,6 @@ package pl.allegro.tech.hermes.consumers.consumer.sender.http; -import com.codahale.metrics.MetricRegistry; import com.github.tomakehurst.wiremock.WireMockServer; -import io.micrometer.core.instrument.simple.SimpleMeterRegistry; import org.eclipse.jetty.client.HttpClient; import org.junit.AfterClass; import org.junit.Before; @@ -10,8 +8,6 @@ import org.junit.Test; import pl.allegro.tech.hermes.api.EndpointAddress; import pl.allegro.tech.hermes.api.EndpointAddressResolverMetadata; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; -import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.common.metric.executor.InstrumentedExecutorServiceFactory; import pl.allegro.tech.hermes.common.metric.executor.ThreadPoolMetrics; import pl.allegro.tech.hermes.consumers.config.ConsumerSenderConfiguration; @@ -26,8 +22,8 @@ import pl.allegro.tech.hermes.consumers.consumer.sender.resolver.ResolvableEndpointAddress; import pl.allegro.tech.hermes.consumers.consumer.sender.resolver.SimpleEndpointAddressResolver; import pl.allegro.tech.hermes.consumers.test.MessageBuilder; -import pl.allegro.tech.hermes.metrics.PathsCompiler; import pl.allegro.tech.hermes.test.helper.endpoint.RemoteServiceEndpoint; +import pl.allegro.tech.hermes.test.helper.metrics.TestMetricsFacadeFactory; import pl.allegro.tech.hermes.test.helper.util.Ports; import java.util.Collections; @@ -69,15 +65,7 @@ public static void setupEnvironment() throws Exception { client = consumerConfiguration.http1SerialClient( new HttpClientsFactory( new InstrumentedExecutorServiceFactory( - new ThreadPoolMetrics( - new MetricsFacade( - new SimpleMeterRegistry(), - new HermesMetrics( - new MetricRegistry(), - new PathsCompiler("localhost") - ) - ) - ) + new ThreadPoolMetrics(TestMetricsFacadeFactory.create()) ), sslContextFactoryProvider), new Http1ClientProperties() diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/ConsumerTestRuntimeEnvironment.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/ConsumerTestRuntimeEnvironment.java index 931445b764..b6965869f5 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/ConsumerTestRuntimeEnvironment.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/ConsumerTestRuntimeEnvironment.java @@ -1,9 +1,7 @@ package pl.allegro.tech.hermes.consumers.supervisor.workload; -import com.codahale.metrics.MetricRegistry; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Maps; -import io.micrometer.core.instrument.simple.SimpleMeterRegistry; import org.apache.curator.framework.CuratorFramework; import pl.allegro.tech.hermes.api.Group; import pl.allegro.tech.hermes.api.Subscription; @@ -11,7 +9,6 @@ import pl.allegro.tech.hermes.common.admin.zookeeper.ZookeeperAdminCache; import pl.allegro.tech.hermes.common.di.factories.ModelAwareZookeeperNotifyingCacheFactory; import pl.allegro.tech.hermes.common.exception.InternalProcessingException; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.consumers.config.CommonConsumerProperties; import pl.allegro.tech.hermes.consumers.config.KafkaProperties; @@ -47,7 +44,7 @@ import pl.allegro.tech.hermes.infrastructure.zookeeper.ZookeeperWorkloadConstraintsRepository; import pl.allegro.tech.hermes.infrastructure.zookeeper.cache.ModelAwareZookeeperNotifyingCache; import pl.allegro.tech.hermes.infrastructure.zookeeper.notifications.ZookeeperInternalNotificationBus; -import pl.allegro.tech.hermes.metrics.PathsCompiler; +import pl.allegro.tech.hermes.test.helper.metrics.TestMetricsFacadeFactory; import java.time.Clock; import java.time.Duration; @@ -86,8 +83,7 @@ class ConsumerTestRuntimeEnvironment { private final SubscriptionRepository subscriptionRepository; private final ObjectMapper objectMapper = new ObjectMapper(); private final ExecutorService executorService = Executors.newSingleThreadExecutor(); - private final Supplier metricsSupplier; - private final Supplier metricsFacadeSupplier; + private final Supplier metricsSupplier; private final WorkloadConstraintsRepository workloadConstraintsRepository; private final CuratorFramework curator; private final ConsumerPartitionAssignmentState partitionAssignmentState; @@ -108,8 +104,7 @@ class ConsumerTestRuntimeEnvironment { this.workloadConstraintsRepository = new ZookeeperWorkloadConstraintsRepository(curator, objectMapper, zookeeperPaths); - this.metricsSupplier = () -> new HermesMetrics(new MetricRegistry(), new PathsCompiler("localhost")); - this.metricsFacadeSupplier = () -> new MetricsFacade(new SimpleMeterRegistry(), this.metricsSupplier.get()); + this.metricsSupplier = TestMetricsFacadeFactory::create; this.nodesRegistryPaths = new ConsumerNodesRegistryPaths(zookeeperPaths, kafkaProperties.getClusterName()); this.zookeeperProperties = new ZookeeperProperties(); } @@ -219,8 +214,7 @@ WorkloadSupervisor spawnConsumer(String consumerId, ConsumersSupervisor consumer } ConsumersSupervisor consumersSupervisor(ConsumerFactory consumerFactory) { - HermesMetrics metrics = metricsSupplier.get(); - MetricsFacade metricsFacade = metricsFacadeSupplier.get(); + MetricsFacade metrics = metricsSupplier.get(); CommonConsumerProperties commonConsumerProperties = new CommonConsumerProperties(); CommonConsumerProperties.BackgroundSupervisor supervisorParameters = new CommonConsumerProperties.BackgroundSupervisor(); supervisorParameters.setInterval(Duration.ofSeconds(1)); @@ -234,7 +228,6 @@ ConsumersSupervisor consumersSupervisor(ConsumerFactory consumerFactory) { mock(UndeliveredMessageLogPersister.class), subscriptionRepository, metrics, - metricsFacade, mock(ConsumerMonitor.class), Clock.systemDefaultZone(), Duration.ofSeconds(60)); diff --git a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/ZookeeperConsumerNodeLoadRegistryTest.java b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/ZookeeperConsumerNodeLoadRegistryTest.java index 2e37399d3c..5803f557d7 100644 --- a/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/ZookeeperConsumerNodeLoadRegistryTest.java +++ b/hermes-consumers/src/test/java/pl/allegro/tech/hermes/consumers/supervisor/workload/weighted/ZookeeperConsumerNodeLoadRegistryTest.java @@ -1,19 +1,17 @@ package pl.allegro.tech.hermes.consumers.supervisor.workload.weighted; -import com.codahale.metrics.MetricRegistry; import org.junit.After; import org.junit.Before; import org.junit.Test; import pl.allegro.tech.hermes.api.SubscriptionName; -import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.consumers.consumer.load.SubscriptionLoadRecorder; import pl.allegro.tech.hermes.consumers.subscription.id.SubscriptionId; import pl.allegro.tech.hermes.consumers.subscription.id.SubscriptionIds; import pl.allegro.tech.hermes.consumers.supervisor.workload.TestSubscriptionIds; import pl.allegro.tech.hermes.infrastructure.zookeeper.ZookeeperPaths; -import pl.allegro.tech.hermes.metrics.PathsCompiler; import pl.allegro.tech.hermes.test.helper.concurrent.ManuallyTriggeredScheduledExecutorService; import pl.allegro.tech.hermes.test.helper.concurrent.TestExecutorServiceFactory; +import pl.allegro.tech.hermes.test.helper.metrics.TestMetricsFacadeFactory; import pl.allegro.tech.hermes.test.helper.time.ModifiableClock; import pl.allegro.tech.hermes.test.helper.zookeeper.ZookeeperBaseTest; @@ -44,7 +42,7 @@ public class ZookeeperConsumerNodeLoadRegistryTest extends ZookeeperBaseTest { Duration.ofMillis(50), new TestExecutorServiceFactory(scheduledExecutorService), clock, - new HermesMetrics(new MetricRegistry(), new PathsCompiler("host")), + TestMetricsFacadeFactory.create(), 100_000 ); diff --git a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/DefaultHermesHistogram.java b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/DefaultHermesHistogram.java new file mode 100644 index 0000000000..33e8d9b14e --- /dev/null +++ b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/DefaultHermesHistogram.java @@ -0,0 +1,23 @@ +package pl.allegro.tech.hermes.metrics; + +public class DefaultHermesHistogram implements HermesHistogram { + private final io.micrometer.core.instrument.DistributionSummary micrometerHistogram; + private final com.codahale.metrics.Histogram graphiteHistogram; + + private DefaultHermesHistogram(io.micrometer.core.instrument.DistributionSummary micrometerHistogram, + com.codahale.metrics.Histogram graphiteHistogram) { + this.micrometerHistogram = micrometerHistogram; + this.graphiteHistogram = graphiteHistogram; + } + + public static DefaultHermesHistogram of(io.micrometer.core.instrument.DistributionSummary micrometerHistogram, + com.codahale.metrics.Histogram graphiteHistogram) { + return new DefaultHermesHistogram(micrometerHistogram, graphiteHistogram); + } + + @Override + public void record(long value) { + micrometerHistogram.record(value); + graphiteHistogram.update(value); + } +} diff --git a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/HermesHistogram.java b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/HermesHistogram.java index 44aaae8ab4..2410154e13 100644 --- a/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/HermesHistogram.java +++ b/hermes-metrics/src/main/java/pl/allegro/tech/hermes/metrics/HermesHistogram.java @@ -1,22 +1,6 @@ package pl.allegro.tech.hermes.metrics; -public class HermesHistogram { - private final io.micrometer.core.instrument.DistributionSummary micrometerHistogram; - private final com.codahale.metrics.Histogram graphiteHistogram; +public interface HermesHistogram { - private HermesHistogram(io.micrometer.core.instrument.DistributionSummary micrometerHistogram, - com.codahale.metrics.Histogram graphiteHistogram) { - this.micrometerHistogram = micrometerHistogram; - this.graphiteHistogram = graphiteHistogram; - } - - public static HermesHistogram of(io.micrometer.core.instrument.DistributionSummary micrometerHistogram, - com.codahale.metrics.Histogram graphiteHistogram) { - return new HermesHistogram(micrometerHistogram, graphiteHistogram); - } - - public void record(int value) { - micrometerHistogram.record(value); - graphiteHistogram.update(value); - } + void record(long value); } diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/metrics/TestMetricsFacadeFactory.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/metrics/TestMetricsFacadeFactory.java new file mode 100644 index 0000000000..1b415b44a9 --- /dev/null +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/metrics/TestMetricsFacadeFactory.java @@ -0,0 +1,17 @@ +package pl.allegro.tech.hermes.test.helper.metrics; + +import com.codahale.metrics.MetricRegistry; +import io.micrometer.core.instrument.simple.SimpleMeterRegistry; +import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; +import pl.allegro.tech.hermes.metrics.PathsCompiler; + +public class TestMetricsFacadeFactory { + + public static MetricsFacade create() { + return new MetricsFacade( + new SimpleMeterRegistry(), + new HermesMetrics(new MetricRegistry(), new PathsCompiler("localhost")) + ); + } +} From 80ee439b2a958ad01a58422de93e52c4784e476a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20F=C4=85derski?= Date: Fri, 18 Aug 2023 12:51:07 +0200 Subject: [PATCH 2/6] Zookeeper counter based on prometheus metrics (#1706) * Zookeeper counter based on prometheus metrics * Fix method name --- .../allegro/tech/hermes/api/TopicMetrics.java | 2 +- .../di/factories/MetricRegistryFactory.java | 13 -- .../MicrometerRegistryParameters.java | 5 + .../PrometheusMeterRegistryFactory.java | 31 ++++- .../common/metric/SubscriptionMetrics.java | 89 ++++++------ .../hermes/common/metric/TopicMetrics.java | 70 ++++++---- .../counter/zookeeper/CounterMatcher.java | 68 +++++++--- .../zookeeper/ZookeeperCounterReporter.java | 115 +++++++--------- .../zookeeper/CounterMatcherTest.groovy | 26 ++-- .../ZookeeperCounterReporterTest.java | 127 ++++++------------ .../consumers/config/CommonConfiguration.java | 8 +- .../config/MicrometerRegistryProperties.java | 29 ++++ .../frontend/config/CommonConfiguration.java | 8 +- .../config/MicrometerRegistryProperties.java | 29 ++++ hermes-management/build.gradle | 5 +- .../config/ManagementConfiguration.java | 18 --- .../config/MicrometerRegistryProperties.java | 4 +- .../config/PrometheusConfiguration.java | 65 +++++++++ 18 files changed, 405 insertions(+), 307 deletions(-) create mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusConfiguration.java diff --git a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/TopicMetrics.java b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/TopicMetrics.java index 76b684c5ec..ddc89bd6cc 100644 --- a/hermes-api/src/main/java/pl/allegro/tech/hermes/api/TopicMetrics.java +++ b/hermes-api/src/main/java/pl/allegro/tech/hermes/api/TopicMetrics.java @@ -69,7 +69,7 @@ public static TopicMetrics unavailable() { } public static class Builder { - private TopicMetrics topicMetrics; + private final TopicMetrics topicMetrics; public Builder() { topicMetrics = new TopicMetrics(); diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/MetricRegistryFactory.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/MetricRegistryFactory.java index d541791598..1d1de61579 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/MetricRegistryFactory.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/MetricRegistryFactory.java @@ -17,8 +17,6 @@ import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricRegistryWithHdrHistogramReservoir; -import pl.allegro.tech.hermes.common.metric.counter.CounterStorage; -import pl.allegro.tech.hermes.common.metric.counter.zookeeper.ZookeeperCounterReporter; import pl.allegro.tech.hermes.common.util.InstanceIdResolver; import java.net.InetSocketAddress; @@ -33,18 +31,15 @@ public class MetricRegistryFactory { private static final Logger logger = LoggerFactory.getLogger(MetricRegistryFactory.class); private final MetricRegistryParameters metricRegistryParameters; private final GraphiteParameters graphiteParameters; - private final CounterStorage counterStorage; private final InstanceIdResolver instanceIdResolver; private final String moduleName; public MetricRegistryFactory(MetricRegistryParameters metricRegistryParameters, GraphiteParameters graphiteParameters, - CounterStorage counterStorage, InstanceIdResolver instanceIdResolver, @Named("moduleName") String moduleName) { this.metricRegistryParameters = metricRegistryParameters; this.graphiteParameters = graphiteParameters; - this.counterStorage = counterStorage; this.instanceIdResolver = instanceIdResolver; this.moduleName = moduleName; } @@ -73,14 +68,6 @@ public MetricRegistry provide() { metricRegistryParameters.getReportPeriod().toSeconds(), TimeUnit.SECONDS ); } - - if (metricRegistryParameters.isZookeeperReporterEnabled()) { - new ZookeeperCounterReporter(registry, counterStorage, graphiteParameters.getPrefix()).start( - metricRegistryParameters.getReportPeriod().toSeconds(), - TimeUnit.SECONDS - ); - } - registerJvmMetrics(registry); return registry; diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/MicrometerRegistryParameters.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/MicrometerRegistryParameters.java index 5a967bb9e8..48a74a84ea 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/MicrometerRegistryParameters.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/MicrometerRegistryParameters.java @@ -1,7 +1,12 @@ package pl.allegro.tech.hermes.common.di.factories; +import java.time.Duration; import java.util.List; public interface MicrometerRegistryParameters { List getPercentiles(); + + boolean zookeeperReporterEnabled(); + + Duration zookeeperReportPeriod(); } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/PrometheusMeterRegistryFactory.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/PrometheusMeterRegistryFactory.java index 9f7d24bb62..6a06ec5eba 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/PrometheusMeterRegistryFactory.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/di/factories/PrometheusMeterRegistryFactory.java @@ -5,22 +5,36 @@ import io.micrometer.core.instrument.distribution.DistributionStatisticConfig; import io.micrometer.prometheus.PrometheusConfig; import io.micrometer.prometheus.PrometheusMeterRegistry; +import pl.allegro.tech.hermes.common.metric.counter.CounterStorage; +import pl.allegro.tech.hermes.common.metric.counter.zookeeper.ZookeeperCounterReporter; + +import java.util.concurrent.TimeUnit; public class PrometheusMeterRegistryFactory { private final MicrometerRegistryParameters parameters; private final PrometheusConfig prometheusConfig; + private final CounterStorage counterStorage; private final String prefix; public PrometheusMeterRegistryFactory(MicrometerRegistryParameters parameters, PrometheusConfig prometheusConfig, - String prefix) { + CounterStorage counterStorage, String prefix) { this.parameters = parameters; this.prometheusConfig = prometheusConfig; + this.counterStorage = counterStorage; this.prefix = prefix + "_"; } public PrometheusMeterRegistry provide() { PrometheusMeterRegistry meterRegistry = new PrometheusMeterRegistry(prometheusConfig); + applyFilters(meterRegistry); + if (parameters.zookeeperReporterEnabled()) { + registerZookeeperReporter(meterRegistry); + } + return meterRegistry; + } + + private void applyFilters(PrometheusMeterRegistry meterRegistry) { meterRegistry.config().meterFilter(new MeterFilter() { @Override public Meter.Id map(Meter.Id id) { @@ -28,14 +42,17 @@ public Meter.Id map(Meter.Id id) { } @Override - public DistributionStatisticConfig configure(Meter.Id id, - DistributionStatisticConfig config) { + public DistributionStatisticConfig configure(Meter.Id id, DistributionStatisticConfig config) { return DistributionStatisticConfig.builder() - .percentiles(parameters.getPercentiles().stream().mapToDouble(Double::doubleValue).toArray()) - .build() - .merge(config); + .percentiles(parameters.getPercentiles() + .stream().mapToDouble(Double::doubleValue).toArray() + ).build().merge(config); } }); - return meterRegistry; + } + + private void registerZookeeperReporter(PrometheusMeterRegistry meterRegistry) { + new ZookeeperCounterReporter(meterRegistry, counterStorage, prefix) + .start(parameters.zookeeperReportPeriod().toSeconds(), TimeUnit.SECONDS); } } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionMetrics.java index a4db4f5efb..ea41df5b58 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/SubscriptionMetrics.java @@ -5,30 +5,14 @@ import io.micrometer.core.instrument.MeterRegistry; import io.micrometer.core.instrument.Tags; import pl.allegro.tech.hermes.api.SubscriptionName; -import pl.allegro.tech.hermes.metrics.DefaultHermesHistogram; import pl.allegro.tech.hermes.metrics.HermesCounter; import pl.allegro.tech.hermes.metrics.HermesHistogram; import pl.allegro.tech.hermes.metrics.HermesTimer; import pl.allegro.tech.hermes.metrics.counters.HermesCounters; -import java.util.concurrent.TimeUnit; import java.util.function.ToDoubleFunction; -import static pl.allegro.tech.hermes.common.metric.Counters.DELIVERED; -import static pl.allegro.tech.hermes.common.metric.Counters.DISCARDED; -import static pl.allegro.tech.hermes.common.metric.Meters.DISCARDED_METER; -import static pl.allegro.tech.hermes.common.metric.Meters.DISCARDED_SUBSCRIPTION_METER; -import static pl.allegro.tech.hermes.common.metric.Meters.DISCARDED_TOPIC_METER; -import static pl.allegro.tech.hermes.common.metric.Meters.FAILED_METER_SUBSCRIPTION; -import static pl.allegro.tech.hermes.common.metric.Meters.FILTERED_METER; -import static pl.allegro.tech.hermes.common.metric.Meters.METER; -import static pl.allegro.tech.hermes.common.metric.Meters.SUBSCRIPTION_BATCH_METER; -import static pl.allegro.tech.hermes.common.metric.Meters.SUBSCRIPTION_METER; -import static pl.allegro.tech.hermes.common.metric.Meters.SUBSCRIPTION_THROUGHPUT_BYTES; -import static pl.allegro.tech.hermes.common.metric.Meters.TOPIC_METER; import static pl.allegro.tech.hermes.common.metric.SubscriptionTagsFactory.subscriptionTags; -import static pl.allegro.tech.hermes.common.metric.Timers.CONSUMER_IDLE_TIME; -import static pl.allegro.tech.hermes.common.metric.Timers.SUBSCRIPTION_LATENCY; public class SubscriptionMetrics { private final HermesMetrics hermesMetrics; @@ -41,68 +25,68 @@ public SubscriptionMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegis public SubscriptionHermesCounter throughputInBytes(SubscriptionName subscription) { return SubscriptionHermesCounter.from( - micrometerCounter("subscription-throughput-bytes", subscription), - hermesMetrics.meter(SUBSCRIPTION_THROUGHPUT_BYTES, subscription.getTopicName(), subscription.getName()), - SUBSCRIPTION_THROUGHPUT_BYTES, subscription); + micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_THROUGHPUT, subscription), + hermesMetrics.meter(Meters.SUBSCRIPTION_THROUGHPUT_BYTES, subscription.getTopicName(), subscription.getName()), + Meters.SUBSCRIPTION_THROUGHPUT_BYTES, subscription); } public HermesCounter successes(SubscriptionName subscription) { return size -> { - hermesMetrics.meter(METER).mark(size); - hermesMetrics.meter(TOPIC_METER, subscription.getTopicName()).mark(size); - hermesMetrics.meter(SUBSCRIPTION_METER, subscription.getTopicName(), subscription.getName()).mark(size); - hermesMetrics.counter(DELIVERED, subscription.getTopicName(), subscription.getName()).inc(size); - micrometerCounter("subscription.delivered", subscription).increment(size); + hermesMetrics.meter(Meters.METER).mark(size); + hermesMetrics.meter(Meters.TOPIC_METER, subscription.getTopicName()).mark(size); + hermesMetrics.meter(Meters.SUBSCRIPTION_METER, subscription.getTopicName(), subscription.getName()).mark(size); + hermesMetrics.counter(Counters.DELIVERED, subscription.getTopicName(), subscription.getName()).inc(size); + micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_DELIVERED, subscription).increment(size); }; } public HermesCounter batchSuccesses(SubscriptionName subscription) { return HermesCounters.from( - micrometerCounter("subscription.batches", subscription), - hermesMetrics.meter(SUBSCRIPTION_BATCH_METER, subscription.getTopicName(), subscription.getName()) + micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_BATCHES, subscription), + hermesMetrics.meter(Meters.SUBSCRIPTION_BATCH_METER, subscription.getTopicName(), subscription.getName()) ); } public HermesCounter discarded(SubscriptionName subscription) { return size -> { - hermesMetrics.meter(DISCARDED_METER).mark(size); - hermesMetrics.meter(DISCARDED_TOPIC_METER, subscription.getTopicName()).mark(size); - hermesMetrics.meter(DISCARDED_SUBSCRIPTION_METER, subscription.getTopicName(), subscription.getName()).mark(size); - hermesMetrics.counter(DISCARDED, subscription.getTopicName(), subscription.getName()).inc(size); - micrometerCounter("subscription.discarded", subscription).increment(size); + hermesMetrics.meter(Meters.DISCARDED_METER).mark(size); + hermesMetrics.meter(Meters.DISCARDED_TOPIC_METER, subscription.getTopicName()).mark(size); + hermesMetrics.meter(Meters.DISCARDED_SUBSCRIPTION_METER, subscription.getTopicName(), subscription.getName()).mark(size); + hermesMetrics.counter(Counters.DISCARDED, subscription.getTopicName(), subscription.getName()).inc(size); + micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_DISCARDED, subscription).increment(size); }; } public HermesTimer latency(SubscriptionName subscription) { return HermesTimer.from( - meterRegistry.timer("subscription.latency", subscriptionTags(subscription)), - hermesMetrics.timer(SUBSCRIPTION_LATENCY, subscription.getTopicName(), subscription.getName()) + meterRegistry.timer(SubscriptionMetricsNames.SUBSCRIPTION_LATENCY, subscriptionTags(subscription)), + hermesMetrics.timer(Timers.SUBSCRIPTION_LATENCY, subscription.getTopicName(), subscription.getName()) ); } public void registerInflightGauge(SubscriptionName subscription, T obj, ToDoubleFunction f) { hermesMetrics.registerInflightGauge(subscription, () -> (int) f.applyAsDouble(obj)); - meterRegistry.gauge("subscription.inflight", subscriptionTags(subscription), obj, f); + meterRegistry.gauge(SubscriptionMetricsNames.SUBSCRIPTION_INFLIGHT, subscriptionTags(subscription), obj, f); } public HermesTimer consumerIdleTimer(SubscriptionName subscription) { return HermesTimer.from( - meterRegistry.timer("subscription.idle-duration", subscriptionTags(subscription)), - hermesMetrics.timer(CONSUMER_IDLE_TIME, subscription.getTopicName(), subscription.getName()) + meterRegistry.timer(SubscriptionMetricsNames.SUBSCRIPTION_IDLE_DURATION, subscriptionTags(subscription)), + hermesMetrics.timer(Timers.CONSUMER_IDLE_TIME, subscription.getTopicName(), subscription.getName()) ); } public HermesCounter filteredOutCounter(SubscriptionName subscription) { return HermesCounters.from( - micrometerCounter("subscription.filtered-out", subscription), - hermesMetrics.meter(FILTERED_METER, subscription.getTopicName(), subscription.getName()) + micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_FILTERED_OUT, subscription), + hermesMetrics.meter(Meters.FILTERED_METER, subscription.getTopicName(), subscription.getName()) ); } public HermesCounter httpAnswerCounter(SubscriptionName subscription, int statusCode) { return size -> { meterRegistry.counter( - "subscription.http-status-codes", + SubscriptionMetricsNames.SUBSCRIPTION_HTTP_STATUS_CODES, Tags.concat(subscriptionTags(subscription), "status_code", String.valueOf(statusCode)) ).increment(size); hermesMetrics.registerConsumerHttpAnswer(subscription, statusCode, size); @@ -111,28 +95,28 @@ public HermesCounter httpAnswerCounter(SubscriptionName subscription, int status public HermesCounter timeoutsCounter(SubscriptionName subscription) { return HermesCounters.from( - micrometerCounter("subscription.timeouts", subscription), + micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_TIMEOUTS, subscription), hermesMetrics.consumerErrorsTimeoutMeter(subscription) ); } public HermesCounter otherErrorsCounter(SubscriptionName subscription) { return HermesCounters.from( - micrometerCounter("subscription.other-errors", subscription), + micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_OTHER_ERRORS, subscription), hermesMetrics.consumerErrorsOtherMeter(subscription) ); } public HermesCounter failuresCounter(SubscriptionName subscription) { return HermesCounters.from( - micrometerCounter("subscription.failures", subscription), - hermesMetrics.meter(FAILED_METER_SUBSCRIPTION, subscription.getTopicName(), subscription.getName()) + micrometerCounter(SubscriptionMetricsNames.SUBSCRIPTION_FAILURES, subscription), + hermesMetrics.meter(Meters.FAILED_METER_SUBSCRIPTION, subscription.getTopicName(), subscription.getName()) ); } public HermesHistogram inflightTimeInMillisHistogram(SubscriptionName subscriptionName) { return value -> { - DistributionSummary.builder("subscription.inflight-time-seconds") + DistributionSummary.builder(SubscriptionMetricsNames.SUBSCRIPTION_INFLIGHT_TIME) .tags(subscriptionTags(subscriptionName)) .register(meterRegistry) .record(value / 1000d); @@ -143,4 +127,21 @@ public HermesHistogram inflightTimeInMillisHistogram(SubscriptionName subscripti private Counter micrometerCounter(String metricName, SubscriptionName subscription) { return meterRegistry.counter(metricName, subscriptionTags(subscription)); } + + public static class SubscriptionMetricsNames { + public static final String SUBSCRIPTION_DELIVERED = "subscription.delivered"; + public static final String SUBSCRIPTION_THROUGHPUT = "subscription.throughput-bytes"; + public static final String SUBSCRIPTION_BATCHES = "subscription.batches"; + public static final String SUBSCRIPTION_DISCARDED = "subscription.discarded"; + public static final String SUBSCRIPTION_LATENCY = "subscription.latency"; + public static final String SUBSCRIPTION_INFLIGHT = "subscription.inflight"; + public static final String SUBSCRIPTION_IDLE_DURATION = "subscription.idle-duration"; + public static final String SUBSCRIPTION_FILTERED_OUT = "subscription.filtered-out"; + public static final String SUBSCRIPTION_HTTP_STATUS_CODES = "subscription.http-status-codes"; + public static final String SUBSCRIPTION_TIMEOUTS = "subscription.timeouts"; + public static final String SUBSCRIPTION_OTHER_ERRORS = "subscription.other-errors"; + public static final String SUBSCRIPTION_FAILURES = "subscription.failures"; + public static final String SUBSCRIPTION_INFLIGHT_TIME = "subscription.inflight-time-seconds"; + } + } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TopicMetrics.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TopicMetrics.java index 331ea9b35d..1d840e3b5d 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TopicMetrics.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/TopicMetrics.java @@ -15,11 +15,6 @@ import pl.allegro.tech.hermes.metrics.counters.MeterBackedHermesCounter; import static pl.allegro.tech.hermes.common.metric.Meters.DELAYED_PROCESSING; -import static pl.allegro.tech.hermes.common.metric.Meters.METER; -import static pl.allegro.tech.hermes.common.metric.Meters.THROUGHPUT_BYTES; -import static pl.allegro.tech.hermes.common.metric.Meters.TOPIC_DELAYED_PROCESSING; -import static pl.allegro.tech.hermes.common.metric.Meters.TOPIC_METER; -import static pl.allegro.tech.hermes.common.metric.Meters.TOPIC_THROUGHPUT_BYTES; public class TopicMetrics { private final HermesMetrics hermesMetrics; @@ -32,107 +27,108 @@ public TopicMetrics(HermesMetrics hermesMetrics, MeterRegistry meterRegistry) { public HermesTimer ackAllGlobalLatency() { return HermesTimer.from( - meterRegistry.timer("ack-all.global-latency"), + meterRegistry.timer(TopicMetricsNames.TOPIC_ACK_ALL_GLOBAL_LATENCY), hermesMetrics.timer(Timers.ACK_ALL_LATENCY) ); } public HermesTimer ackAllTopicLatency(TopicName topic) { return HermesTimer.from( - micrometerTimer("ack-all.topic-latency", topic), + micrometerTimer(TopicMetricsNames.TOPIC_ACK_ALL_LATENCY, topic), hermesMetrics.timer(Timers.ACK_ALL_TOPIC_LATENCY, topic)); } public HermesTimer ackAllBrokerLatency() { return HermesTimer.from( - meterRegistry.timer("ack-all.broker-latency"), + meterRegistry.timer(TopicMetricsNames.TOPIC_ACK_ALL_BROKER_LATENCY), hermesMetrics.timer(Timers.ACK_ALL_BROKER_LATENCY)); } public HermesTimer ackLeaderGlobalLatency() { return HermesTimer.from( - meterRegistry.timer("ack-leader.global-latency"), + meterRegistry.timer(TopicMetricsNames.TOPIC_ACK_LEADER_GLOBAL_LATENCY), hermesMetrics.timer(Timers.ACK_LEADER_LATENCY)); } public HermesTimer ackLeaderTopicLatency(TopicName topic) { return HermesTimer.from( - micrometerTimer("ack-leader.topic-latency", topic), + micrometerTimer(TopicMetricsNames.TOPIC_ACK_LEADER_LATENCY, topic), hermesMetrics.timer(Timers.ACK_LEADER_TOPIC_LATENCY, topic)); } public HermesTimer ackLeaderBrokerLatency() { return HermesTimer.from( - meterRegistry.timer("ack-leader.broker-latency"), + meterRegistry.timer(TopicMetricsNames.TOPIC_ACK_LEADER_BROKER_LATENCY), hermesMetrics.timer(Timers.ACK_LEADER_BROKER_LATENCY)); } public MeterBackedHermesCounter topicThroughputBytes(TopicName topicName) { return HermesCounters.from( - micrometerCounter("topic-throughput-bytes", topicName), - hermesMetrics.meter(TOPIC_THROUGHPUT_BYTES, topicName) + micrometerCounter(TopicMetricsNames.TOPIC_THROUGHPUT, topicName), + hermesMetrics.meter(Meters.TOPIC_THROUGHPUT_BYTES, topicName) ); } public MeterBackedHermesCounter topicGlobalThroughputBytes() { return HermesCounters.from( - meterRegistry.counter("topic-global-throughput-bytes"), - hermesMetrics.meter(THROUGHPUT_BYTES) + meterRegistry.counter(TopicMetricsNames.TOPIC_GLOBAL_THROUGHPUT), + hermesMetrics.meter(Meters.THROUGHPUT_BYTES) ); } public HermesCounter topicPublished(TopicName topicName) { return HermesCounters.from( - micrometerCounter("published", topicName), + micrometerCounter(TopicMetricsNames.TOPIC_PUBLISHED, topicName), hermesMetrics.counter(Counters.PUBLISHED, topicName) ); } public HermesCounter topicGlobalRequestCounter() { return HermesCounters.from( - meterRegistry.counter("topic-global-requests"), - hermesMetrics.meter(METER) + meterRegistry.counter(TopicMetricsNames.TOPIC_GLOBAL_REQUESTS), + hermesMetrics.meter(Meters.METER) ); } public HermesCounter topicRequestCounter(TopicName topicName) { return HermesCounters.from( - micrometerCounter("topic-requests", topicName), - hermesMetrics.meter(TOPIC_METER, topicName) + micrometerCounter(TopicMetricsNames.TOPIC_REQUESTS, topicName), + hermesMetrics.meter(Meters.TOPIC_METER, topicName) ); } public HermesCounter topicGlobalDelayedProcessingCounter() { return HermesCounters.from( - meterRegistry.counter("topic-global-delayed-processing"), + meterRegistry.counter(TopicMetricsNames.TOPIC_GLOBAL_DELAYED_PROCESSING), hermesMetrics.meter(DELAYED_PROCESSING) ); } public HermesCounter topicDelayedProcessingCounter(TopicName topicName) { return HermesCounters.from( - micrometerCounter("topic-delayed-processing", topicName), - hermesMetrics.meter(TOPIC_DELAYED_PROCESSING, topicName) + micrometerCounter(TopicMetricsNames.TOPIC_DELAYED_PROCESSING, topicName), + hermesMetrics.meter(Meters.TOPIC_DELAYED_PROCESSING, topicName) ); } public HermesCounter topicGlobalHttpStatusCodeCounter(int statusCode) { return HermesCounters.from( - meterRegistry.counter("topic-global-http-status-codes", Tags.of("status_code", String.valueOf(statusCode))), + meterRegistry.counter(TopicMetricsNames.TOPIC_GLOBAL_HTTP_STATUS_CODES, Tags.of("status_code", String.valueOf(statusCode))), hermesMetrics.httpStatusCodeMeter(statusCode) ); } public HermesCounter topicHttpStatusCodeCounter(TopicName topicName, int statusCode) { return HermesCounters.from( - meterRegistry.counter("topic-http-status-codes", topicTags(topicName).and("status_code", String.valueOf(statusCode))), + meterRegistry.counter(TopicMetricsNames.TOPIC_HTTP_STATUS_CODES, topicTags(topicName) + .and("status_code", String.valueOf(statusCode))), hermesMetrics.httpStatusCodeMeter(statusCode, topicName) ); } public HermesHistogram topicGlobalMessageContentSizeHistogram() { return DefaultHermesHistogram.of( - DistributionSummary.builder("topic-global-message-size-bytes") + DistributionSummary.builder(TopicMetricsNames.TOPIC_GLOBAL_MESSAGE_SIZE_BYTES) .register(meterRegistry), hermesMetrics.messageContentSizeHistogram() ); @@ -140,7 +136,7 @@ public HermesHistogram topicGlobalMessageContentSizeHistogram() { public HermesHistogram topicMessageContentSizeHistogram(TopicName topicName) { return DefaultHermesHistogram.of( - DistributionSummary.builder("topic-message-size-bytes") + DistributionSummary.builder(TopicMetricsNames.TOPIC_MESSAGE_SIZE_BYTES) .tags(topicTags(topicName)) .register(meterRegistry), hermesMetrics.messageContentSizeHistogram(topicName) @@ -161,4 +157,24 @@ private Tags topicTags(TopicName topicName) { Tag.of("topic", topicName.getName()) ); } + + public static class TopicMetricsNames { + public static final String TOPIC_ACK_ALL_GLOBAL_LATENCY = "topic.ack-all.global-latency"; + public static final String TOPIC_ACK_ALL_LATENCY = "topic.ack-all.latency"; + public static final String TOPIC_ACK_ALL_BROKER_LATENCY = "topic.ack-all.broker-latency"; + public static final String TOPIC_ACK_LEADER_GLOBAL_LATENCY = "topic.ack-leader.global-latency"; + public static final String TOPIC_ACK_LEADER_LATENCY = "topic.ack-leader.latency"; + public static final String TOPIC_ACK_LEADER_BROKER_LATENCY = "topic.ack-leader.broker-latency"; + public static final String TOPIC_THROUGHPUT = "topic.throughput-bytes"; + public static final String TOPIC_GLOBAL_THROUGHPUT = "topic.global-throughput-bytes"; + public static final String TOPIC_PUBLISHED = "topic.published"; + public static final String TOPIC_GLOBAL_REQUESTS = "topic.global-requests"; + public static final String TOPIC_REQUESTS = "topic.requests"; + public static final String TOPIC_GLOBAL_DELAYED_PROCESSING = "topic-global-delayed-processing"; + public static final String TOPIC_DELAYED_PROCESSING = "topic-delayed-processing"; + public static final String TOPIC_GLOBAL_HTTP_STATUS_CODES = "topic-global-http-status-codes"; + public static final String TOPIC_HTTP_STATUS_CODES = "topic-http-status-codes"; + public static final String TOPIC_GLOBAL_MESSAGE_SIZE_BYTES = "topic-global-message-size-bytes"; + public static final String TOPIC_MESSAGE_SIZE_BYTES = "topic-message-size-bytes"; + } } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/CounterMatcher.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/CounterMatcher.java index b1fdced29c..7e52b5186b 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/CounterMatcher.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/CounterMatcher.java @@ -1,68 +1,92 @@ package pl.allegro.tech.hermes.common.metric.counter.zookeeper; +import io.micrometer.core.instrument.Counter; +import pl.allegro.tech.hermes.api.TopicName; + import java.util.Optional; +import static pl.allegro.tech.hermes.common.metric.SubscriptionMetrics.SubscriptionMetricsNames; +import static pl.allegro.tech.hermes.common.metric.TopicMetrics.TopicMetricsNames; + class CounterMatcher { - private static final int TOPIC_METRICS_PARTS = 3; - private static final int SUBSCRIPTION_METRIC_PARTS = 4; + private static final String GROUP_TAG_NAME = "group"; + private static final String TOPIC_TAG_NAME = "topic"; + private static final String SUBSCRIPTION_TAG_NAME = "subscription"; - private final String counterName; - private String topicName; + private final Counter counter; + private final String metricSearchPrefix; + private TopicName topicName; + private long value; private Optional subscription; - private int metricParts; - public CounterMatcher(String counterName) { - this.counterName = counterName; - parseCounter(counterName); + public CounterMatcher(Counter counter, String metricSearchPrefix) { + this.counter = counter; + this.metricSearchPrefix = metricSearchPrefix; + parseCounter(this.counter); } - private void parseCounter(String counterName) { - String[] splitted = counterName.split("\\."); - metricParts = splitted.length; + private void parseCounter(Counter counter) { if (isTopicPublished() || isTopicThroughput()) { - topicName = splitted[splitted.length - 2] + "." + splitted[splitted.length - 1]; + topicName = new TopicName(counter.getId().getTag(GROUP_TAG_NAME), counter.getId().getTag(TOPIC_TAG_NAME)); subscription = Optional.empty(); } else if ( isSubscriptionDelivered() || isSubscriptionThroughput() || isSubscriptionDiscarded() || isSubscriptionFiltered() - ) { - subscription = Optional.of(splitted[splitted.length - 1]); - topicName = splitted[splitted.length - 3] + "." + splitted[splitted.length - 2]; + ) { + topicName = new TopicName(counter.getId().getTag(GROUP_TAG_NAME), counter.getId().getTag(TOPIC_TAG_NAME)); + subscription = Optional.of(counter.getId().getTag(SUBSCRIPTION_TAG_NAME)); } + value = (long) counter.count(); } public boolean isTopicPublished() { - return counterName.startsWith("published."); + return isTopicCounter() && nameEquals(TopicMetricsNames.TOPIC_PUBLISHED); } public boolean isTopicThroughput() { - return metricParts == TOPIC_METRICS_PARTS && counterName.startsWith("throughput."); + return isTopicCounter() && nameEquals(TopicMetricsNames.TOPIC_THROUGHPUT); } public boolean isSubscriptionThroughput() { - return metricParts == SUBSCRIPTION_METRIC_PARTS && counterName.startsWith("throughput."); + return isSubscriptionCounter() && nameEquals(SubscriptionMetricsNames.SUBSCRIPTION_THROUGHPUT); } public boolean isSubscriptionDelivered() { - return counterName.startsWith("delivered."); + return isSubscriptionCounter() && nameEquals(SubscriptionMetricsNames.SUBSCRIPTION_DELIVERED); } public boolean isSubscriptionDiscarded() { - return counterName.startsWith("discarded."); + return isSubscriptionCounter() && nameEquals(SubscriptionMetricsNames.SUBSCRIPTION_DISCARDED); } public boolean isSubscriptionFiltered() { - return counterName.startsWith("filtered."); + return isSubscriptionCounter() && nameEquals(SubscriptionMetricsNames.SUBSCRIPTION_FILTERED_OUT); } - public String getTopicName() { + public TopicName getTopicName() { return topicName; } public String getSubscriptionName() { return subscription.orElse(""); } + + public long getValue() { + return value; + } + + private boolean isTopicCounter() { + return counter.getId().getTag(TOPIC_TAG_NAME) != null; + } + + private boolean isSubscriptionCounter() { + return counter.getId().getTag(SUBSCRIPTION_TAG_NAME) != null; + } + + private boolean nameEquals(String name) { + return counter.getId().getName().equals(metricSearchPrefix + name); + } } diff --git a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporter.java b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporter.java index 9a17cf7f77..c66efdc640 100644 --- a/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporter.java +++ b/hermes-common/src/main/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporter.java @@ -1,108 +1,99 @@ package pl.allegro.tech.hermes.common.metric.counter.zookeeper; -import com.codahale.metrics.Counter; -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Histogram; -import com.codahale.metrics.Meter; -import com.codahale.metrics.Metric; -import com.codahale.metrics.MetricFilter; -import com.codahale.metrics.MetricRegistry; -import com.codahale.metrics.ScheduledReporter; -import com.codahale.metrics.Timer; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.MeterRegistry; +import io.micrometer.core.instrument.search.Search; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import pl.allegro.tech.hermes.api.TopicName; import pl.allegro.tech.hermes.common.metric.HermesMetrics; -import pl.allegro.tech.hermes.common.metric.Meters; import pl.allegro.tech.hermes.common.metric.counter.CounterStorage; -import java.util.SortedMap; +import java.util.Collection; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import static pl.allegro.tech.hermes.api.TopicName.fromQualifiedName; - -public class ZookeeperCounterReporter extends ScheduledReporter { - - private static final String ZOOKEEPER_REPORTER_NAME = "zookeeper-reporter"; - private static final TimeUnit RATE_UNIT = TimeUnit.SECONDS; - private static final TimeUnit DURATION_UNIT = TimeUnit.MILLISECONDS; +public class ZookeeperCounterReporter { + private static final Logger logger = LoggerFactory.getLogger(ZookeeperCounterReporter.class); private final CounterStorage counterStorage; + private final String metricsSearchPrefix; + private final MeterRegistry meterRegistry; - public ZookeeperCounterReporter(MetricRegistry registry, + private final ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder() + .setNameFormat("zookeeper-reporter-scheduled-executor-%d") + .setDaemon(true) + .build()); + + public ZookeeperCounterReporter(MeterRegistry registry, CounterStorage counterStorage, - String graphitePrefix - ) { - super( - registry, - ZOOKEEPER_REPORTER_NAME, - new ZookeeperMetricsFilter(graphitePrefix), - RATE_UNIT, - DURATION_UNIT - ); + String metricsSearchPrefix) { + this.meterRegistry = registry; this.counterStorage = counterStorage; + this.metricsSearchPrefix = metricsSearchPrefix; } - @Override - public void report(SortedMap gauges, - SortedMap counters, - SortedMap histograms, - SortedMap meters, - SortedMap timers) { - - counters.forEach(this::reportCounter); + public void start(long period, TimeUnit unit) { + scheduledExecutorService.scheduleWithFixedDelay(this::report, 0, period, unit); + } - meters - .entrySet() - .stream() - .filter(meterEntry -> meterEntry.getKey().startsWith(Meters.THROUGHPUT_BYTES)) - .forEach(meterEntry -> reportVolumeCounter(meterEntry.getKey(), meterEntry.getValue().getCount())); + public void report() { + try { + Collection counters = Search.in(meterRegistry).counters(); + counters.forEach(counter -> { + CounterMatcher matcher = new CounterMatcher(counter, metricsSearchPrefix); + reportCounter(matcher); + reportVolumeCounter(matcher); + }); + } catch (RuntimeException ex) { + logger.error("Error during reporting metrics to Zookeeper...", ex); + } } - private void reportVolumeCounter(String metricName, long value) { - CounterMatcher matcher = new CounterMatcher(metricName); + private void reportVolumeCounter(CounterMatcher matcher) { if (matcher.isTopicThroughput()) { counterStorage.incrementVolumeCounter( escapedTopicName(matcher.getTopicName()), - value + matcher.getValue() ); } else if (matcher.isSubscriptionThroughput()) { counterStorage.incrementVolumeCounter( escapedTopicName(matcher.getTopicName()), escapeMetricsReplacementChar(matcher.getSubscriptionName()), - value + matcher.getValue() ); } } - private void reportCounter(String counterName, Counter counter) { - if (counter.getCount() == 0) { + private void reportCounter(CounterMatcher matcher) { + if (matcher.getValue() == 0) { return; } - CounterMatcher matcher = new CounterMatcher(counterName); - long value = counter.getCount(); - if (matcher.isTopicPublished()) { counterStorage.setTopicPublishedCounter( escapedTopicName(matcher.getTopicName()), - value + matcher.getValue() ); } else if (matcher.isSubscriptionDelivered()) { counterStorage.setSubscriptionDeliveredCounter( escapedTopicName(matcher.getTopicName()), escapeMetricsReplacementChar(matcher.getSubscriptionName()), - value + matcher.getValue() ); } else if (matcher.isSubscriptionDiscarded()) { counterStorage.setSubscriptionDiscardedCounter( escapedTopicName(matcher.getTopicName()), escapeMetricsReplacementChar(matcher.getSubscriptionName()), - value + matcher.getValue() ); } } - private static TopicName escapedTopicName(String qualifiedTopicName) { - TopicName topicName = fromQualifiedName(qualifiedTopicName); + private static TopicName escapedTopicName(TopicName topicName) { return new TopicName( escapeMetricsReplacementChar(topicName.getGroupName()), topicName.getName() @@ -112,18 +103,4 @@ private static TopicName escapedTopicName(String qualifiedTopicName) { private static String escapeMetricsReplacementChar(String value) { return value.replaceAll(HermesMetrics.REPLACEMENT_CHAR, "\\."); } - - private static final class ZookeeperMetricsFilter implements MetricFilter { - private final String offsetPrefix; - - private ZookeeperMetricsFilter(String graphitePrefix) { - offsetPrefix = graphitePrefix + "." + "consumer.offset"; - } - - @Override - public boolean matches(String name, Metric metric) { - return (metric instanceof Counter && !name.startsWith(offsetPrefix)) - || (metric instanceof Meter && name.startsWith(Meters.THROUGHPUT_BYTES + ".")); - } - } } diff --git a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/metric/counter/zookeeper/CounterMatcherTest.groovy b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/metric/counter/zookeeper/CounterMatcherTest.groovy index b12e4f3fa5..1ed88c7538 100644 --- a/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/metric/counter/zookeeper/CounterMatcherTest.groovy +++ b/hermes-common/src/test/groovy/pl/allegro/tech/hermes/common/metric/counter/zookeeper/CounterMatcherTest.groovy @@ -1,13 +1,19 @@ package pl.allegro.tech.hermes.common.metric.counter.zookeeper +import io.micrometer.core.instrument.simple.SimpleMeterRegistry +import pl.allegro.tech.hermes.api.TopicName +import spock.lang.Shared import spock.lang.Specification class CounterMatcherTest extends Specification { + @Shared + def meterRegistry = new SimpleMeterRegistry() + def "should match topic published"() { given: - def counterName = "published.lagMetricGroup.topic" - def counterMatcher = new CounterMatcher(counterName) + def counter = meterRegistry.counter("topic.published", "group", "lagMetricGroup", "topic", "topic") + def counterMatcher = new CounterMatcher(counter, "") when: def isTopic = counterMatcher.isTopicPublished() @@ -15,13 +21,14 @@ class CounterMatcherTest extends Specification { then: isTopic - topicName == "lagMetricGroup.topic" + topicName == new TopicName("lagMetricGroup", "topic") } def "should match subscription delivered"() { given: - def counterName = "delivered.lagMetricGroup.topic.subscription" - def counterMatcher = new CounterMatcher(counterName) + def counter = meterRegistry.counter("subscription.delivered", "group", "lagMetricGroup", + "topic", "topic", "subscription", "subscription") + def counterMatcher = new CounterMatcher(counter, "") when: def isSubscription = counterMatcher.isSubscriptionDelivered() @@ -30,14 +37,15 @@ class CounterMatcherTest extends Specification { then: isSubscription - topicName == "lagMetricGroup.topic" + topicName == new TopicName("lagMetricGroup", "topic") subscriptionName == "subscription" } def "should match subscription discarded"() { given: - def counterName = "discarded.lagMetricGroup.topic.subscription" - def counterMatcher = new CounterMatcher(counterName) + def counter = meterRegistry.counter("subscription.discarded", "group", "lagMetricGroup", + "topic", "topic", "subscription", "subscription") + def counterMatcher = new CounterMatcher(counter, "") when: def isSubscription = counterMatcher.isSubscriptionDiscarded() @@ -46,7 +54,7 @@ class CounterMatcherTest extends Specification { then: isSubscription - topicName == "lagMetricGroup.topic" + topicName == new TopicName("lagMetricGroup", "topic") subscriptionName == "subscription" } } \ No newline at end of file diff --git a/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporterTest.java b/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporterTest.java index 28d06c5632..3788cfb46e 100644 --- a/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporterTest.java +++ b/hermes-common/src/test/java/pl/allegro/tech/hermes/common/metric/counter/zookeeper/ZookeeperCounterReporterTest.java @@ -1,78 +1,41 @@ package pl.allegro.tech.hermes.common.metric.counter.zookeeper; -import com.codahale.metrics.Counter; -import com.codahale.metrics.Gauge; -import com.codahale.metrics.Histogram; -import com.codahale.metrics.Meter; import com.codahale.metrics.MetricRegistry; -import com.codahale.metrics.Timer; +import io.micrometer.core.instrument.MeterRegistry; +import io.micrometer.core.instrument.simple.SimpleMeterRegistry; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; +import pl.allegro.tech.hermes.api.SubscriptionName; import pl.allegro.tech.hermes.api.TopicName; +import pl.allegro.tech.hermes.common.metric.HermesMetrics; +import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.common.metric.counter.CounterStorage; import pl.allegro.tech.hermes.common.util.InstanceIdResolver; import pl.allegro.tech.hermes.metrics.PathsCompiler; -import java.util.SortedMap; -import java.util.TreeMap; - import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import static pl.allegro.tech.hermes.common.metric.Counters.DELIVERED; -import static pl.allegro.tech.hermes.common.metric.Counters.DISCARDED; -import static pl.allegro.tech.hermes.common.metric.Counters.PUBLISHED; -import static pl.allegro.tech.hermes.common.metric.Meters.SUBSCRIPTION_THROUGHPUT_BYTES; -import static pl.allegro.tech.hermes.common.metric.Meters.TOPIC_THROUGHPUT_BYTES; -import static pl.allegro.tech.hermes.metrics.PathContext.pathContext; @RunWith(MockitoJUnitRunner.class) public class ZookeeperCounterReporterTest { - - public static final SortedMap EMPTY_TIMERS = new TreeMap<>(); - public static final SortedMap EMPTY_METERS = new TreeMap<>(); - public static final SortedMap EMPTY_COUNTERS = new TreeMap<>(); - public static final SortedMap EMPTY_HISTOGRAMS = new TreeMap<>(); - public static final SortedMap EMPTY_GAUGES = null; - public static final String GROUP_NAME_UNDERSCORE = "pl_allegro_tech_skylab"; public static final String GROUP_NAME = "pl.allegro.tech.skylab"; public static final String TOPIC_NAME_UNDERSCORE = "topic_1"; - public static final String SUBSCRIPTION_NAME_UNDERSCORE = "subscription_name"; public static final String SUBSCRIPTION_NAME = "subscription.name"; - public static final TopicName QUALIFIED_TOPIC_NAME = new TopicName(GROUP_NAME, TOPIC_NAME_UNDERSCORE); - public static final long COUNT = 100L; - public static final String GRAPHITE_PREFIX = "tech.hermes"; - - private static final PathsCompiler pathsCompiler = new PathsCompiler("localhost.domain"); - - public static final String METRIC_NAME_FOR_PUBLISHED = pathsCompiler.compile(PUBLISHED, pathContext() - .withGroup(GROUP_NAME_UNDERSCORE).withTopic(TOPIC_NAME_UNDERSCORE).build()); + public static final TopicName topic = new TopicName(GROUP_NAME, TOPIC_NAME_UNDERSCORE); + public static final SubscriptionName subscription = new SubscriptionName(SUBSCRIPTION_NAME, topic); - public static final String METRIC_NAME_FOR_DELIVERED = pathsCompiler.compile(DELIVERED, pathContext() - .withGroup(GROUP_NAME_UNDERSCORE).withTopic(TOPIC_NAME_UNDERSCORE).withSubscription(SUBSCRIPTION_NAME_UNDERSCORE).build()); - - public static final String METRIC_NAME_FOR_DISCARDED = pathsCompiler.compile(DISCARDED, pathContext() - .withGroup(GROUP_NAME_UNDERSCORE).withTopic(TOPIC_NAME_UNDERSCORE).withSubscription(SUBSCRIPTION_NAME_UNDERSCORE).build()); - - public static final String METRIC_NAME_FOR_SUBSCRIPTION_THROUGHPUT = pathsCompiler.compile(SUBSCRIPTION_THROUGHPUT_BYTES, pathContext() - .withGroup(GROUP_NAME_UNDERSCORE).withTopic(TOPIC_NAME_UNDERSCORE).withSubscription(SUBSCRIPTION_NAME_UNDERSCORE).build()); - - public static final String METRIC_NAME_FOR_TOPIC_THRESHOLD = pathsCompiler.compile(TOPIC_THROUGHPUT_BYTES, pathContext() - .withGroup(GROUP_NAME_UNDERSCORE).withTopic(TOPIC_NAME_UNDERSCORE).build()); + public static final long COUNT = 100L; @Mock private CounterStorage counterStorage; - @Mock - private MetricRegistry metricRegistry; - - @Mock - private Counter counter; + private final MeterRegistry meterRegistry = new SimpleMeterRegistry(); - @Mock - private Meter meter; + private final MetricsFacade metricsFacade = new MetricsFacade( + meterRegistry, new HermesMetrics(new MetricRegistry(), new PathsCompiler("localhost"))); @Mock private InstanceIdResolver instanceIdResolver; @@ -82,72 +45,66 @@ public class ZookeeperCounterReporterTest { @Before public void before() { when(instanceIdResolver.resolve()).thenReturn("localhost.domain"); - zookeeperCounterReporter = new ZookeeperCounterReporter(metricRegistry, counterStorage, GRAPHITE_PREFIX); + zookeeperCounterReporter = new ZookeeperCounterReporter(meterRegistry, counterStorage, ""); } @Test public void shouldReportPublishedMessages() { - SortedMap counters = prepareCounters(METRIC_NAME_FOR_PUBLISHED); - when(counter.getCount()).thenReturn(COUNT); + // given + metricsFacade.topics().topicPublished(topic).increment(COUNT); - zookeeperCounterReporter.report(EMPTY_GAUGES, counters, EMPTY_HISTOGRAMS, EMPTY_METERS, EMPTY_TIMERS); + // when + zookeeperCounterReporter.report(); - verify(counterStorage).setTopicPublishedCounter(QUALIFIED_TOPIC_NAME, COUNT); + // then + verify(counterStorage).setTopicPublishedCounter(topic, COUNT); } @Test public void shouldReportDeliveredMessages() { - SortedMap counters = prepareCounters(METRIC_NAME_FOR_DELIVERED); - when(counter.getCount()).thenReturn(COUNT); + // given + metricsFacade.subscriptions().successes(subscription).increment(COUNT); - zookeeperCounterReporter.report(EMPTY_GAUGES, counters, EMPTY_HISTOGRAMS, EMPTY_METERS, EMPTY_TIMERS); + // when + zookeeperCounterReporter.report(); - verify(counterStorage).setSubscriptionDeliveredCounter(QUALIFIED_TOPIC_NAME, SUBSCRIPTION_NAME, COUNT); + // then + verify(counterStorage).setSubscriptionDeliveredCounter(topic, SUBSCRIPTION_NAME, COUNT); } @Test public void shouldReportDiscardedMessages() { - SortedMap counters = prepareCounters(METRIC_NAME_FOR_DISCARDED); - when(counter.getCount()).thenReturn(COUNT); + // given + metricsFacade.subscriptions().discarded(subscription).increment(COUNT); - zookeeperCounterReporter.report(EMPTY_GAUGES, counters, EMPTY_HISTOGRAMS, EMPTY_METERS, EMPTY_TIMERS); + // when + zookeeperCounterReporter.report(); - verify(counterStorage).setSubscriptionDiscardedCounter( - QUALIFIED_TOPIC_NAME, SUBSCRIPTION_NAME, COUNT - ); + // then + verify(counterStorage).setSubscriptionDiscardedCounter(topic, SUBSCRIPTION_NAME, COUNT); } @Test public void shouldReportSubscriptionVolumeCounter() { - SortedMap meters = new TreeMap<>(); - meters.put(METRIC_NAME_FOR_SUBSCRIPTION_THROUGHPUT, meter); - when(meter.getCount()).thenReturn(COUNT); + // given + metricsFacade.subscriptions().throughputInBytes(subscription).increment(COUNT); - zookeeperCounterReporter.report(EMPTY_GAUGES, EMPTY_COUNTERS, EMPTY_HISTOGRAMS, meters, EMPTY_TIMERS); + // when + zookeeperCounterReporter.report(); - verify(counterStorage).incrementVolumeCounter( - QUALIFIED_TOPIC_NAME, SUBSCRIPTION_NAME, COUNT - ); + // then + verify(counterStorage).incrementVolumeCounter(topic, SUBSCRIPTION_NAME, COUNT); } @Test public void shouldReportTopicVolumeCounter() { - SortedMap meters = new TreeMap<>(); - meters.put(METRIC_NAME_FOR_TOPIC_THRESHOLD, meter); - when(meter.getCount()).thenReturn(COUNT); + // given + metricsFacade.topics().topicThroughputBytes(topic).increment(COUNT); - zookeeperCounterReporter.report(EMPTY_GAUGES, EMPTY_COUNTERS, EMPTY_HISTOGRAMS, meters, EMPTY_TIMERS); + // when + zookeeperCounterReporter.report(); - verify(counterStorage).incrementVolumeCounter( - QUALIFIED_TOPIC_NAME, COUNT - ); + // then + verify(counterStorage).incrementVolumeCounter(topic, COUNT); } - - private SortedMap prepareCounters(String metricName) { - SortedMap counters = new TreeMap<>(); - counters.put(metricName, counter); - - return counters; - } - } \ No newline at end of file diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java index 6fdd451dad..260e224196 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/CommonConfiguration.java @@ -251,10 +251,9 @@ public MetricsFacade metricsFacade(MeterRegistry meterRegistry, HermesMetrics he @Bean public MetricRegistry metricRegistry(MetricsProperties metricsProperties, GraphiteProperties graphiteProperties, - CounterStorage counterStorage, InstanceIdResolver instanceIdResolver, @Named("moduleName") String moduleName) { - return new MetricRegistryFactory(metricsProperties, graphiteProperties, counterStorage, instanceIdResolver, moduleName) + return new MetricRegistryFactory(metricsProperties, graphiteProperties, instanceIdResolver, moduleName) .provide(); } @@ -265,9 +264,10 @@ PrometheusConfig prometheusConfig(PrometheusProperties properties) { @Bean public PrometheusMeterRegistry micrometerRegistry(MicrometerRegistryParameters micrometerRegistryParameters, - PrometheusConfig prometheusConfig) { + PrometheusConfig prometheusConfig, + CounterStorage counterStorage) { return new PrometheusMeterRegistryFactory(micrometerRegistryParameters, - prometheusConfig, "hermes-consumers").provide(); + prometheusConfig, counterStorage, "hermes-consumers").provide(); } @Bean diff --git a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/MicrometerRegistryProperties.java b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/MicrometerRegistryProperties.java index cfb07ffc96..d5431b7de1 100644 --- a/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/MicrometerRegistryProperties.java +++ b/hermes-consumers/src/main/java/pl/allegro/tech/hermes/consumers/config/MicrometerRegistryProperties.java @@ -3,19 +3,48 @@ import org.springframework.boot.context.properties.ConfigurationProperties; import pl.allegro.tech.hermes.common.di.factories.MicrometerRegistryParameters; +import java.time.Duration; import java.util.List; @ConfigurationProperties(prefix = "consumer.metrics.micrometer") public class MicrometerRegistryProperties implements MicrometerRegistryParameters { private List percentiles = List.of(0.5, 0.99, 0.999); + private boolean zookeeperReporterEnabled = true; + private Duration reportPeriod = Duration.ofSeconds(20); @Override public List getPercentiles() { return percentiles; } + @Override + public boolean zookeeperReporterEnabled() { + return zookeeperReporterEnabled; + } + + @Override + public Duration zookeeperReportPeriod() { + return reportPeriod; + } + public void setPercentiles(List percentiles) { this.percentiles = percentiles; } + + public boolean isZookeeperReporterEnabled() { + return zookeeperReporterEnabled; + } + + public void setZookeeperReporterEnabled(boolean zookeeperReporterEnabled) { + this.zookeeperReporterEnabled = zookeeperReporterEnabled; + } + + public Duration getReportPeriod() { + return reportPeriod; + } + + public void setReportPeriod(Duration reportPeriod) { + this.reportPeriod = reportPeriod; + } } diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/CommonConfiguration.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/CommonConfiguration.java index c3da9b601b..6ee77fb667 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/CommonConfiguration.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/CommonConfiguration.java @@ -301,10 +301,9 @@ public MetricsFacade micrometerHermesMetrics(MeterRegistry meterRegistry, Hermes @Bean public MetricRegistry metricRegistry(MetricRegistryProperties metricRegistryProperties, GraphiteProperties graphiteProperties, - CounterStorage counterStorage, InstanceIdResolver instanceIdResolver, @Named("moduleName") String moduleName) { - return new MetricRegistryFactory(metricRegistryProperties, graphiteProperties, counterStorage, + return new MetricRegistryFactory(metricRegistryProperties, graphiteProperties, instanceIdResolver, moduleName).provide(); } @@ -315,9 +314,10 @@ PrometheusConfig prometheusConfig(PrometheusProperties properties) { @Bean public PrometheusMeterRegistry micrometerRegistry(MicrometerRegistryParameters micrometerRegistryParameters, - PrometheusConfig prometheusConfig) { + PrometheusConfig prometheusConfig, + CounterStorage counterStorage) { return new PrometheusMeterRegistryFactory(micrometerRegistryParameters, - prometheusConfig, "hermes-frontend").provide(); + prometheusConfig, counterStorage, "hermes-frontend").provide(); } @Bean diff --git a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/MicrometerRegistryProperties.java b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/MicrometerRegistryProperties.java index 52358007d7..8a99c812bf 100644 --- a/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/MicrometerRegistryProperties.java +++ b/hermes-frontend/src/main/java/pl/allegro/tech/hermes/frontend/config/MicrometerRegistryProperties.java @@ -3,6 +3,7 @@ import org.springframework.boot.context.properties.ConfigurationProperties; import pl.allegro.tech.hermes.common.di.factories.MicrometerRegistryParameters; +import java.time.Duration; import java.util.List; @@ -10,13 +11,41 @@ public class MicrometerRegistryProperties implements MicrometerRegistryParameters { private List percentiles = List.of(0.5, 0.99, 0.999); + private boolean zookeeperReporterEnabled = true; + private Duration reportPeriod = Duration.ofSeconds(20); @Override public List getPercentiles() { return percentiles; } + @Override + public boolean zookeeperReporterEnabled() { + return zookeeperReporterEnabled; + } + + @Override + public Duration zookeeperReportPeriod() { + return reportPeriod; + } + public void setPercentiles(List percentiles) { this.percentiles = percentiles; } + + public boolean isZookeeperReporterEnabled() { + return zookeeperReporterEnabled; + } + + public void setZookeeperReporterEnabled(boolean zookeeperReporterEnabled) { + this.zookeeperReporterEnabled = zookeeperReporterEnabled; + } + + public Duration getReportPeriod() { + return reportPeriod; + } + + public void setReportPeriod(Duration reportPeriod) { + this.reportPeriod = reportPeriod; + } } diff --git a/hermes-management/build.gradle b/hermes-management/build.gradle index 55579b7110..eeed48aa5b 100644 --- a/hermes-management/build.gradle +++ b/hermes-management/build.gradle @@ -55,7 +55,10 @@ task buildHermesConsole(type: Exec) { task attachHermesConsole(type: Copy, dependsOn: 'buildHermesConsole') { from '../hermes-console/dist/static' - into(sourceSets.main.output.resourcesDir.path + '/static') + def staticDirectory = sourceSets.main.output.resourcesDir.path + '/static' + // remove previous static dir if exists and start with clear setup + delete staticDirectory + into(staticDirectory) } tasks.register('prepareIndexTemplate') { diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java index 72b0b0334b..ea9c71775f 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java @@ -8,8 +8,6 @@ import com.fasterxml.jackson.databind.SerializationFeature; import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; import io.micrometer.core.instrument.MeterRegistry; -import io.micrometer.prometheus.PrometheusConfig; -import io.micrometer.prometheus.PrometheusMeterRegistry; import org.springframework.beans.factory.annotation.Autowired; import org.springframework.boot.autoconfigure.condition.ConditionalOnMissingBean; import org.springframework.boot.context.properties.EnableConfigurationProperties; @@ -17,8 +15,6 @@ import org.springframework.context.annotation.Configuration; import pl.allegro.tech.hermes.api.Topic; import pl.allegro.tech.hermes.common.clock.ClockFactory; -import pl.allegro.tech.hermes.common.di.factories.MicrometerRegistryParameters; -import pl.allegro.tech.hermes.common.di.factories.PrometheusMeterRegistryFactory; import pl.allegro.tech.hermes.common.metric.HermesMetrics; import pl.allegro.tech.hermes.common.metric.MetricsFacade; import pl.allegro.tech.hermes.common.util.InetAddressInstanceIdResolver; @@ -66,20 +62,6 @@ public MetricRegistry metricRegistry() { return new MetricRegistry(); } - @Bean - @ConditionalOnMissingBean - public PrometheusMeterRegistry micrometerRegistry(MicrometerRegistryParameters micrometerRegistryParameters, - PrometheusConfig prometheusConfig) { - return new PrometheusMeterRegistryFactory(micrometerRegistryParameters, - prometheusConfig, "hermes-management").provide(); - } - - @Bean - @ConditionalOnMissingBean - PrometheusConfig prometheusConfig(PrometheusProperties properties) { - return new PrometheusConfigAdapter(properties); - } - @Bean public InstanceIdResolver instanceIdResolver() { return new InetAddressInstanceIdResolver(); diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/MicrometerRegistryProperties.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/MicrometerRegistryProperties.java index 668da99c8e..9f131b66c8 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/MicrometerRegistryProperties.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/MicrometerRegistryProperties.java @@ -1,16 +1,14 @@ package pl.allegro.tech.hermes.management.config; import org.springframework.boot.context.properties.ConfigurationProperties; -import pl.allegro.tech.hermes.common.di.factories.MicrometerRegistryParameters; import java.util.List; @ConfigurationProperties(prefix = "metrics.micrometer") -public class MicrometerRegistryProperties implements MicrometerRegistryParameters { +public class MicrometerRegistryProperties { private List percentiles = List.of(0.5, 0.99, 0.999); - @Override public List getPercentiles() { return percentiles; } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusConfiguration.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusConfiguration.java new file mode 100644 index 0000000000..b7bbb4b1a0 --- /dev/null +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusConfiguration.java @@ -0,0 +1,65 @@ +package pl.allegro.tech.hermes.management.config; + +import io.micrometer.core.instrument.Meter; +import io.micrometer.core.instrument.config.MeterFilter; +import io.micrometer.core.instrument.distribution.DistributionStatisticConfig; +import io.micrometer.prometheus.PrometheusConfig; +import io.micrometer.prometheus.PrometheusMeterRegistry; +import org.springframework.boot.autoconfigure.condition.ConditionalOnMissingBean; +import org.springframework.boot.context.properties.EnableConfigurationProperties; +import org.springframework.context.annotation.Bean; + +@EnableConfigurationProperties(MicrometerRegistryProperties.class) +public class PrometheusConfiguration { + + @Bean + @ConditionalOnMissingBean + public PrometheusMeterRegistry micrometerRegistry(MicrometerRegistryProperties properties, + PrometheusConfig prometheusConfig) { + return new PrometheusMeterRegistryFactory(properties, + prometheusConfig, "hermes-management").provide(); + } + + @Bean + @ConditionalOnMissingBean + PrometheusConfig prometheusConfig(PrometheusProperties properties) { + return new PrometheusConfigAdapter(properties); + } + + + static class PrometheusMeterRegistryFactory { + private final MicrometerRegistryProperties parameters; + private final PrometheusConfig prometheusConfig; + private final String prefix; + + PrometheusMeterRegistryFactory(MicrometerRegistryProperties properties, + PrometheusConfig prometheusConfig, + String prefix) { + this.parameters = properties; + this.prometheusConfig = prometheusConfig; + this.prefix = prefix + "_"; + } + + PrometheusMeterRegistry provide() { + PrometheusMeterRegistry meterRegistry = new PrometheusMeterRegistry(prometheusConfig); + applyFilters(meterRegistry); + return meterRegistry; + } + + void applyFilters(PrometheusMeterRegistry meterRegistry) { + meterRegistry.config().meterFilter(new MeterFilter() { + @Override + public Meter.Id map(Meter.Id id) { + return id.withName(prefix + id.getName()); + } + + @Override + public DistributionStatisticConfig configure(Meter.Id id, DistributionStatisticConfig config) { + return DistributionStatisticConfig.builder() + .percentiles(parameters.getPercentiles().stream().mapToDouble(Double::doubleValue).toArray() + ).build().merge(config); + } + }); + } + } +} From 1c22e61ae5e2b3842f56cb6c8c50808b71a8e2b0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Rafa=C5=82=20Harabie=C5=84?= Date: Tue, 5 Sep 2023 13:10:51 +0200 Subject: [PATCH 3/6] Bump Wiremock to 3.0.1 (#1721) Fixes #1720 --- build.gradle | 2 +- hermes-client/build.gradle | 2 +- hermes-mock/build.gradle | 2 +- hermes-test-helper/build.gradle | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/build.gradle b/build.gradle index 1f2ea2a21f..9bd9bb73b0 100644 --- a/build.gradle +++ b/build.gradle @@ -56,7 +56,7 @@ allprojects { curator : '5.4.0', dropwizard_metrics: '4.1.0', micrometer_metrics: '1.11.1', - wiremock : '3.0.0-beta-10', + wiremock : '3.0.1', spock : '2.4-M1-groovy-4.0', groovy : '4.0.12', alpn_api : '1.1.2.v20150522', diff --git a/hermes-client/build.gradle b/hermes-client/build.gradle index 6e4bc4403a..4b1e628bfc 100644 --- a/hermes-client/build.gradle +++ b/hermes-client/build.gradle @@ -18,7 +18,7 @@ dependencies { testImplementation group: 'org.spockframework', name: 'spock-core', version: versions.spock testImplementation group: 'org.spockframework', name: 'spock-junit4', version: versions.spock - testImplementation group: 'com.github.tomakehurst', name: 'wiremock-standalone', version: versions.wiremock + testImplementation group: 'org.wiremock', name: 'wiremock-standalone', version: versions.wiremock testImplementation group: 'javax.servlet', name: 'javax.servlet-api', version: '3.1.0' testImplementation group: 'com.jayway.jsonpath', name: 'json-path', version: '2.5.0' diff --git a/hermes-mock/build.gradle b/hermes-mock/build.gradle index a99cc32bfc..1bb8e4c147 100644 --- a/hermes-mock/build.gradle +++ b/hermes-mock/build.gradle @@ -6,7 +6,7 @@ plugins { dependencies { implementation group: 'junit', name: 'junit', version: '4.11' - api group: 'com.github.tomakehurst', name: 'wiremock', version: versions.wiremock + api group: 'org.wiremock', name: 'wiremock', version: versions.wiremock implementation group: 'com.jayway.awaitility', name: 'awaitility', version: '1.6.1' api group: 'org.apache.avro', name: 'avro', version: versions.avro implementation group: 'tech.allegro.schema.json2avro', name: 'converter', version: versions.json2avro diff --git a/hermes-test-helper/build.gradle b/hermes-test-helper/build.gradle index 0b65649a8e..73a5b4a151 100644 --- a/hermes-test-helper/build.gradle +++ b/hermes-test-helper/build.gradle @@ -11,7 +11,7 @@ dependencies { implementation group: 'org.glassfish.jersey.inject', name: 'jersey-hk2', version: versions.jersey implementation group: 'org.glassfish.jersey.ext', name: 'jersey-proxy-client', version: versions.jersey api group: 'commons-io', name: 'commons-io', version: '2.4' - api group: 'com.github.tomakehurst', name: 'wiremock-standalone', version: versions.wiremock + api group: 'org.wiremock', name: 'wiremock-standalone', version: versions.wiremock api (group: 'org.apache.curator', name: 'curator-test', version: versions.curator) { exclude module: 'slf4j-log4j12' exclude module: 'log4j' From 8ea0082fea6e5034d45f10975b5ab50c2d70e844 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Rafa=C5=82=20Harabie=C5=84?= Date: Mon, 18 Sep 2023 13:51:07 +0200 Subject: [PATCH 4/6] Bump GitHub actions (#1724) * Bump GitHub actions Avoid using versions that using Node 12 environment that generate warnings in workflow runs. Use new versions instead that depend on Node 16. * Remove chmod on gradlew Git tracks executable permission. No need to change it in workflow. --- .github/workflows/checkstyle.yml | 4 +--- .github/workflows/ci.yml | 6 ++---- .github/workflows/codeql-analysis.yml | 2 +- .github/workflows/gradle-wrapper-validation.yml | 2 +- .github/workflows/markdown-links-check.yml | 2 +- .github/workflows/release.yml | 2 +- 6 files changed, 7 insertions(+), 11 deletions(-) diff --git a/.github/workflows/checkstyle.yml b/.github/workflows/checkstyle.yml index 6a276eb781..81e0232268 100644 --- a/.github/workflows/checkstyle.yml +++ b/.github/workflows/checkstyle.yml @@ -10,7 +10,7 @@ jobs: checkstyle: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v3 with: fetch-depth: 0 - uses: reviewdog/action-setup@v1 @@ -21,8 +21,6 @@ jobs: with: java-version: 17 distribution: 'temurin' - - name: Grant execute permission for gradlew - run: chmod +x gradlew - name: Run check style run: ./gradlew --continue clean checkstyleMain checkstyleTest checkstyleIntegration checkstyleJmh -PmaxCheckstyleWarnings=0 - name: Run reviewdog diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 80e7b4d34a..bacae24a7e 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -19,10 +19,10 @@ jobs: fail-fast: false name: ${{ matrix.tasks.alias }} steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v3 with: fetch-depth: 0 - - uses: actions/cache@v2 + - uses: actions/cache@v3 with: path: | ~/.gradle/caches @@ -35,8 +35,6 @@ jobs: with: java-version: 17 distribution: 'temurin' - - name: Grant execute permission for gradlew - run: chmod +x gradlew - name: Build with Gradle run: ./gradlew assemble - name: Run task with Gradle diff --git a/.github/workflows/codeql-analysis.yml b/.github/workflows/codeql-analysis.yml index 1370528439..56c5ff5084 100644 --- a/.github/workflows/codeql-analysis.yml +++ b/.github/workflows/codeql-analysis.yml @@ -35,7 +35,7 @@ jobs: steps: - name: Checkout repository - uses: actions/checkout@v2 + uses: actions/checkout@v3 - name: Set up JDK 17 uses: actions/setup-java@v3 diff --git a/.github/workflows/gradle-wrapper-validation.yml b/.github/workflows/gradle-wrapper-validation.yml index 73bc480bc2..d3c1263959 100644 --- a/.github/workflows/gradle-wrapper-validation.yml +++ b/.github/workflows/gradle-wrapper-validation.yml @@ -12,5 +12,5 @@ jobs: name: "Validation" runs-on: ubuntu-latest steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v3 - uses: gradle/wrapper-validation-action@v1 diff --git a/.github/workflows/markdown-links-check.yml b/.github/workflows/markdown-links-check.yml index d4ba39baba..6920411bd5 100644 --- a/.github/workflows/markdown-links-check.yml +++ b/.github/workflows/markdown-links-check.yml @@ -12,7 +12,7 @@ jobs: check-links: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v3 - uses: gaurav-nelson/github-action-markdown-link-check@v1 with: use-quiet-mode: 'yes' diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 5275e80baa..c6485311cc 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -16,7 +16,7 @@ jobs: environment: ci steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v3 with: fetch-depth: 0 - uses: gradle/wrapper-validation-action@v1 From 4b6e46e6e4dc69561cc0f799f898db28401af7e4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Rafa=C5=82=20Harabie=C5=84?= Date: Mon, 18 Sep 2023 14:06:28 +0200 Subject: [PATCH 5/6] Generate test summary in CI workflow (#1722) Parse test results and put summary + table with failed and skipped tests into GitHub job summary --- .github/workflows/ci.yml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index bacae24a7e..2faf77b4af 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -46,3 +46,9 @@ jobs: name: ${{ matrix.tasks.name }}-test-report path: '**/build/test-results/**/TEST-*.xml' retention-days: 90 + - name: Generate test summary + uses: test-summary/action@v2 + if: success() || failure() + with: + paths: '**/build/test-results/**/TEST-*.xml' + show: fail, skip From 079f0a11142c859131ef93a4bc298a527887a41f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Daniel=20F=C4=85derski?= Date: Tue, 19 Sep 2023 13:21:10 +0200 Subject: [PATCH 6/6] Fetching external metrics from Prometheus (#1711) * Fetching external metrics from Prometheus * Rewriting metrics tests for prometheus * Metrics tests based on prometheus * Unhelathy topic/subscriptions tests rewritten to prometheus * Rewrite tests to stop using Graphite metrics storage * Fix not working tests * Fix checkstyle errors * Add some fixes * Add tests for graphite external monitoring * Add code style fixes * Add code refactor * Fix error after refactor * Fix error after refactor * Fix unworking test * Fix unworking tests * Adjust prometheus config for management * Adjust prometheus config for management * Adjust prometheus config for management * Fix prometheus url encoding * Fix prometheus url encoding * Fix prometheus config * Test prometheus uri * Test prometheus uri * Cr fixes + metrics documentation. * Cr fixes --- docker/latest/management/management.yaml | 6 +- docs/docs/configuration/metrics.md | 52 +-- .../management/api/UnhealthyEndpoint.java | 2 +- ...> ExternalMonitoringClientProperties.java} | 15 +- .../ExternalMonitoringConfiguration.java | 96 +++++ .../config/GraphiteClientConfiguration.java | 75 ---- .../GraphiteMonitoringMetricsProperties.java | 14 + .../config/ManagementConfiguration.java | 1 - .../management/config/MetricsProperties.java | 27 -- ...nitoringClientPropertiesConfiguration.java | 27 ++ .../config/PrometheusConfigAdapter.java | 2 +- .../config/PrometheusConfiguration.java | 13 +- .../PrometheusMonitoringClientProperties.java | 22 + .../config/PrometheusProperties.java | 2 +- .../subscription/SubscriptionService.java | 4 - .../graphite/CachingGraphiteClient.java | 9 +- .../graphite/GraphiteClient.java | 4 +- .../graphite/GraphiteMetrics.java | 37 -- .../graphite/GraphiteMetricsProvider.java | 110 +++++ .../graphite/RestTemplateGraphiteClient.java | 9 +- .../HybridSubscriptionMetricsRepository.java | 113 +---- .../metrics/HybridTopicMetricsRepository.java | 36 +- .../metrics/MonitoringMetricsContainer.java | 46 +++ ...MonitoringSubscriptionMetricsProvider.java | 78 ++++ .../MonitoringTopicMetricsProvider.java | 42 ++ .../prometheus/CachingPrometheusClient.java | 44 ++ .../prometheus/PrometheusClient.java | 7 + .../prometheus/PrometheusResponse.java | 64 +++ .../RestTemplatePrometheusClient.java | 121 ++++++ .../VictoriaMetricsMetricsProvider.java | 102 +++++ .../hermes/management/stub/MetricsPaths.java | 15 - .../graphite/CachingGraphiteClientTest.groovy | 7 +- .../RestTemplateGraphiteClientTest.groovy | 13 +- ...dSubscriptionMetricsRepositoryTest.groovy} | 15 +- ...iteBasedTopicMetricsRepositoryTest.groovy} | 30 +- ...edSubscriptionMetricsRepositoryTest.groovy | 93 +++++ ...heusBasedTopicMetricsRepositoryTest.groovy | 55 +++ .../CachingPrometheusClientTest.groovy | 54 +++ .../RestTemplatePrometheusClientTest.groovy | 85 ++++ .../__files/full_response.json | 143 +++++++ .../__files/partial_response.json | 102 +++++ .../hermes/test/helper/client/Hermes.java | 9 +- .../helper/endpoint/HermesAPIOperations.java | 4 - .../hermes/integration/IntegrationTest.java | 8 +- .../tech/hermes/integration/MetricsTest.java | 32 +- .../integration/env/EnvironmentAware.java | 2 + .../env/HermesIntegrationEnvironment.java | 1 + .../hermes/integration/env/JmsStarter.java | 1 - .../env/PrometheusHttpMockStarter.java | 10 + .../integration/env/SharedServices.java | 4 +- .../integration/helper/GraphiteEndpoint.java | 60 +-- .../helper/PrometheusEndpoint.java | 209 ++++++++++ ...scriptionsForOwnerBasedOnGraphiteTest.java | 387 ++++++++++++++++++ ...istUnhealthySubscriptionsForOwnerTest.java | 114 +++--- .../management/QueryEndpointTest.java | 29 +- .../SubscriptionManagementTest.java | 29 +- .../setup/HermesManagementInstance.java | 13 +- .../src/test/resources/application.yaml | 10 +- 58 files changed, 2144 insertions(+), 570 deletions(-) rename hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/{GraphiteClientProperties.java => ExternalMonitoringClientProperties.java} (81%) create mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringConfiguration.java delete mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/GraphiteClientConfiguration.java create mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/GraphiteMonitoringMetricsProperties.java delete mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/MetricsProperties.java create mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/MonitoringClientPropertiesConfiguration.java create mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusMonitoringClientProperties.java delete mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteMetrics.java create mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteMetricsProvider.java create mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/MonitoringMetricsContainer.java create mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/MonitoringSubscriptionMetricsProvider.java create mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/MonitoringTopicMetricsProvider.java create mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/CachingPrometheusClient.java create mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusClient.java create mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusResponse.java create mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClient.java create mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/VictoriaMetricsMetricsProvider.java delete mode 100644 hermes-management/src/main/java/pl/allegro/tech/hermes/management/stub/MetricsPaths.java rename hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/{HybridSubscriptionMetricsRepositoryTest.groovy => HybridGraphiteBasedSubscriptionMetricsRepositoryTest.groovy} (90%) rename hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/{HybridTopicMetricsRepositoryTest.groovy => HybridGraphiteBasedTopicMetricsRepositoryTest.groovy} (79%) create mode 100644 hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedSubscriptionMetricsRepositoryTest.groovy create mode 100644 hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedTopicMetricsRepositoryTest.groovy create mode 100644 hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/CachingPrometheusClientTest.groovy create mode 100644 hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClientTest.groovy create mode 100644 hermes-management/src/test/resources/prometheus-stubs/__files/full_response.json create mode 100644 hermes-management/src/test/resources/prometheus-stubs/__files/partial_response.json create mode 100644 integration/src/integration/java/pl/allegro/tech/hermes/integration/env/PrometheusHttpMockStarter.java create mode 100644 integration/src/integration/java/pl/allegro/tech/hermes/integration/helper/PrometheusEndpoint.java create mode 100644 integration/src/integration/java/pl/allegro/tech/hermes/integration/management/ListUnhealthySubscriptionsForOwnerBasedOnGraphiteTest.java diff --git a/docker/latest/management/management.yaml b/docker/latest/management/management.yaml index b76744b5a8..0953e3660d 100644 --- a/docker/latest/management/management.yaml +++ b/docker/latest/management/management.yaml @@ -17,10 +17,10 @@ kafka: connectionTimeout: 3000 bootstrapKafkaServer: kafka:29092 -metrics: - graphiteHttpUri: graphite:8082 - graphite: +graphite: + client: enabled: true + externalMonitoringUrl: graphite:8082 server: port: 8090 diff --git a/docs/docs/configuration/metrics.md b/docs/docs/configuration/metrics.md index aa1c01d5c9..86f78fe134 100644 --- a/docs/docs/configuration/metrics.md +++ b/docs/docs/configuration/metrics.md @@ -1,44 +1,20 @@ # Metrics -Hermes Frontend and Consumers use [Dropwizard Metrics](https://dropwizard.github.io/metrics/3.1.0/) library to gather -and publish metrics to Metric Store. +Hermes Frontend, Consumers and Management use [Micrometer Metrics](https://github.com/micrometer-metrics/micrometer) library to gather +and expose metrics. -If you would like to preview or debug metrics, set `{modulePrefix}.metrics.consoleReporterEnabled` to `true`, so they will be printed -to stdout. +## Prometheus +By default, Hermes includes Prometheus reporter. It exposes metrics on `/status/prometheus` endpoint. +Reporter configuration can be configured using following options: -## Graphite +Option | Description | Default value +---------------------------------------------- |-------------------------------------------------------| ------------- +{modulePrefix}.metrics.prometheus.step | The step size to use in computing windowed statistics | 60s +{modulePrefix}.metrics.prometheus.descriptions | If meter descriptions should be sent to Prometheus | true -By default, Hermes includes Graphite reporter, which can be configured using following options: +In order to be able to access basic metrics via Management API, it needs to be configured to reach VictoriaMetrics API: -Option | Description | Default value ----------------------------------------------- | -------------------------------------- | ------------- -{modulePrefix}.metrics.graphiteReporterEnabled | enable Graphite reporter | false -{modulePrefix}.graphite.host | Graphite host | localhost -{modulePrefix}.graphite.port | Graphite port | 2003 -{modulePrefix}.graphite.prefix | prefix for all metrics | stats.tech.hermes -{modulePrefix}.metrics.reportPeriod | how often to report metrics | 20s - -In order to be able to access basic metrics via Management API, it needs to be configured to reach Graphite API: - -Option | Description | Default value ------------------------ | ------------------------ | ------------- -metrics.graphiteHttpUri | URI to Graphite HTTP API | http://localhost:80 -metrics.prefix | prefix for all metrics | stats.tech.hermes - -## Custom - -You can register any custom reporter that is compatible with Dropwizard `MetricRegistry`. - -For the Consumers and Frontend modules register the reporter as a bean, for example: - -```java -@Configuration -public class CustomHermesConsumersConfiguration { - - @Bean - @Primary - public MetricRegistry myMetricRegistry(MetricRegistry metricRegistry) { - return new MyMetricsReporter(metricRegistry); - } -} -``` +Option | Description | Default value +------------------------------------------|-----------------------------------------------| ------------- +prometheus.client.enabled | Should fetch external metrics from Prometheus | true +prometheus.client.externalMonitoringUrl | URI to VictoriaMetrics HTTP API | http://localhost:18090 diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/api/UnhealthyEndpoint.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/api/UnhealthyEndpoint.java index dae9810ab8..b28ac97321 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/api/UnhealthyEndpoint.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/api/UnhealthyEndpoint.java @@ -51,7 +51,7 @@ public Response listUnhealthy( )) .orElseThrow(() -> new OwnerSource.OwnerNotFound(ownerSourceName, id)); return Response.ok() - .entity(new GenericEntity>(unhealthySubscriptions) { + .entity(new GenericEntity<>(unhealthySubscriptions) { }) .build(); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/GraphiteClientProperties.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringClientProperties.java similarity index 81% rename from hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/GraphiteClientProperties.java rename to hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringClientProperties.java index 0ef4608d6f..92f20ea63c 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/GraphiteClientProperties.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringClientProperties.java @@ -1,9 +1,6 @@ package pl.allegro.tech.hermes.management.config; -import org.springframework.boot.context.properties.ConfigurationProperties; - -@ConfigurationProperties("graphite.client") -public class GraphiteClientProperties { +public class ExternalMonitoringClientProperties { private int connectionTimeoutMillis = 1000; @@ -17,6 +14,8 @@ public class GraphiteClientProperties { private int cacheSize = 100_000; + private String externalMonitoringUrl = "http://localhost:18090"; + public int getConnectionTimeoutMillis() { return connectionTimeoutMillis; } @@ -64,4 +63,12 @@ public int getMaxConnectionsPerRoute() { public void setMaxConnectionsPerRoute(int maxConnectionsPerRoute) { this.maxConnectionsPerRoute = maxConnectionsPerRoute; } + + public String getExternalMonitoringUrl() { + return externalMonitoringUrl; + } + + public void setExternalMonitoringUrl(String externalMonitoringUrl) { + this.externalMonitoringUrl = externalMonitoringUrl; + } } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringConfiguration.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringConfiguration.java new file mode 100644 index 0000000000..44d9816ded --- /dev/null +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ExternalMonitoringConfiguration.java @@ -0,0 +1,96 @@ +package pl.allegro.tech.hermes.management.config; + +import org.apache.hc.client5.http.classic.HttpClient; +import org.apache.hc.client5.http.config.RequestConfig; +import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; +import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManager; +import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManagerBuilder; +import org.apache.hc.core5.util.Timeout; +import org.springframework.beans.factory.annotation.Qualifier; +import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; +import org.springframework.http.client.ClientHttpRequestFactory; +import org.springframework.http.client.HttpComponentsClientHttpRequestFactory; +import org.springframework.web.client.RestTemplate; +import pl.allegro.tech.hermes.management.infrastructure.graphite.CachingGraphiteClient; +import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteClient; +import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteMetricsProvider; +import pl.allegro.tech.hermes.management.infrastructure.graphite.RestTemplateGraphiteClient; +import pl.allegro.tech.hermes.management.infrastructure.prometheus.CachingPrometheusClient; +import pl.allegro.tech.hermes.management.infrastructure.prometheus.PrometheusClient; +import pl.allegro.tech.hermes.management.infrastructure.prometheus.RestTemplatePrometheusClient; +import pl.allegro.tech.hermes.management.infrastructure.prometheus.VictoriaMetricsMetricsProvider; + +import java.net.URI; + +import static com.google.common.base.Ticker.systemTicker; + +@Configuration +public class ExternalMonitoringConfiguration { + + @Bean + @ConditionalOnProperty(value = "graphite.client.enabled", havingValue = "true") + public GraphiteMetricsProvider graphiteMetricsProvider(GraphiteClient graphiteClient, + GraphiteMonitoringMetricsProperties properties) { + return new GraphiteMetricsProvider(graphiteClient, properties.getPrefix()); + } + + @Bean + @ConditionalOnProperty(value = "graphite.client.enabled", havingValue = "true") + public GraphiteClient graphiteClient(@Qualifier("monitoringRestTemplate") RestTemplate graphiteRestTemplate, + GraphiteMonitoringMetricsProperties graphiteClientProperties) { + RestTemplateGraphiteClient underlyingGraphiteClient = + new RestTemplateGraphiteClient(graphiteRestTemplate, URI.create(graphiteClientProperties.getExternalMonitoringUrl())); + return new CachingGraphiteClient( + underlyingGraphiteClient, + systemTicker(), + graphiteClientProperties.getCacheTtlSeconds(), + graphiteClientProperties.getCacheSize() + ); + } + + @Bean + @ConditionalOnProperty(value = "prometheus.client.enabled", havingValue = "true") + public VictoriaMetricsMetricsProvider prometheusMetricsProvider(PrometheusClient prometheusClient, + PrometheusMonitoringClientProperties properties) { + return new VictoriaMetricsMetricsProvider(prometheusClient, + properties.getConsumersMetricsPrefix(), properties.getFrontendMetricsPrefix()); + } + + @Bean + @ConditionalOnProperty(value = "prometheus.client.enabled", havingValue = "true") + public PrometheusClient prometheusClient(@Qualifier("monitoringRestTemplate") RestTemplate graphiteRestTemplate, + PrometheusMonitoringClientProperties clientProperties) { + RestTemplatePrometheusClient underlyingPrometheusClient = + new RestTemplatePrometheusClient(graphiteRestTemplate, URI.create(clientProperties.getExternalMonitoringUrl())); + return new CachingPrometheusClient( + underlyingPrometheusClient, + systemTicker(), + clientProperties.getCacheTtlSeconds(), + clientProperties.getCacheSize() + ); + } + + @Bean("monitoringRestTemplate") + public RestTemplate restTemplate(ExternalMonitoringClientProperties clientProperties) { + PoolingHttpClientConnectionManager connectionManager = PoolingHttpClientConnectionManagerBuilder.create() + .setMaxConnTotal(clientProperties.getMaxConnections()) + .setMaxConnPerRoute(clientProperties.getMaxConnectionsPerRoute()) + .build(); + + RequestConfig requestConfig = RequestConfig.custom() + .setConnectTimeout(Timeout.ofMilliseconds(clientProperties.getConnectionTimeoutMillis())) + .setResponseTimeout(Timeout.ofMilliseconds(clientProperties.getSocketTimeoutMillis())) + .build(); + + HttpClient client = HttpClientBuilder.create() + .setDefaultRequestConfig(requestConfig) + .setConnectionManager(connectionManager) + .build(); + + ClientHttpRequestFactory clientHttpRequestFactory = new HttpComponentsClientHttpRequestFactory(client); + + return new RestTemplate(clientHttpRequestFactory); + } +} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/GraphiteClientConfiguration.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/GraphiteClientConfiguration.java deleted file mode 100644 index ec5c9e6263..0000000000 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/GraphiteClientConfiguration.java +++ /dev/null @@ -1,75 +0,0 @@ -package pl.allegro.tech.hermes.management.config; - -import org.apache.hc.client5.http.classic.HttpClient; -import org.apache.hc.client5.http.config.RequestConfig; -import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; -import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManager; -import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManagerBuilder; -import org.apache.hc.core5.util.Timeout; -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.beans.factory.annotation.Qualifier; -import org.springframework.boot.context.properties.EnableConfigurationProperties; -import org.springframework.context.annotation.Bean; -import org.springframework.context.annotation.Configuration; -import org.springframework.http.client.ClientHttpRequestFactory; -import org.springframework.http.client.HttpComponentsClientHttpRequestFactory; -import org.springframework.web.client.RestTemplate; -import pl.allegro.tech.hermes.management.infrastructure.graphite.CachingGraphiteClient; -import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteClient; -import pl.allegro.tech.hermes.management.infrastructure.graphite.RestTemplateGraphiteClient; -import pl.allegro.tech.hermes.management.stub.MetricsPaths; - -import java.net.URI; - -import static com.google.common.base.Ticker.systemTicker; - -@Configuration -@EnableConfigurationProperties({MetricsProperties.class, GraphiteClientProperties.class}) -public class GraphiteClientConfiguration { - - @Autowired - MetricsProperties metricsProperties; - - @Autowired - GraphiteClientProperties graphiteClientProperties; - - - @Bean - public MetricsPaths metricsPaths() { - return new MetricsPaths(metricsProperties.getPrefix()); - } - - @Bean - public GraphiteClient graphiteClient(@Qualifier("graphiteRestTemplate") RestTemplate graphiteRestTemplate) { - RestTemplateGraphiteClient underlyingGraphiteClient = - new RestTemplateGraphiteClient(graphiteRestTemplate, URI.create(metricsProperties.getGraphiteHttpUri())); - return new CachingGraphiteClient( - underlyingGraphiteClient, - systemTicker(), - graphiteClientProperties.getCacheTtlSeconds(), - graphiteClientProperties.getCacheSize() - ); - } - - @Bean("graphiteRestTemplate") - public RestTemplate restTemplate(GraphiteClientProperties properties) { - PoolingHttpClientConnectionManager connectionManager = PoolingHttpClientConnectionManagerBuilder.create() - .setMaxConnTotal(properties.getMaxConnections()) - .setMaxConnPerRoute(properties.getMaxConnectionsPerRoute()) - .build(); - - RequestConfig requestConfig = RequestConfig.custom() - .setConnectTimeout(Timeout.ofMilliseconds(properties.getConnectionTimeoutMillis())) - .setResponseTimeout(Timeout.ofMilliseconds(properties.getSocketTimeoutMillis())) - .build(); - - HttpClient client = HttpClientBuilder.create() - .setDefaultRequestConfig(requestConfig) - .setConnectionManager(connectionManager) - .build(); - - ClientHttpRequestFactory clientHttpRequestFactory = new HttpComponentsClientHttpRequestFactory(client); - - return new RestTemplate(clientHttpRequestFactory); - } -} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/GraphiteMonitoringMetricsProperties.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/GraphiteMonitoringMetricsProperties.java new file mode 100644 index 0000000000..4ae837902f --- /dev/null +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/GraphiteMonitoringMetricsProperties.java @@ -0,0 +1,14 @@ +package pl.allegro.tech.hermes.management.config; + +public class GraphiteMonitoringMetricsProperties extends ExternalMonitoringClientProperties { + + private String prefix = "stats.tech.hermes"; + + public String getPrefix() { + return prefix; + } + + public void setPrefix(String prefix) { + this.prefix = prefix; + } +} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java index ea9c71775f..8277843789 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/ManagementConfiguration.java @@ -28,7 +28,6 @@ @Configuration @EnableConfigurationProperties({ TopicProperties.class, - MetricsProperties.class, HttpClientProperties.class, ConsistencyCheckerProperties.class, PrometheusProperties.class, diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/MetricsProperties.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/MetricsProperties.java deleted file mode 100644 index 326f60443c..0000000000 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/MetricsProperties.java +++ /dev/null @@ -1,27 +0,0 @@ -package pl.allegro.tech.hermes.management.config; - -import org.springframework.boot.context.properties.ConfigurationProperties; - -@ConfigurationProperties(prefix = "metrics") -public class MetricsProperties { - - private String graphiteHttpUri = "http://localhost"; - - private String prefix = "stats.tech.hermes"; - - public String getGraphiteHttpUri() { - return graphiteHttpUri; - } - - public void setGraphiteHttpUri(String graphiteHttpUri) { - this.graphiteHttpUri = graphiteHttpUri; - } - - public String getPrefix() { - return prefix; - } - - public void setPrefix(String prefix) { - this.prefix = prefix; - } -} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/MonitoringClientPropertiesConfiguration.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/MonitoringClientPropertiesConfiguration.java new file mode 100644 index 0000000000..a67745dc33 --- /dev/null +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/MonitoringClientPropertiesConfiguration.java @@ -0,0 +1,27 @@ +package pl.allegro.tech.hermes.management.config; + +import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty; +import org.springframework.boot.context.properties.ConfigurationProperties; +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; + +/* +These properties beans must be in different configuration class than ExternalMonitoringConfiguration.java. It allows +avoiding circular dependencies between beans. + */ +@Configuration +public class MonitoringClientPropertiesConfiguration { + @Bean + @ConfigurationProperties("graphite.client") + @ConditionalOnProperty(value = "graphite.client.enabled", havingValue = "true") + public GraphiteMonitoringMetricsProperties graphiteMonitoringClientProperties() { + return new GraphiteMonitoringMetricsProperties(); + } + + @Bean + @ConfigurationProperties("prometheus.client") + @ConditionalOnProperty(value = "prometheus.client.enabled", havingValue = "true") + public PrometheusMonitoringClientProperties prometheusMonitoringClientProperties() { + return new PrometheusMonitoringClientProperties(); + } +} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusConfigAdapter.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusConfigAdapter.java index fcdddca0a6..b7bc154b40 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusConfigAdapter.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusConfigAdapter.java @@ -14,7 +14,7 @@ public PrometheusConfigAdapter(PrometheusProperties prometheusReporterProperties @Override public boolean descriptions() { - return prometheusReporterProperties.isDescriptions(); + return prometheusReporterProperties.getDescriptions(); } @Override diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusConfiguration.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusConfiguration.java index b7bbb4b1a0..c5d9ed53ff 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusConfiguration.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusConfiguration.java @@ -8,7 +8,9 @@ import org.springframework.boot.autoconfigure.condition.ConditionalOnMissingBean; import org.springframework.boot.context.properties.EnableConfigurationProperties; import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; +@Configuration @EnableConfigurationProperties(MicrometerRegistryProperties.class) public class PrometheusConfiguration { @@ -22,17 +24,16 @@ public PrometheusMeterRegistry micrometerRegistry(MicrometerRegistryProperties p @Bean @ConditionalOnMissingBean - PrometheusConfig prometheusConfig(PrometheusProperties properties) { + public PrometheusConfig prometheusConfig(PrometheusProperties properties) { return new PrometheusConfigAdapter(properties); } - - static class PrometheusMeterRegistryFactory { + public static class PrometheusMeterRegistryFactory { private final MicrometerRegistryProperties parameters; private final PrometheusConfig prometheusConfig; private final String prefix; - PrometheusMeterRegistryFactory(MicrometerRegistryProperties properties, + public PrometheusMeterRegistryFactory(MicrometerRegistryProperties properties, PrometheusConfig prometheusConfig, String prefix) { this.parameters = properties; @@ -40,13 +41,13 @@ static class PrometheusMeterRegistryFactory { this.prefix = prefix + "_"; } - PrometheusMeterRegistry provide() { + public PrometheusMeterRegistry provide() { PrometheusMeterRegistry meterRegistry = new PrometheusMeterRegistry(prometheusConfig); applyFilters(meterRegistry); return meterRegistry; } - void applyFilters(PrometheusMeterRegistry meterRegistry) { + private void applyFilters(PrometheusMeterRegistry meterRegistry) { meterRegistry.config().meterFilter(new MeterFilter() { @Override public Meter.Id map(Meter.Id id) { diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusMonitoringClientProperties.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusMonitoringClientProperties.java new file mode 100644 index 0000000000..4c0977f192 --- /dev/null +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusMonitoringClientProperties.java @@ -0,0 +1,22 @@ +package pl.allegro.tech.hermes.management.config; + +public class PrometheusMonitoringClientProperties extends ExternalMonitoringClientProperties { + private String consumersMetricsPrefix = "hermes_consumers"; + private String frontendMetricsPrefix = "hermes_frontend"; + + public String getConsumersMetricsPrefix() { + return consumersMetricsPrefix; + } + + public void setConsumersMetricsPrefix(String consumersMetricsPrefix) { + this.consumersMetricsPrefix = consumersMetricsPrefix; + } + + public String getFrontendMetricsPrefix() { + return frontendMetricsPrefix; + } + + public void setFrontendMetricsPrefix(String frontendMetricsPrefix) { + this.frontendMetricsPrefix = frontendMetricsPrefix; + } +} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusProperties.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusProperties.java index f5f5f107a4..1dcebc1eaf 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusProperties.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/config/PrometheusProperties.java @@ -18,7 +18,7 @@ public void setStep(Duration step) { this.step = step; } - public boolean isDescriptions() { + public boolean getDescriptions() { return this.descriptions; } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/SubscriptionService.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/SubscriptionService.java index a4238f6efb..6afe91eeae 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/SubscriptionService.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/domain/subscription/SubscriptionService.java @@ -340,10 +340,6 @@ private List getUnhealthyList(Collection> getOwnerSubscriptions(OwnerId ownerId) { - return CompletableFuture.supplyAsync(() -> getForOwnerId(ownerId)); - } - private List> filterSubscriptions(Collection subscriptions, boolean respectMonitoringSeverity, List subscriptionNames, diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/CachingGraphiteClient.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/CachingGraphiteClient.java index b4a3c38352..aeae87869d 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/CachingGraphiteClient.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/CachingGraphiteClient.java @@ -5,6 +5,7 @@ import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import pl.allegro.tech.hermes.api.MetricDecimalValue; +import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer; import java.util.Map; import java.util.concurrent.ExecutionException; @@ -32,10 +33,10 @@ public CachingGraphiteClient(GraphiteClient underlyingGraphiteClient, Ticker tic } @Override - public GraphiteMetrics readMetrics(String... metricPaths) { + public MonitoringMetricsContainer readMetrics(String... metricPaths) { try { Map graphiteMetrics = graphiteMetricsCache.getAll(asList(metricPaths)); - return new GraphiteMetrics(graphiteMetrics); + return MonitoringMetricsContainer.initialized(graphiteMetrics); } catch (ExecutionException e) { // should never happen because the loader does not throw any checked exceptions throw new RuntimeException(e); @@ -51,10 +52,10 @@ public MetricDecimalValue load(String metricPath) { @Override public Map loadAll(Iterable metricPaths) { String[] metricPathsArray = toArray(metricPaths, String.class); - GraphiteMetrics graphiteMetrics = underlyingGraphiteClient.readMetrics(metricPathsArray); + MonitoringMetricsContainer metricsContainer = underlyingGraphiteClient.readMetrics(metricPathsArray); return stream(metricPathsArray).collect(toMap( identity(), - graphiteMetrics::metricValue + metricsContainer::metricValue )); } } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteClient.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteClient.java index e8a4b2cba7..489991f76f 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteClient.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteClient.java @@ -1,6 +1,8 @@ package pl.allegro.tech.hermes.management.infrastructure.graphite; +import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer; + public interface GraphiteClient { - GraphiteMetrics readMetrics(String... metricPaths); + MonitoringMetricsContainer readMetrics(String... metricPaths); } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteMetrics.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteMetrics.java deleted file mode 100644 index 0beddb9b38..0000000000 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteMetrics.java +++ /dev/null @@ -1,37 +0,0 @@ -package pl.allegro.tech.hermes.management.infrastructure.graphite; - -import pl.allegro.tech.hermes.api.MetricDecimalValue; - -import java.util.HashMap; -import java.util.Map; - -public class GraphiteMetrics { - - private static final MetricDecimalValue DEFAULT_VALUE = MetricDecimalValue.of("0.0"); - - private final Map metrics = new HashMap<>(); - - public GraphiteMetrics() { - } - - public GraphiteMetrics(Map metrics) { - this.metrics.putAll(metrics); - } - - public static GraphiteMetrics unavailable(String... metrics) { - GraphiteMetrics graphiteMetrics = new GraphiteMetrics(); - for (String metric : metrics) { - graphiteMetrics.addMetricValue(metric, MetricDecimalValue.unavailable()); - } - return graphiteMetrics; - } - - public GraphiteMetrics addMetricValue(String metricPath, MetricDecimalValue value) { - this.metrics.put(metricPath, value); - return this; - } - - public MetricDecimalValue metricValue(String metricPath) { - return metrics.getOrDefault(metricPath, DEFAULT_VALUE); - } -} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteMetricsProvider.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteMetricsProvider.java new file mode 100644 index 0000000000..3e355607ec --- /dev/null +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/GraphiteMetricsProvider.java @@ -0,0 +1,110 @@ +package pl.allegro.tech.hermes.management.infrastructure.graphite; + +import pl.allegro.tech.hermes.api.SubscriptionName; +import pl.allegro.tech.hermes.api.TopicName; +import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer; +import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringSubscriptionMetricsProvider; +import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringTopicMetricsProvider; + +import static pl.allegro.tech.hermes.common.metric.HermesMetrics.escapeDots; + +public class GraphiteMetricsProvider implements MonitoringSubscriptionMetricsProvider, MonitoringTopicMetricsProvider { + + private static final String SUBSCRIPTION_PATH = "%s.%s.%s"; + + private static final String SUBSCRIPTION_RATE_PATTERN = "sumSeries(%s.consumer.*.meter.%s.m1_rate)"; + private static final String SUBSCRIPTION_THROUGHPUT_PATTERN = "sumSeries(%s.consumer.*.throughput.%s.m1_rate)"; + private static final String SUBSCRIPTION_HTTP_STATUSES_PATTERN = "sumSeries(%s.consumer.*.status.%s.%s.m1_rate)"; + private static final String SUBSCRIPTION_ERROR_TIMEOUT_PATTERN = "sumSeries(%s.consumer.*.status.%s.errors.timeout.m1_rate)"; + private static final String SUBSCRIPTION_ERROR_OTHER_PATTERN = "sumSeries(%s.consumer.*.status.%s.errors.other.m1_rate)"; + private static final String SUBSCRIPTION_BATCH_RATE_PATTERN = "sumSeries(%s.consumer.*.meter.%s.batch.m1_rate)"; + + private static final String TOPIC_RATE_PATTERN = "sumSeries(%s.producer.*.meter.%s.%s.m1_rate)"; + private static final String TOPIC_DELIVERY_RATE_PATTERN = "sumSeries(%s.consumer.*.meter.%s.%s.m1_rate)"; + private static final String TOPIC_THROUGHPUT_PATTERN = "sumSeries(%s.producer.*.throughput.%s.%s.m1_rate)"; + + private final GraphiteClient graphiteClient; + private final String prefix; + + public GraphiteMetricsProvider(GraphiteClient graphiteClient, String prefix) { + this.graphiteClient = graphiteClient; + this.prefix = prefix; + } + + @Override + public MonitoringSubscriptionMetrics subscriptionMetrics(SubscriptionName name) { + String rateMetric = metricPath(name); + String timeouts = metricPathTimeouts(name); + String throughput = metricPathThroughput(name); + String otherErrors = metricPathOtherErrors(name); + String codes2xxPath = metricPathHttpStatuses(name, "2xx"); + String codes4xxPath = metricPathHttpStatuses(name, "4xx"); + String codes5xxPath = metricPathHttpStatuses(name, "5xx"); + String batchPath = metricPathBatchRate(name); + + MonitoringMetricsContainer metricsContainer = graphiteClient.readMetrics(codes2xxPath, codes4xxPath, codes5xxPath, + rateMetric, throughput, timeouts, otherErrors, batchPath); + + return MonitoringSubscriptionMetricsProvider.metricsBuilder() + .withRate(metricsContainer.metricValue(rateMetric)) + .withTimeouts(metricsContainer.metricValue(timeouts)) + .withThroughput(metricsContainer.metricValue(throughput)) + .withOtherErrors(metricsContainer.metricValue(otherErrors)) + .withCodes2xx(metricsContainer.metricValue(codes2xxPath)) + .withCode4xx(metricsContainer.metricValue(codes4xxPath)) + .withCode5xx(metricsContainer.metricValue(codes5xxPath)) + .withMetricPathBatchRate(metricsContainer.metricValue(batchPath)) + .build(); + } + + @Override + public MonitoringTopicMetrics topicMetrics(TopicName topicName) { + String rateMetric = metricPath(TOPIC_RATE_PATTERN, topicName); + String deliveryRateMetric = metricPath(TOPIC_DELIVERY_RATE_PATTERN, topicName); + String throughputMetric = metricPath(TOPIC_THROUGHPUT_PATTERN, topicName); + + MonitoringMetricsContainer metrics = graphiteClient.readMetrics(rateMetric, deliveryRateMetric, throughputMetric); + return MonitoringTopicMetricsProvider.metricsBuilder() + .withRate(metrics.metricValue(rateMetric)) + .withDeliveryRate(metrics.metricValue(deliveryRateMetric)) + .withThroughput(metrics.metricValue(throughputMetric)) + .build(); + } + + private String metricPath(SubscriptionName name) { + return String.format(SUBSCRIPTION_RATE_PATTERN, prefix, subscriptionNameToPath(name) + ); + } + + private String metricPath(String pattern, TopicName topicName) { + return String.format(pattern, prefix, escapeDots(topicName.getGroupName()), + escapeDots(topicName.getName())); + } + + private String metricPathThroughput(SubscriptionName name) { + return String.format(SUBSCRIPTION_THROUGHPUT_PATTERN, prefix, subscriptionNameToPath(name)); + } + + private String metricPathHttpStatuses(SubscriptionName name, String statusCodeClass) { + return String.format(SUBSCRIPTION_HTTP_STATUSES_PATTERN, prefix, subscriptionNameToPath(name), statusCodeClass); + } + + private String metricPathTimeouts(SubscriptionName name) { + return String.format(SUBSCRIPTION_ERROR_TIMEOUT_PATTERN, prefix, subscriptionNameToPath(name) + ); + } + + private String metricPathOtherErrors(SubscriptionName name) { + return String.format(SUBSCRIPTION_ERROR_OTHER_PATTERN, prefix, subscriptionNameToPath(name)); + } + + private String metricPathBatchRate(SubscriptionName name) { + return String.format(SUBSCRIPTION_BATCH_RATE_PATTERN, prefix, subscriptionNameToPath(name)); + } + + private String subscriptionNameToPath(SubscriptionName name) { + return String.format(SUBSCRIPTION_PATH, + escapeDots(name.getTopicName().getGroupName()), name.getTopicName().getName(), escapeDots(name.getName()) + ); + } +} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClient.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClient.java index 8c9d080ccd..baad2da445 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClient.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClient.java @@ -10,6 +10,7 @@ import org.springframework.http.ResponseEntity; import org.springframework.web.client.RestTemplate; import pl.allegro.tech.hermes.api.MetricDecimalValue; +import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer; import java.io.UnsupportedEncodingException; import java.net.URI; @@ -40,14 +41,14 @@ public RestTemplateGraphiteClient(RestTemplate restTemplate, URI graphiteUri) { } @Override - public GraphiteMetrics readMetrics(String... metricPaths) { + public MonitoringMetricsContainer readMetrics(String... metricPaths) { try { - GraphiteMetrics response = new GraphiteMetrics(); - queryGraphite(metricPaths).stream().forEach(metric -> response.addMetricValue(metric.getTarget(), getFirstValue(metric))); + MonitoringMetricsContainer response = MonitoringMetricsContainer.createEmpty(); + queryGraphite(metricPaths).forEach(metric -> response.addMetricValue(metric.getTarget(), getFirstValue(metric))); return response; } catch (Exception exception) { logger.warn("Unable to read from Graphite: {}", getRootCauseMessage(exception)); - return GraphiteMetrics.unavailable(metricPaths); + return MonitoringMetricsContainer.unavailable(); } } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridSubscriptionMetricsRepository.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridSubscriptionMetricsRepository.java index 436076ed6f..ac14b42657 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridSubscriptionMetricsRepository.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridSubscriptionMetricsRepository.java @@ -10,44 +10,26 @@ import pl.allegro.tech.hermes.infrastructure.zookeeper.ZookeeperPaths; import pl.allegro.tech.hermes.management.domain.subscription.SubscriptionLagSource; import pl.allegro.tech.hermes.management.domain.subscription.SubscriptionMetricsRepository; -import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteClient; -import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteMetrics; -import pl.allegro.tech.hermes.management.stub.MetricsPaths; +import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringSubscriptionMetricsProvider.MonitoringSubscriptionMetrics; import java.util.function.Supplier; import static org.apache.commons.lang.exception.ExceptionUtils.getRootCauseMessage; -import static pl.allegro.tech.hermes.common.metric.HermesMetrics.escapeDots; @Component public class HybridSubscriptionMetricsRepository implements SubscriptionMetricsRepository { private static final Logger logger = LoggerFactory.getLogger(HybridSubscriptionMetricsRepository.class); - private static final String SUBSCRIPTION_PATH = "%s.%s.%s"; - - private static final String SUBSCRIPTION_RATE_PATTERN = "sumSeries(%s.consumer.*.meter.%s.m1_rate)"; - private static final String SUBSCRIPTION_THROUGHPUT_PATTERN = "sumSeries(%s.consumer.*.throughput.%s.m1_rate)"; - private static final String SUBSCRIPTION_HTTP_STATUSES_PATTERN = "sumSeries(%s.consumer.*.status.%s.%s.m1_rate)"; - private static final String SUBSCRIPTION_ERROR_TIMEOUT_PATTERN = "sumSeries(%s.consumer.*.status.%s.errors.timeout.m1_rate)"; - private static final String SUBSCRIPTION_ERROR_OTHER_PATTERN = "sumSeries(%s.consumer.*.status.%s.errors.other.m1_rate)"; - private static final String SUBSCRIPTION_BATCH_RATE_PATTERN = "sumSeries(%s.consumer.*.meter.%s.batch.m1_rate)"; - - private final GraphiteClient graphiteClient; - - private final MetricsPaths metricsPaths; - + private final MonitoringSubscriptionMetricsProvider monitoringSubscriptionMetricsProvider; private final SummedSharedCounter summedSharedCounter; - private final ZookeeperPaths zookeeperPaths; - private final SubscriptionLagSource lagSource; - public HybridSubscriptionMetricsRepository(GraphiteClient graphiteClient, MetricsPaths metricsPaths, + public HybridSubscriptionMetricsRepository(MonitoringSubscriptionMetricsProvider monitoringSubscriptionMetricsProvider, SummedSharedCounter summedSharedCounter, ZookeeperPaths zookeeperPaths, SubscriptionLagSource lagSource) { - this.graphiteClient = graphiteClient; - this.metricsPaths = metricsPaths; + this.monitoringSubscriptionMetricsProvider = monitoringSubscriptionMetricsProvider; this.summedSharedCounter = summedSharedCounter; this.zookeeperPaths = zookeeperPaths; this.lagSource = lagSource; @@ -57,32 +39,22 @@ public HybridSubscriptionMetricsRepository(GraphiteClient graphiteClient, Metric public SubscriptionMetrics loadMetrics(TopicName topicName, String subscriptionName) { SubscriptionName name = new SubscriptionName(subscriptionName, topicName); - String rateMetric = metricPath(name); - String timeouts = metricPathTimeouts(name); - String throughput = metricPathThroughput(name); - String otherErrors = metricPathOtherErrors(name); - String codes2xxPath = metricPathHttpStatuses(name, "2xx"); - String codes4xxPath = metricPathHttpStatuses(name, "4xx"); - String codes5xxPath = metricPathHttpStatuses(name, "5xx"); - String batchPath = metricPathBatchRate(name); - - GraphiteMetrics graphiteMetrics = graphiteClient.readMetrics(codes2xxPath, codes4xxPath, codes5xxPath, - rateMetric, throughput, timeouts, otherErrors, batchPath); + MonitoringSubscriptionMetrics monitoringMetrics = monitoringSubscriptionMetricsProvider.subscriptionMetrics(name); ZookeeperMetrics zookeeperMetrics = readZookeeperMetrics(name); return SubscriptionMetrics.Builder.subscriptionMetrics() - .withRate(graphiteMetrics.metricValue(rateMetric)) - .withDelivered(zookeeperMetrics.delivered) + .withRate(monitoringMetrics.rate()) + .withCodes2xx(monitoringMetrics.codes2xx()) + .withCodes4xx(monitoringMetrics.code4xx()) + .withCodes5xx(monitoringMetrics.code5xx()) + .withTimeouts(monitoringMetrics.timeouts()) + .withOtherErrors(monitoringMetrics.otherErrors()) + .withThroughput(monitoringMetrics.throughput()) + .withBatchRate(monitoringMetrics.metricPathBatchRate()) .withDiscarded(zookeeperMetrics.discarded) + .withDelivered(zookeeperMetrics.delivered) .withVolume(zookeeperMetrics.volume) - .withCodes2xx(graphiteMetrics.metricValue(codes2xxPath)) - .withCodes4xx(graphiteMetrics.metricValue(codes4xxPath)) - .withCodes5xx(graphiteMetrics.metricValue(codes5xxPath)) - .withTimeouts(graphiteMetrics.metricValue(timeouts)) - .withOtherErrors(graphiteMetrics.metricValue(otherErrors)) .withLag(lagSource.getLag(topicName, subscriptionName)) - .withThroughput(graphiteMetrics.metricValue(throughput)) - .withBatchRate(graphiteMetrics.metricValue(batchPath)) .build(); } @@ -114,60 +86,5 @@ private long readZookeeperMetric(Supplier supplier, SubscriptionName name) } } - private String metricPath(SubscriptionName name) { - return String.format(SUBSCRIPTION_RATE_PATTERN, - metricsPaths.prefix(), subscriptionNameToPath(name) - ); - } - - private String metricPathThroughput(SubscriptionName name) { - return String.format(SUBSCRIPTION_THROUGHPUT_PATTERN, - metricsPaths.prefix(), subscriptionNameToPath(name) - ); - } - - private String metricPathHttpStatuses(SubscriptionName name, String statusCodeClass) { - return String.format(SUBSCRIPTION_HTTP_STATUSES_PATTERN, - metricsPaths.prefix(), subscriptionNameToPath(name), statusCodeClass - ); - } - - private String metricPathTimeouts(SubscriptionName name) { - return String.format(SUBSCRIPTION_ERROR_TIMEOUT_PATTERN, - metricsPaths.prefix(), subscriptionNameToPath(name) - ); - } - - private String metricPathOtherErrors(SubscriptionName name) { - return String.format(SUBSCRIPTION_ERROR_OTHER_PATTERN, - metricsPaths.prefix(), subscriptionNameToPath(name) - ); - } - - private String metricPathBatchRate(SubscriptionName name) { - return String.format(SUBSCRIPTION_BATCH_RATE_PATTERN, - metricsPaths.prefix(), subscriptionNameToPath(name) - ); - } - - private String subscriptionNameToPath(SubscriptionName name) { - return String.format(SUBSCRIPTION_PATH, - escapeDots(name.getTopicName().getGroupName()), name.getTopicName().getName(), escapeDots(name.getName()) - ); - } - - private static class ZookeeperMetrics { - - final long delivered; - - final long discarded; - - final long volume; - - ZookeeperMetrics(long delivered, long discarded, long volume) { - this.delivered = delivered; - this.discarded = discarded; - this.volume = volume; - } - } + private record ZookeeperMetrics(long delivered, long discarded, long volume) { } } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridTopicMetricsRepository.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridTopicMetricsRepository.java index 485632da87..90d6255994 100644 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridTopicMetricsRepository.java +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridTopicMetricsRepository.java @@ -6,24 +6,11 @@ import pl.allegro.tech.hermes.domain.subscription.SubscriptionRepository; import pl.allegro.tech.hermes.infrastructure.zookeeper.ZookeeperPaths; import pl.allegro.tech.hermes.management.domain.topic.TopicMetricsRepository; -import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteClient; -import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteMetrics; -import pl.allegro.tech.hermes.management.stub.MetricsPaths; - -import static pl.allegro.tech.hermes.common.metric.HermesMetrics.escapeDots; @Component public class HybridTopicMetricsRepository implements TopicMetricsRepository { - private static final String RATE_PATTERN = "sumSeries(%s.producer.*.meter.%s.%s.m1_rate)"; - - private static final String DELIVERY_RATE_PATTERN = "sumSeries(%s.consumer.*.meter.%s.%s.m1_rate)"; - - private static final String THROUGHPUT_PATTERN = "sumSeries(%s.producer.*.throughput.%s.%s.m1_rate)"; - - private final GraphiteClient graphiteClient; - - private final MetricsPaths metricsPaths; + private final MonitoringTopicMetricsProvider monitoringTopicMetricsProvider; private final SummedSharedCounter summedSharedCounter; @@ -31,11 +18,10 @@ public class HybridTopicMetricsRepository implements TopicMetricsRepository { private final SubscriptionRepository subscriptionRepository; - public HybridTopicMetricsRepository(GraphiteClient graphiteClient, MetricsPaths metricsPaths, + public HybridTopicMetricsRepository(MonitoringTopicMetricsProvider monitoringTopicMetricsProvider, SummedSharedCounter summedSharedCounter, ZookeeperPaths zookeeperPaths, SubscriptionRepository subscriptionRepository) { - this.graphiteClient = graphiteClient; - this.metricsPaths = metricsPaths; + this.monitoringTopicMetricsProvider = monitoringTopicMetricsProvider; this.summedSharedCounter = summedSharedCounter; this.zookeeperPaths = zookeeperPaths; this.subscriptionRepository = subscriptionRepository; @@ -43,23 +29,15 @@ public HybridTopicMetricsRepository(GraphiteClient graphiteClient, MetricsPaths @Override public TopicMetrics loadMetrics(TopicName topicName) { - String rateMetric = metricPath(RATE_PATTERN, topicName); - String deliveryRateMetric = metricPath(DELIVERY_RATE_PATTERN, topicName); - String throughputMetric = metricPath(THROUGHPUT_PATTERN, topicName); - - GraphiteMetrics metrics = graphiteClient.readMetrics(rateMetric, deliveryRateMetric); + MonitoringTopicMetricsProvider.MonitoringTopicMetrics metrics = monitoringTopicMetricsProvider.topicMetrics(topicName); return TopicMetrics.Builder.topicMetrics() - .withRate(metrics.metricValue(rateMetric)) - .withDeliveryRate(metrics.metricValue(deliveryRateMetric)) + .withRate(metrics.rate()) + .withDeliveryRate(metrics.deliveryRate()) + .withThroughput(metrics.throughput()) .withPublished(summedSharedCounter.getValue(zookeeperPaths.topicMetricPath(topicName, "published"))) .withVolume(summedSharedCounter.getValue(zookeeperPaths.topicMetricPath(topicName, "volume"))) .withSubscriptions(subscriptionRepository.listSubscriptionNames(topicName).size()) - .withThroughput(metrics.metricValue(throughputMetric)) .build(); } - - private String metricPath(String pattern, TopicName topicName) { - return String.format(pattern, metricsPaths.prefix(), escapeDots(topicName.getGroupName()), escapeDots(topicName.getName())); - } } diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/MonitoringMetricsContainer.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/MonitoringMetricsContainer.java new file mode 100644 index 0000000000..3ab5a29256 --- /dev/null +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/MonitoringMetricsContainer.java @@ -0,0 +1,46 @@ +package pl.allegro.tech.hermes.management.infrastructure.metrics; + +import pl.allegro.tech.hermes.api.MetricDecimalValue; + +import java.util.HashMap; +import java.util.Map; + +public class MonitoringMetricsContainer { + + private static final MetricDecimalValue DEFAULT_VALUE = MetricDecimalValue.of("0.0"); + + private final Map metrics; + private final boolean isAvailable; + + private MonitoringMetricsContainer(boolean isAvailable, Map metrics) { + this.metrics = metrics; + this.isAvailable = isAvailable; + } + + public static MonitoringMetricsContainer createEmpty() { + return new MonitoringMetricsContainer(true, new HashMap<>()); + } + + public static MonitoringMetricsContainer initialized(Map metrics) { + return new MonitoringMetricsContainer(true, metrics); + } + + public static MonitoringMetricsContainer unavailable() { + return new MonitoringMetricsContainer(false, new HashMap<>()); + } + + public MonitoringMetricsContainer addMetricValue(String metricPath, MetricDecimalValue value) { + if (!isAvailable) { + throw new IllegalStateException("Adding value to unavailable metrics container"); + } + this.metrics.put(metricPath, value); + return this; + } + + public MetricDecimalValue metricValue(String metricPath) { + if (!isAvailable) { + return MetricDecimalValue.unavailable(); + } + return metrics.getOrDefault(metricPath, DEFAULT_VALUE); + } +} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/MonitoringSubscriptionMetricsProvider.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/MonitoringSubscriptionMetricsProvider.java new file mode 100644 index 0000000000..d98cccdb19 --- /dev/null +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/MonitoringSubscriptionMetricsProvider.java @@ -0,0 +1,78 @@ +package pl.allegro.tech.hermes.management.infrastructure.metrics; + +import pl.allegro.tech.hermes.api.MetricDecimalValue; +import pl.allegro.tech.hermes.api.SubscriptionName; + +public interface MonitoringSubscriptionMetricsProvider { + MonitoringSubscriptionMetrics subscriptionMetrics(SubscriptionName subscriptionName); + + record MonitoringSubscriptionMetrics(MetricDecimalValue rate, + MetricDecimalValue timeouts, + MetricDecimalValue throughput, + MetricDecimalValue otherErrors, + MetricDecimalValue codes2xx, + MetricDecimalValue code4xx, + MetricDecimalValue code5xx, + MetricDecimalValue metricPathBatchRate) { + } + + static MetricsBuilder metricsBuilder() { + return new MetricsBuilder(); + } + + class MetricsBuilder { + private MetricDecimalValue rate; + private MetricDecimalValue timeouts; + private MetricDecimalValue throughput; + private MetricDecimalValue otherErrors; + private MetricDecimalValue codes2xx; + private MetricDecimalValue code4xx; + private MetricDecimalValue code5xx; + private MetricDecimalValue metricPathBatchRate; + + public MetricsBuilder withRate(MetricDecimalValue rate) { + this.rate = rate; + return this; + } + + public MetricsBuilder withTimeouts(MetricDecimalValue timeouts) { + this.timeouts = timeouts; + return this; + } + + public MetricsBuilder withThroughput(MetricDecimalValue throughput) { + this.throughput = throughput; + return this; + } + + public MetricsBuilder withOtherErrors(MetricDecimalValue otherErrors) { + this.otherErrors = otherErrors; + return this; + } + + public MetricsBuilder withCodes2xx(MetricDecimalValue codes2xx) { + this.codes2xx = codes2xx; + return this; + } + + public MetricsBuilder withCode4xx(MetricDecimalValue code4xx) { + this.code4xx = code4xx; + return this; + } + + public MetricsBuilder withCode5xx(MetricDecimalValue code5xx) { + this.code5xx = code5xx; + return this; + } + + public MetricsBuilder withMetricPathBatchRate(MetricDecimalValue metricPathBatchRate) { + this.metricPathBatchRate = metricPathBatchRate; + return this; + } + + public MonitoringSubscriptionMetrics build() { + return new MonitoringSubscriptionMetrics(rate, timeouts, throughput, otherErrors, codes2xx, + code4xx, code5xx, metricPathBatchRate); + } + } +} \ No newline at end of file diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/MonitoringTopicMetricsProvider.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/MonitoringTopicMetricsProvider.java new file mode 100644 index 0000000000..176f61f358 --- /dev/null +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/metrics/MonitoringTopicMetricsProvider.java @@ -0,0 +1,42 @@ +package pl.allegro.tech.hermes.management.infrastructure.metrics; + +import pl.allegro.tech.hermes.api.MetricDecimalValue; +import pl.allegro.tech.hermes.api.TopicName; + +public interface MonitoringTopicMetricsProvider { + MonitoringTopicMetrics topicMetrics(TopicName topicName); + + record MonitoringTopicMetrics(MetricDecimalValue rate, + MetricDecimalValue deliveryRate, + MetricDecimalValue throughput) { + } + + static MetricsBuilder metricsBuilder() { + return new MetricsBuilder(); + } + + class MetricsBuilder { + private MetricDecimalValue rate; + private MetricDecimalValue deliveryRate; + private MetricDecimalValue throughput; + + public MetricsBuilder withRate(MetricDecimalValue rate) { + this.rate = rate; + return this; + } + + public MetricsBuilder withDeliveryRate(MetricDecimalValue deliveryRate) { + this.deliveryRate = deliveryRate; + return this; + } + + public MetricsBuilder withThroughput(MetricDecimalValue throughput) { + this.throughput = throughput; + return this; + } + + public MonitoringTopicMetrics build() { + return new MonitoringTopicMetrics(rate, deliveryRate, throughput); + } + } +} \ No newline at end of file diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/CachingPrometheusClient.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/CachingPrometheusClient.java new file mode 100644 index 0000000000..942beb67a7 --- /dev/null +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/CachingPrometheusClient.java @@ -0,0 +1,44 @@ +package pl.allegro.tech.hermes.management.infrastructure.prometheus; + +import com.google.common.base.Ticker; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer; + +import java.util.concurrent.ExecutionException; + +import static java.util.concurrent.TimeUnit.SECONDS; + +public class CachingPrometheusClient implements PrometheusClient { + + private final PrometheusClient underlyingPrometheusClient; + private final LoadingCache prometheusMetricsCache; + + public CachingPrometheusClient(PrometheusClient underlyingPrometheusClient, Ticker ticker, + long cacheTtlInSeconds, long cacheSize) { + this.underlyingPrometheusClient = underlyingPrometheusClient; + this.prometheusMetricsCache = CacheBuilder.newBuilder() + .ticker(ticker) + .expireAfterWrite(cacheTtlInSeconds, SECONDS) + .maximumSize(cacheSize) + .build(new PrometheusMetricsCacheLoader()); + } + + @Override + public MonitoringMetricsContainer readMetrics(String query) { + try { + return prometheusMetricsCache.get(query); + } catch (ExecutionException e) { + // should never happen because the loader does not throw any checked exceptions + throw new RuntimeException(e); + } + } + + private class PrometheusMetricsCacheLoader extends CacheLoader { + @Override + public MonitoringMetricsContainer load(String query) { + return underlyingPrometheusClient.readMetrics(query); + } + } +} \ No newline at end of file diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusClient.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusClient.java new file mode 100644 index 0000000000..6c5c4a5493 --- /dev/null +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusClient.java @@ -0,0 +1,7 @@ +package pl.allegro.tech.hermes.management.infrastructure.prometheus; + +import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer; + +public interface PrometheusClient { + MonitoringMetricsContainer readMetrics(String query); +} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusResponse.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusResponse.java new file mode 100644 index 0000000000..bcb3156dc7 --- /dev/null +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/PrometheusResponse.java @@ -0,0 +1,64 @@ +package pl.allegro.tech.hermes.management.infrastructure.prometheus; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Optional; + +record PrometheusResponse(@JsonProperty("status") String status, + @JsonProperty("data") Data data) { + + boolean isSuccess() { + return status.equals("success") && data.isVector(); + } + + record Data(@JsonProperty("resultType") String resultType, + @JsonProperty("result") List results) { + boolean isVector() { + return resultType.equals("vector"); + } + } + + @JsonIgnoreProperties(ignoreUnknown = true) + record VectorResult( + @JsonProperty("metric") MetricName metricName, + @JsonProperty("value") List vector) { + + private static final int VALID_VECTOR_LENGTH = 2; + private static final int SCALAR_INDEX_VALUE = 1; + + Optional getValue() { + if (vector.size() != VALID_VECTOR_LENGTH) { + return Optional.empty(); + } + return Optional.of(Double.parseDouble(vector.get(SCALAR_INDEX_VALUE))); + } + + VectorResult renameMetric(String newMetricName) { + return new VectorResult(new MetricName(newMetricName, metricName.statusCode), vector); + } + } + + @JsonIgnoreProperties(ignoreUnknown = true) + record MetricName( + @JsonProperty(value = "__name__") String name, + @JsonProperty(value = "status_code") Optional statusCode) { + boolean is2xxStatusCode() { + return hasStatusCode() && statusCode.get().startsWith("2"); + } + + boolean is4xxStatusCode() { + return hasStatusCode() && statusCode.get().startsWith("4"); + } + + boolean is5xxStatusCode() { + return hasStatusCode() && statusCode.get().startsWith("5"); + } + + private boolean hasStatusCode() { + return statusCode.isPresent(); + } + } + +} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClient.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClient.java new file mode 100644 index 0000000000..3d5139e17a --- /dev/null +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClient.java @@ -0,0 +1,121 @@ +package pl.allegro.tech.hermes.management.infrastructure.prometheus; + +import com.google.common.base.Preconditions; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.http.HttpEntity; +import org.springframework.http.HttpMethod; +import org.springframework.http.ResponseEntity; +import org.springframework.web.client.RestTemplate; +import pl.allegro.tech.hermes.api.MetricDecimalValue; +import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer; + +import java.net.URI; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.net.URLEncoder.encode; +import static java.nio.charset.StandardCharsets.UTF_8; + + +public class RestTemplatePrometheusClient implements PrometheusClient { + + private static final Logger logger = LoggerFactory.getLogger(RestTemplatePrometheusClient.class); + + private final URI prometheusUri; + private final RestTemplate restTemplate; + + public RestTemplatePrometheusClient(RestTemplate restTemplate, URI prometheusUri) { + this.restTemplate = restTemplate; + this.prometheusUri = prometheusUri; + } + + @Override + public MonitoringMetricsContainer readMetrics(String query) { + try { + PrometheusResponse response = queryPrometheus(query); + Preconditions.checkNotNull(response, "Prometheus response is null"); + Preconditions.checkState(response.isSuccess(), "Prometheus response does not contain valid data"); + + Map> metricsGroupedByName = groupMetricsByName(response); + return produceMetricsContainer(metricsGroupedByName); + } catch (Exception exception) { + logger.warn("Unable to read from Prometheus...", exception); + return MonitoringMetricsContainer.unavailable(); + } + } + + private PrometheusResponse queryPrometheus(String query) { + URI queryUri = URI.create(prometheusUri.toString() + "/api/v1/query?query=" + encode(query, UTF_8)); + + ResponseEntity response = restTemplate.exchange(queryUri, + HttpMethod.GET, HttpEntity.EMPTY, PrometheusResponse.class); + return response.getBody(); + } + + private static Map> groupMetricsByName(PrometheusResponse response) { + return response.data().results().stream() + .map(RestTemplatePrometheusClient::renameStatusCodesMetricsNames) + .collect(Collectors.groupingBy(r -> r.metricName().name())); + } + + private static MonitoringMetricsContainer produceMetricsContainer( + Map> metricsGroupedByName) { + MonitoringMetricsContainer metricsContainer = MonitoringMetricsContainer.createEmpty(); + + Stream> metricsSummedByStatusCodeFamily = metricsGroupedByName.entrySet().stream() + .map(RestTemplatePrometheusClient::sumMetricsWithTheSameName); + + metricsSummedByStatusCodeFamily.forEach(pair -> metricsContainer.addMetricValue( + pair.getKey(), + MetricDecimalValue.of(pair.getValue().toString()))); + return metricsContainer; + } + + private static PrometheusResponse.VectorResult renameStatusCodesMetricsNames(PrometheusResponse.VectorResult r) { + /* + Renames any metric containing status_code tag to the _2xx/3xx/4xx/5xx> metric name. For example: + VectorResult( + metricName=MetricName( + name=hermes_consumers_subscription_http_status_codes_total, + statusCode=Optional[200]), + vector=[...] + ) + ----> + VectorResult( + metricName=MetricName( + name=hermes_consumers_subscription_http_status_codes_total_2xx, + statusCode=Optional[200]), + vector=[...] + ) + It allows then to sum metrics accordingly to the status code family. + */ + String suffix = ""; + if (r.metricName().is2xxStatusCode()) { + suffix = "_2xx"; + } else if (r.metricName().is4xxStatusCode()) { + suffix = "_4xx"; + } else if (r.metricName().is5xxStatusCode()) { + suffix = "_5xx"; + } + return r.renameMetric(r.metricName().name() + suffix); + } + + /* + We have to sum some metrics on the client side because Prometheus does not support this kind of aggregation when using + query for multiple __name__ metrics. + */ + private static Pair sumMetricsWithTheSameName(Map.Entry> e) { + return Pair.of( + e.getKey(), + e.getValue().stream() + .map(PrometheusResponse.VectorResult::getValue) + .filter(Optional::isPresent) + .map(Optional::get) + .mapToDouble(d -> d).sum()); + } +} diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/VictoriaMetricsMetricsProvider.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/VictoriaMetricsMetricsProvider.java new file mode 100644 index 0000000000..29209b9dde --- /dev/null +++ b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/infrastructure/prometheus/VictoriaMetricsMetricsProvider.java @@ -0,0 +1,102 @@ +package pl.allegro.tech.hermes.management.infrastructure.prometheus; + +import pl.allegro.tech.hermes.api.SubscriptionName; +import pl.allegro.tech.hermes.api.TopicName; +import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer; +import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringSubscriptionMetricsProvider; +import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringTopicMetricsProvider; + +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class VictoriaMetricsMetricsProvider implements MonitoringSubscriptionMetricsProvider, MonitoringTopicMetricsProvider { + + private static final String SUBSCRIPTION_DELIVERED = "subscription_delivered_total"; + private static final String SUBSCRIPTION_TIMEOUTS = "subscription_timeouts_total"; + private static final String SUBSCRIPTION_THROUGHPUT = "subscription_throughput_bytes_total"; + private static final String SUBSCRIPTION_OTHER_ERRORS = "subscription_other_errors_total"; + private static final String SUBSCRIPTION_BATCHES = "subscription_batches_total"; + private static final String SUBSCRIPTION_STATUS_CODES = "subscription_http_status_codes_total"; + private static final String SUBSCRIPTION_STATUS_CODES_2XX = SUBSCRIPTION_STATUS_CODES + "_2xx"; + private static final String SUBSCRIPTION_STATUS_CODES_4XX = SUBSCRIPTION_STATUS_CODES + "_4xx"; + private static final String SUBSCRIPTION_STATUS_CODES_5XX = SUBSCRIPTION_STATUS_CODES + "_5xx"; + + private static final String TOPIC_RATE = "topic_requests_total"; + private static final String TOPIC_DELIVERY_RATE = "subscription_delivered_total"; + private static final String TOPIC_THROUGHPUT_RATE = "topic_throughput_bytes_total"; + + private final String consumersMetricsPrefix; + private final String frontendMetricsPrefix; + private final String subscriptionMetricsToQuery; + private final String topicMetricsToQuery; + private final PrometheusClient prometheusClient; + + public VictoriaMetricsMetricsProvider(PrometheusClient prometheusClient, String consumersMetricsPrefix, + String frontendMetricsPrefix) { + this.prometheusClient = prometheusClient; + this.consumersMetricsPrefix = consumersMetricsPrefix.isEmpty() ? "" : consumersMetricsPrefix + "_"; + this.frontendMetricsPrefix = frontendMetricsPrefix.isEmpty() ? "" : frontendMetricsPrefix + "_"; + this.subscriptionMetricsToQuery = Stream.of(SUBSCRIPTION_DELIVERED, SUBSCRIPTION_TIMEOUTS, + SUBSCRIPTION_THROUGHPUT, SUBSCRIPTION_OTHER_ERRORS, SUBSCRIPTION_BATCHES, + SUBSCRIPTION_STATUS_CODES) + .map(this::consumerMetricName) + .collect(Collectors.joining("|")); + this.topicMetricsToQuery = String.join("|", List.of( + frontendMetricName(TOPIC_RATE), + consumerMetricName(TOPIC_DELIVERY_RATE), + frontendMetricName(TOPIC_THROUGHPUT_RATE) + )); + } + + @Override + public MonitoringSubscriptionMetrics subscriptionMetrics(SubscriptionName subscriptionName) { + /* + The query is based on MetricsQL, available only in VictoriaMetrics + https://docs.victoriametrics.com/MetricsQL.html. Basic PromQL does not support `keep_metric_names` param. + */ + String queryFormat = "sum by (__name__,group,topic,subscription,status_code)" + + "(irate({__name__=~'%s',group='%s',topic='%s',subscription='%s'}[1m]) keep_metric_names)"; + String query = String.format(queryFormat, subscriptionMetricsToQuery, subscriptionName.getTopicName().getGroupName(), + subscriptionName.getTopicName().getName(), subscriptionName.getName()); + MonitoringMetricsContainer prometheusMetricsContainer = prometheusClient.readMetrics(query); + return MonitoringSubscriptionMetricsProvider + .metricsBuilder() + .withRate(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_DELIVERED))) + .withTimeouts(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_TIMEOUTS))) + .withThroughput(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_THROUGHPUT))) + .withOtherErrors(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_OTHER_ERRORS))) + .withMetricPathBatchRate(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_BATCHES))) + .withCodes2xx(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_STATUS_CODES_2XX))) + .withCode4xx(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_STATUS_CODES_4XX))) + .withCode5xx(prometheusMetricsContainer.metricValue(consumerMetricName(SUBSCRIPTION_STATUS_CODES_5XX))) + .build(); + } + + @Override + public MonitoringTopicMetrics topicMetrics(TopicName topicName) { + /* + The query is based on MetricsQL, available only in VictoriaMetrics + https://docs.victoriametrics.com/MetricsQL.html. Basic PromQL does not support `keep_metric_names` param. + */ + String queryFormat = "sum by (__name__, group, topic) (irate({__name__=~'%s', group='%s', " + + "topic='%s'}[1m]) keep_metric_names)"; + String query = String.format(queryFormat, topicMetricsToQuery, topicName.getGroupName(), topicName.getName()); + MonitoringMetricsContainer prometheusMetricsContainer = prometheusClient.readMetrics(query); + return MonitoringTopicMetricsProvider + .metricsBuilder() + .withRate(prometheusMetricsContainer.metricValue(frontendMetricName(TOPIC_RATE))) + .withDeliveryRate(prometheusMetricsContainer.metricValue(consumerMetricName(TOPIC_DELIVERY_RATE))) + .withThroughput(prometheusMetricsContainer.metricValue(frontendMetricName(TOPIC_THROUGHPUT_RATE))) + .build(); + } + + + private String consumerMetricName(String name) { + return consumersMetricsPrefix + name; + } + + private String frontendMetricName(String name) { + return frontendMetricsPrefix + name; + } +} \ No newline at end of file diff --git a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/stub/MetricsPaths.java b/hermes-management/src/main/java/pl/allegro/tech/hermes/management/stub/MetricsPaths.java deleted file mode 100644 index 90b22cd096..0000000000 --- a/hermes-management/src/main/java/pl/allegro/tech/hermes/management/stub/MetricsPaths.java +++ /dev/null @@ -1,15 +0,0 @@ -package pl.allegro.tech.hermes.management.stub; - -public class MetricsPaths { - - private final String prefix; - - public MetricsPaths(String prefix) { - this.prefix = prefix; - } - - public String prefix() { - return prefix; - } - -} diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/graphite/CachingGraphiteClientTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/graphite/CachingGraphiteClientTest.groovy index 5b6b4ad950..cde1f26f27 100644 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/graphite/CachingGraphiteClientTest.groovy +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/graphite/CachingGraphiteClientTest.groovy @@ -1,5 +1,6 @@ package pl.allegro.tech.hermes.management.infrastructure.graphite +import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer import pl.allegro.tech.hermes.test.helper.cache.FakeTicker import spock.lang.Specification import spock.lang.Subject @@ -21,7 +22,7 @@ class CachingGraphiteClientTest extends Specification { def "should return metrics from the underlying client"() { given: - underlyingClient.readMetrics("metric_1", "metric_2") >> new GraphiteMetrics([metric_1: of("1"), metric_2: of("2")]) + underlyingClient.readMetrics("metric_1", "metric_2") >> MonitoringMetricsContainer.initialized([metric_1: of("1"), metric_2: of("2")]) when: def metrics = cachingClient.readMetrics("metric_1", "metric_2") @@ -38,7 +39,7 @@ class CachingGraphiteClientTest extends Specification { cachingClient.readMetrics("metric_1", "metric_2") then: - 1 * underlyingClient.readMetrics("metric_1", "metric_2") >> new GraphiteMetrics([metric_1: of("1"), metric_2: of("2")]) + 1 * underlyingClient.readMetrics("metric_1", "metric_2") >> MonitoringMetricsContainer.initialized([metric_1: of("1"), metric_2: of("2")]) } def "should get metrics from the underlying client after TTL expires"() { @@ -48,6 +49,6 @@ class CachingGraphiteClientTest extends Specification { cachingClient.readMetrics("metric_1", "metric_2") then: - 2 * underlyingClient.readMetrics("metric_1", "metric_2") >> new GraphiteMetrics([metric_1: of("1"), metric_2: of("2")]) + 2 * underlyingClient.readMetrics("metric_1", "metric_2") >> MonitoringMetricsContainer.initialized([metric_1: of("1"), metric_2: of("2")]) } } diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClientTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClientTest.groovy index 5856e16077..e73c9be9ef 100644 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClientTest.groovy +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/graphite/RestTemplateGraphiteClientTest.groovy @@ -5,6 +5,7 @@ import com.github.tomakehurst.wiremock.junit.WireMockRule import jakarta.ws.rs.core.MediaType import org.junit.Rule import org.springframework.web.client.RestTemplate +import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer import pl.allegro.tech.hermes.test.helper.util.Ports import spock.lang.Specification @@ -32,7 +33,7 @@ class RestTemplateGraphiteClientTest extends Specification { ]) when: - GraphiteMetrics metrics = client.readMetrics("metric1", "metric2") + MonitoringMetricsContainer metrics = client.readMetrics("metric1", "metric2") then: metrics.metricValue("metric1") == of("10") @@ -44,8 +45,8 @@ class RestTemplateGraphiteClientTest extends Specification { mockGraphite([[ metric: 'metric', data: [null] ]]) when: - GraphiteMetrics metrics = client.readMetrics("metric"); - + MonitoringMetricsContainer metrics = client.readMetrics("metric"); + then: metrics.metricValue("metric1") == of("0.0") } @@ -57,7 +58,7 @@ class RestTemplateGraphiteClientTest extends Specification { ]) when: - GraphiteMetrics metrics = client.readMetrics("metric"); + MonitoringMetricsContainer metrics = client.readMetrics("metric"); then: metrics.metricValue("metric") == of("13") @@ -70,7 +71,7 @@ class RestTemplateGraphiteClientTest extends Specification { ]) when: - GraphiteMetrics metrics = client.readMetrics('sumSeries(stats.tech.hermes.*.m1_rate)'); + MonitoringMetricsContainer metrics = client.readMetrics('sumSeries(stats.tech.hermes.*.m1_rate)'); then: metrics.metricValue('sumSeries%28stats.tech.hermes.%2A.m1_rate%29') == of("13") @@ -96,5 +97,5 @@ class RestTemplateGraphiteClientTest extends Specification { String datapointsString = datapoints.collect({ "[$it, $timestamp]" }).join(',') return '{"target": "' + query + '", "datapoints": [' + datapointsString + '], "tags": []}' } - + } diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridSubscriptionMetricsRepositoryTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridGraphiteBasedSubscriptionMetricsRepositoryTest.groovy similarity index 90% rename from hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridSubscriptionMetricsRepositoryTest.groovy rename to hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridGraphiteBasedSubscriptionMetricsRepositoryTest.groovy index dd53e322ee..0e94f343ac 100644 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridSubscriptionMetricsRepositoryTest.groovy +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridGraphiteBasedSubscriptionMetricsRepositoryTest.groovy @@ -7,25 +7,24 @@ import pl.allegro.tech.hermes.api.TopicName import pl.allegro.tech.hermes.infrastructure.zookeeper.ZookeeperPaths import pl.allegro.tech.hermes.management.domain.subscription.SubscriptionLagSource import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteClient -import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteMetrics -import pl.allegro.tech.hermes.management.stub.MetricsPaths +import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteMetricsProvider import spock.lang.Specification import static pl.allegro.tech.hermes.api.MetricDecimalValue.of -class HybridSubscriptionMetricsRepositoryTest extends Specification { +class HybridGraphiteBasedSubscriptionMetricsRepositoryTest extends Specification { private GraphiteClient client = Stub(GraphiteClient) - private MetricsPaths paths = new MetricsPaths("stats") - private SummedSharedCounter summedSharedCounter = Stub(SummedSharedCounter) private ZookeeperPaths zookeeperPaths = new ZookeeperPaths("/hermes") private SubscriptionLagSource lagSource = new NoOpSubscriptionLagSource() - private HybridSubscriptionMetricsRepository repository = new HybridSubscriptionMetricsRepository(client, paths, + private GraphiteMetricsProvider graphiteMetricsProvider = new GraphiteMetricsProvider(client, "stats"); + + private HybridSubscriptionMetricsRepository repository = new HybridSubscriptionMetricsRepository(graphiteMetricsProvider, summedSharedCounter, zookeeperPaths, lagSource) def "should read subscription metrics from multiple places"() { @@ -34,7 +33,7 @@ class HybridSubscriptionMetricsRepositoryTest extends Specification { String timeouts = 'sumSeries(stats.consumer.*.status.group.topic.subscription.errors.timeout.m1_rate)' String otherErrors = 'sumSeries(stats.consumer.*.status.group.topic.subscription.errors.other.m1_rate)' - client.readMetrics(_ as String, _ as String, _ as String, rate, _ as String, timeouts, otherErrors, _ as String) >> new GraphiteMetrics() + client.readMetrics(_ as String, _ as String, _ as String, rate, _ as String, timeouts, otherErrors, _ as String) >> MonitoringMetricsContainer.createEmpty() .addMetricValue(rate, of('10')) .addMetricValue(timeouts, of('100')) .addMetricValue(otherErrors, of('1000')) @@ -58,7 +57,7 @@ class HybridSubscriptionMetricsRepositoryTest extends Specification { def "should read subscription metrics for all http status codes"() { given: client.readMetrics(getHttpStatusCodeForFamily(2), getHttpStatusCodeForFamily(4), getHttpStatusCodeForFamily(5), - _ as String, _ as String, _ as String, _ as String, _ as String) >> new GraphiteMetrics() + _ as String, _ as String, _ as String, _ as String, _ as String) >> MonitoringMetricsContainer.createEmpty() .addMetricValue(getHttpStatusCodeForFamily(2), of('2')) .addMetricValue(getHttpStatusCodeForFamily(4), of('4')) .addMetricValue(getHttpStatusCodeForFamily(5), of('5')) diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridTopicMetricsRepositoryTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridGraphiteBasedTopicMetricsRepositoryTest.groovy similarity index 79% rename from hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridTopicMetricsRepositoryTest.groovy rename to hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridGraphiteBasedTopicMetricsRepositoryTest.groovy index 2883a0dbfe..4c792c7e36 100644 --- a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridTopicMetricsRepositoryTest.groovy +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridGraphiteBasedTopicMetricsRepositoryTest.groovy @@ -5,34 +5,34 @@ import pl.allegro.tech.hermes.api.TopicName import pl.allegro.tech.hermes.domain.subscription.SubscriptionRepository import pl.allegro.tech.hermes.infrastructure.zookeeper.ZookeeperPaths import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteClient -import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteMetrics -import pl.allegro.tech.hermes.management.stub.MetricsPaths +import pl.allegro.tech.hermes.management.infrastructure.graphite.GraphiteMetricsProvider import spock.lang.Specification import static pl.allegro.tech.hermes.api.MetricDecimalValue.of -class HybridTopicMetricsRepositoryTest extends Specification { - +class HybridGraphiteBasedTopicMetricsRepositoryTest extends Specification { + private GraphiteClient client = Stub(GraphiteClient) - - private MetricsPaths paths = new MetricsPaths("stats") private SummedSharedCounter summedSharedCounter = Stub(SummedSharedCounter) - + private ZookeeperPaths zookeeperPaths = new ZookeeperPaths("/hermes") - private SubscriptionRepository subscriptionRepository = Mock(SubscriptionRepository); - - private HybridTopicMetricsRepository repository = new HybridTopicMetricsRepository(client, paths, + private SubscriptionRepository subscriptionRepository = Mock(SubscriptionRepository) + + private GraphiteMetricsProvider graphiteMetricsProvider = new GraphiteMetricsProvider(client, "stats") + + private HybridTopicMetricsRepository repository = new HybridTopicMetricsRepository(graphiteMetricsProvider, summedSharedCounter, zookeeperPaths, subscriptionRepository) - + def "should load metrics from graphite and zookeeper"() { given: String rate = 'sumSeries(stats.producer.*.meter.group.topic.m1_rate)' String deliveryRate = 'sumSeries(stats.consumer.*.meter.group.topic.m1_rate)' + String throughput = 'sumSeries(stats.producer.*.throughput.group.topic.m1_rate)' TopicName topic = new TopicName('group', 'topic') - - client.readMetrics(rate, deliveryRate) >> new GraphiteMetrics() + + client.readMetrics(rate, deliveryRate, throughput) >> MonitoringMetricsContainer.createEmpty() .addMetricValue(rate, of('10')) .addMetricValue(deliveryRate, of('20')) summedSharedCounter.getValue('/hermes/groups/group/topics/topic/metrics/published') >> 100 @@ -41,7 +41,7 @@ class HybridTopicMetricsRepositoryTest extends Specification { when: TopicMetrics metrics = repository.loadMetrics(topic) - + then: metrics.rate == of('10') metrics.deliveryRate == of('20') @@ -49,5 +49,5 @@ class HybridTopicMetricsRepositoryTest extends Specification { metrics.subscriptions == 2 metrics.volume == 1024 } - + } diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedSubscriptionMetricsRepositoryTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedSubscriptionMetricsRepositoryTest.groovy new file mode 100644 index 0000000000..e5581587f0 --- /dev/null +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedSubscriptionMetricsRepositoryTest.groovy @@ -0,0 +1,93 @@ +package pl.allegro.tech.hermes.management.infrastructure.metrics + +import pl.allegro.tech.hermes.api.MetricLongValue +import pl.allegro.tech.hermes.api.PersistentSubscriptionMetrics +import pl.allegro.tech.hermes.api.SubscriptionMetrics +import pl.allegro.tech.hermes.api.TopicName +import pl.allegro.tech.hermes.infrastructure.zookeeper.ZookeeperPaths +import pl.allegro.tech.hermes.management.domain.subscription.SubscriptionLagSource +import pl.allegro.tech.hermes.management.infrastructure.prometheus.PrometheusClient +import pl.allegro.tech.hermes.management.infrastructure.prometheus.VictoriaMetricsMetricsProvider +import spock.lang.Specification + +import static pl.allegro.tech.hermes.api.MetricDecimalValue.of + +class HybridPrometheusBasedSubscriptionMetricsRepositoryTest extends Specification { + + private PrometheusClient client = Stub(PrometheusClient) + + private SummedSharedCounter summedSharedCounter = Stub(SummedSharedCounter) + + private ZookeeperPaths zookeeperPaths = new ZookeeperPaths("/hermes") + + private SubscriptionLagSource lagSource = new NoOpSubscriptionLagSource() + + private VictoriaMetricsMetricsProvider prometheusMetricsProvider = new VictoriaMetricsMetricsProvider(client, "hermes_consumers", "hermes_frontend"); + + private HybridSubscriptionMetricsRepository repository = new HybridSubscriptionMetricsRepository(prometheusMetricsProvider, + summedSharedCounter, zookeeperPaths, lagSource) + + private static final String query = "sum by (__name__,group,topic,subscription,status_code)" + + "(irate({__name__=~'hermes_consumers_subscription_delivered_total" + + "|hermes_consumers_subscription_timeouts_total" + + "|hermes_consumers_subscription_throughput_bytes_total" + + "|hermes_consumers_subscription_other_errors_total" + + "|hermes_consumers_subscription_batches_total" + + "|hermes_consumers_subscription_http_status_codes_total'," + + "group='group',topic='topic',subscription='subscription'}[1m]) keep_metric_names)" + + def "should read subscription metrics from multiple places"() { + given: + client.readMetrics(query) >> MonitoringMetricsContainer.createEmpty() + .addMetricValue("hermes_consumers_subscription_delivered_total", of('10')) + .addMetricValue("hermes_consumers_subscription_timeouts_total", of('100')) + .addMetricValue("hermes_consumers_subscription_other_errors_total", of('1000')) + summedSharedCounter.getValue('/hermes/groups/group/topics/topic/subscriptions/subscription/metrics/delivered') >> 100 + summedSharedCounter.getValue('/hermes/groups/group/topics/topic/subscriptions/subscription/metrics/discarded') >> 1 + summedSharedCounter.getValue('/hermes/groups/group/topics/topic/subscriptions/subscription/metrics/volume') >> 16 + + when: + SubscriptionMetrics metrics = repository.loadMetrics( + new TopicName('group', 'topic'), 'subscription') + + then: + metrics.rate == of('10') + metrics.delivered == 100 + metrics.discarded == 1 + metrics.volume == 16 + metrics.timeouts == of("100") + metrics.otherErrors == of("1000") + metrics.lag == MetricLongValue.of(-1) + } + + def "should read subscription metrics for all http status codes"() { + given: + client.readMetrics(query) >> MonitoringMetricsContainer.createEmpty() + .addMetricValue("hermes_consumers_subscription_http_status_codes_total_2xx", of('2')) + .addMetricValue("hermes_consumers_subscription_http_status_codes_total_4xx", of('4')) + .addMetricValue("hermes_consumers_subscription_http_status_codes_total_5xx", of('5')) + + when: + SubscriptionMetrics metrics = repository.loadMetrics(new TopicName('group', 'topic'), 'subscription') + + then: + metrics.codes2xx == of('2') + metrics.codes4xx == of('4') + metrics.codes5xx == of('5') + } + + def "should read subscription zookeeper metrics"() { + given: + summedSharedCounter.getValue('/hermes/groups/group/topics/topic/subscriptions/subscription/metrics/delivered') >> 1000 + summedSharedCounter.getValue('/hermes/groups/group/topics/topic/subscriptions/subscription/metrics/discarded') >> 10 + summedSharedCounter.getValue('/hermes/groups/group/topics/topic/subscriptions/subscription/metrics/volume') >> 16 + + when: + PersistentSubscriptionMetrics zookeeperMetrics = repository.loadZookeeperMetrics(new TopicName('group', 'topic'), 'subscription') + + then: + zookeeperMetrics.delivered == 1000 + zookeeperMetrics.discarded == 10 + zookeeperMetrics.volume == 16 + } +} diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedTopicMetricsRepositoryTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedTopicMetricsRepositoryTest.groovy new file mode 100644 index 0000000000..0833b55a15 --- /dev/null +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/metrics/HybridPrometheusBasedTopicMetricsRepositoryTest.groovy @@ -0,0 +1,55 @@ +package pl.allegro.tech.hermes.management.infrastructure.metrics + +import pl.allegro.tech.hermes.api.TopicMetrics +import pl.allegro.tech.hermes.api.TopicName +import pl.allegro.tech.hermes.domain.subscription.SubscriptionRepository +import pl.allegro.tech.hermes.infrastructure.zookeeper.ZookeeperPaths +import pl.allegro.tech.hermes.management.infrastructure.prometheus.PrometheusClient +import pl.allegro.tech.hermes.management.infrastructure.prometheus.VictoriaMetricsMetricsProvider +import spock.lang.Specification + +import static pl.allegro.tech.hermes.api.MetricDecimalValue.of + +class HybridPrometheusBasedTopicMetricsRepositoryTest extends Specification { + + private PrometheusClient client = Stub(PrometheusClient) + + private SummedSharedCounter summedSharedCounter = Stub(SummedSharedCounter) + + private ZookeeperPaths zookeeperPaths = new ZookeeperPaths("/hermes") + + private SubscriptionRepository subscriptionRepository = Mock(SubscriptionRepository) + + private VictoriaMetricsMetricsProvider prometheusMetricsProvider = new VictoriaMetricsMetricsProvider(client, + "hermes_consumers", "hermes_frontend") + + private HybridTopicMetricsRepository repository = new HybridTopicMetricsRepository(prometheusMetricsProvider, + summedSharedCounter, zookeeperPaths, subscriptionRepository) + + def "should load metrics from graphite and zookeeper"() { + given: + String query = "sum by (__name__, group, topic) (irate({__name__=~'hermes_frontend_topic_requests_total" + + "|hermes_consumers_subscription_delivered_total" + + "|hermes_frontend_topic_throughput_bytes_total', group='group', " + + "topic='topic'}[1m]) keep_metric_names)" + TopicName topic = new TopicName('group', 'topic') + + client.readMetrics(query) >> MonitoringMetricsContainer.createEmpty() + .addMetricValue("hermes_frontend_topic_requests_total", of('10')) + .addMetricValue("hermes_consumers_subscription_delivered_total", of('20')) + summedSharedCounter.getValue('/hermes/groups/group/topics/topic/metrics/published') >> 100 + summedSharedCounter.getValue('/hermes/groups/group/topics/topic/metrics/volume') >> 1024 + subscriptionRepository.listSubscriptionNames(topic) >> ["subscription1", "subscription2"] + + when: + TopicMetrics metrics = repository.loadMetrics(topic) + + then: + metrics.rate == of('10') + metrics.deliveryRate == of('20') + metrics.published == 100 + metrics.subscriptions == 2 + metrics.volume == 1024 + } + +} \ No newline at end of file diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/CachingPrometheusClientTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/CachingPrometheusClientTest.groovy new file mode 100644 index 0000000000..1345607d0f --- /dev/null +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/CachingPrometheusClientTest.groovy @@ -0,0 +1,54 @@ +package pl.allegro.tech.hermes.management.infrastructure.prometheus + +import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer +import pl.allegro.tech.hermes.test.helper.cache.FakeTicker +import spock.lang.Specification +import spock.lang.Subject + +import java.time.Duration + +import static pl.allegro.tech.hermes.api.MetricDecimalValue.of + +class CachingPrometheusClientTest extends Specification { + static final CACHE_TTL_IN_SECONDS = 30 + static final CACHE_SIZE = 100_000 + static final CACHE_TTL = Duration.ofSeconds(CACHE_TTL_IN_SECONDS) + + def underlyingClient = Mock(PrometheusClient) + def ticker = new FakeTicker() + + @Subject + def cachingClient = new CachingPrometheusClient(underlyingClient, ticker, CACHE_TTL_IN_SECONDS, CACHE_SIZE) + + def "should return metrics from the underlying client"() { + given: + underlyingClient.readMetrics("someQuery") >> MonitoringMetricsContainer.initialized([metric_1: of("1"), metric_2: of("2")]) + + when: + def metrics = cachingClient.readMetrics("someQuery") + + then: + metrics.metricValue("metric_1") == of("1") + metrics.metricValue("metric_2") == of("2") + } + + def "should return metrics from cache while TTL has not expired"() { + when: + cachingClient.readMetrics("someQuery") + ticker.advance(CACHE_TTL.minusSeconds(1)) + cachingClient.readMetrics("someQuery") + + then: + 1 * underlyingClient.readMetrics("someQuery") >> MonitoringMetricsContainer.initialized([metric_1: of("1"), metric_2: of("2")]) + } + + def "should get metrics from the underlying client after TTL expires"() { + when: + cachingClient.readMetrics("someQuery") + ticker.advance(CACHE_TTL.plusSeconds(1)) + cachingClient.readMetrics("someQuery") + + then: + 2 * underlyingClient.readMetrics("someQuery") >> MonitoringMetricsContainer.initialized([metric_1: of("1"), metric_2: of("2")]) + } +} diff --git a/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClientTest.groovy b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClientTest.groovy new file mode 100644 index 0000000000..c44a9c8fa7 --- /dev/null +++ b/hermes-management/src/test/groovy/pl/allegro/tech/hermes/management/infrastructure/prometheus/RestTemplatePrometheusClientTest.groovy @@ -0,0 +1,85 @@ +package pl.allegro.tech.hermes.management.infrastructure.prometheus + +import com.github.tomakehurst.wiremock.client.WireMock +import com.github.tomakehurst.wiremock.junit.WireMockRule +import jakarta.ws.rs.core.MediaType +import org.junit.Rule +import org.springframework.web.client.RestTemplate +import pl.allegro.tech.hermes.management.infrastructure.metrics.MonitoringMetricsContainer +import pl.allegro.tech.hermes.test.helper.util.Ports +import spock.lang.Specification + +import java.nio.charset.StandardCharsets + +import static com.github.tomakehurst.wiremock.client.WireMock.urlEqualTo +import static com.github.tomakehurst.wiremock.core.WireMockConfiguration.wireMockConfig +import static pl.allegro.tech.hermes.api.MetricDecimalValue.of + +class RestTemplatePrometheusClientTest extends Specification { + + private static final int PROMETHEUS_HTTP_PORT = Ports.nextAvailable() + private static final String query = "sum by (__name__,group,topic,subscription,status_code)" + + "(irate({__name__=~'hermes_consumers_subscription_delivered_total" + + "|hermes_consumers_subscription_timeouts_total" + + "|hermes_consumers_subscription_throughput_bytes_total" + + "|hermes_consumers_subscription_other_errors_total" + + "|hermes_consumers_subscription_batches_total" + + "|hermes_consumers_subscription_http_status_codes_total'," + + "group='pl.allegro.tech.hermes',topic='Monitor',subscription='consumer1'}[1m]) keep_metric_names)" + + @Rule + WireMockRule wireMockRule = new WireMockRule( + wireMockConfig().port(PROMETHEUS_HTTP_PORT).usingFilesUnderClasspath("prometheus-stubs")) + + private RestTemplatePrometheusClient client + + void setup() { + RestTemplate restTemplate = new RestTemplate(); + client = new RestTemplatePrometheusClient(restTemplate, URI.create("http://localhost:$PROMETHEUS_HTTP_PORT"),); + } + + def "should get metrics for path"() { + given: + mockPrometheus(query, "full_response.json"); + + when: + MonitoringMetricsContainer metrics = client.readMetrics(query) + + then: + metrics.metricValue("hermes_consumers_subscription_delivered_total") == of("1.0") + metrics.metricValue("hermes_consumers_subscription_timeouts_total") == of("2.0") + metrics.metricValue("hermes_consumers_subscription_throughput_bytes_total") == of("3.0") + metrics.metricValue("hermes_consumers_subscription_other_errors_total") == of("4.0") + metrics.metricValue("hermes_consumers_subscription_batches_total") == of("5.0") + metrics.metricValue("hermes_consumers_subscription_http_status_codes_total_2xx") == of("2.0") + metrics.metricValue("hermes_consumers_subscription_http_status_codes_total_4xx") == of("1.0") + metrics.metricValue("hermes_consumers_subscription_http_status_codes_total_5xx") == of("2.0") + } + + def "should return default value when metric has no value"() { + given: + mockPrometheus(query, "partial_response.json") + + when: + MonitoringMetricsContainer metrics = client.readMetrics(query) + + then: + metrics.metricValue("hermes_consumers_subscription_delivered_total") == of("0.0") + metrics.metricValue("hermes_consumers_subscription_timeouts_total") == of("2.0") + metrics.metricValue("hermes_consumers_subscription_throughput_bytes_total") == of("3.0") + metrics.metricValue("hermes_consumers_subscription_other_errors_total") == of("4.0") + metrics.metricValue("hermes_consumers_subscription_batches_total") == of("5.0") + metrics.metricValue("hermes_consumers_subscription_http_status_codes_total_2xx") == of("2.0") + metrics.metricValue("hermes_consumers_subscription_http_status_codes_total_4xx") == of("1.0") + metrics.metricValue("hermes_consumers_subscription_http_status_codes_total_5xx") == of("0.0") + } + + private void mockPrometheus(String query, String responseFile) { + String encodedQuery = URLEncoder.encode(query, StandardCharsets.UTF_8) + WireMock.stubFor(WireMock.get(urlEqualTo(String.format("/api/v1/query?query=%s", encodedQuery))) + .willReturn(WireMock.aResponse() + .withStatus(200) + .withHeader("Content-Type", MediaType.APPLICATION_JSON) + .withBodyFile(responseFile))) + } +} diff --git a/hermes-management/src/test/resources/prometheus-stubs/__files/full_response.json b/hermes-management/src/test/resources/prometheus-stubs/__files/full_response.json new file mode 100644 index 0000000000..9a9f457be7 --- /dev/null +++ b/hermes-management/src/test/resources/prometheus-stubs/__files/full_response.json @@ -0,0 +1,143 @@ +{ + "status": "success", + "data": { + "resultType": "vector", + "result": [ + { + "metric": { + "__name__": "hermes_consumers_subscription_delivered_total", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "1" + ], + "group": 1 + }, + { + "metric": { + "__name__": "hermes_consumers_subscription_timeouts_total", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "2" + ], + "group": 1 + }, + { + "metric": { + "__name__": "hermes_consumers_subscription_throughput_bytes_total", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "3" + ], + "group": 1 + }, + { + "metric": { + "__name__": "hermes_consumers_subscription_other_errors_total", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "4" + ], + "group": 1 + }, + { + "metric": { + "__name__": "hermes_consumers_subscription_batches_total", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "5" + ], + "group": 1 + }, + { + "metric": { + "__name__": "hermes_consumers_subscription_http_status_codes_total", + "status_code": "200", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "1" + ], + "group": 1 + }, + { + "metric": { + "__name__": "hermes_consumers_subscription_http_status_codes_total", + "status_code": "201", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "1" + ], + "group": 1 + }, + { + "metric": { + "__name__": "hermes_consumers_subscription_http_status_codes_total", + "status_code": "401", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "1" + ], + "group": 1 + }, + { + "metric": { + "__name__": "hermes_consumers_subscription_http_status_codes_total", + "status_code": "503", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "1" + ], + "group": 1 + }, + { + "metric": { + "__name__": "hermes_consumers_subscription_http_status_codes_total", + "status_code": "500", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "1" + ], + "group": 1 + } + ] + } +} \ No newline at end of file diff --git a/hermes-management/src/test/resources/prometheus-stubs/__files/partial_response.json b/hermes-management/src/test/resources/prometheus-stubs/__files/partial_response.json new file mode 100644 index 0000000000..a0f61afb2a --- /dev/null +++ b/hermes-management/src/test/resources/prometheus-stubs/__files/partial_response.json @@ -0,0 +1,102 @@ +{ + "status": "success", + "data": { + "resultType": "vector", + "result": [ + { + "metric": { + "__name__": "hermes_consumers_subscription_timeouts_total", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "2" + ], + "group": 1 + }, + { + "metric": { + "__name__": "hermes_consumers_subscription_throughput_bytes_total", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "3" + ], + "group": 1 + }, + { + "metric": { + "__name__": "hermes_consumers_subscription_other_errors_total", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "4" + ], + "group": 1 + }, + { + "metric": { + "__name__": "hermes_consumers_subscription_batches_total", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "5" + ], + "group": 1 + }, + { + "metric": { + "__name__": "hermes_consumers_subscription_http_status_codes_total", + "status_code": "200", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "1" + ], + "group": 1 + }, + { + "metric": { + "__name__": "hermes_consumers_subscription_http_status_codes_total", + "status_code": "201", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "1" + ], + "group": 1 + }, + { + "metric": { + "__name__": "hermes_consumers_subscription_http_status_codes_total", + "status_code": "401", + "group": "pl.allegro.tech.hermes", + "subscription": "hermesSubscription", + "topic": "hermesTopic" + }, + "value": [ + 1692281425.609, + "1" + ], + "group": 1 + } + ] + } +} \ No newline at end of file diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/Hermes.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/Hermes.java index 08b4919c96..399d7204d0 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/Hermes.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/client/Hermes.java @@ -3,6 +3,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; import com.fasterxml.jackson.jakarta.rs.json.JacksonXmlBindJsonProvider; +import jakarta.ws.rs.client.ClientRequestContext; +import java.util.function.Function; import org.glassfish.jersey.client.ClientConfig; import org.glassfish.jersey.client.ClientProperties; import org.glassfish.jersey.client.proxy.WebResourceFactory; @@ -25,10 +27,8 @@ import java.util.ArrayList; import java.util.Collection; -import java.util.function.Function; import jakarta.ws.rs.Path; import jakarta.ws.rs.client.ClientBuilder; -import jakarta.ws.rs.client.ClientRequestContext; import jakarta.ws.rs.client.ClientRequestFilter; import jakarta.ws.rs.client.WebTarget; @@ -166,11 +166,6 @@ public OfflineRetransmissionEndpoint createOfflineRetransmissionEndpoint() { return createProxy(url, OfflineRetransmissionEndpoint.class, managementConfig); } - public AsyncMessagePublisher createAsyncMessagePublisher() { - String resource = TopicEndpoint.class.getAnnotation(Path.class).value(); - return new AsyncMessagePublisher(getClientBuilder(publisherConfig).build().target(url).path(resource)); - } - public WebTarget createWebTargetForPublishing() { String resource = TopicEndpoint.class.getAnnotation(Path.class).value(); return getClientBuilder(publisherConfig).build().target(url).path(resource); diff --git a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/endpoint/HermesAPIOperations.java b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/endpoint/HermesAPIOperations.java index ce34e6bde3..2c2e5cd898 100644 --- a/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/endpoint/HermesAPIOperations.java +++ b/hermes-test-helper/src/main/java/pl/allegro/tech/hermes/test/helper/endpoint/HermesAPIOperations.java @@ -177,10 +177,6 @@ public Response suspendSubscription(Topic topic, String subscription) { return endpoints.subscription().updateState(topic.getQualifiedName(), subscription, Subscription.State.SUSPENDED); } - public Response activateSubscription(Topic topic, String subscription) { - return endpoints.subscription().updateState(topic.getQualifiedName(), subscription, Subscription.State.ACTIVE); - } - public void updateSubscription(String group, String topic, String subscription, PatchData patch) { String qualifiedTopicName = group + "." + topic; diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/IntegrationTest.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/IntegrationTest.java index 0c309f2916..c65011901e 100644 --- a/integration/src/integration/java/pl/allegro/tech/hermes/integration/IntegrationTest.java +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/IntegrationTest.java @@ -53,8 +53,8 @@ public void initializeIntegrationTest() { public void after() { auditEvents.reset(); try { - removeSubscriptions(); - removeTopics(); + removeSubscriptions(management, wait); + removeTopics(management, wait); } catch (RuntimeException e) { logger.error("Error while removing topics and subscriptions", e); } @@ -65,7 +65,7 @@ public void afterSuite() { auditEvents.stop(); } - private void removeSubscriptions() { + protected void removeSubscriptions(HermesEndpoints management, Waiter wait) { management.query().querySubscriptions("{\"query\": {}}").forEach(sub -> { Response response = management.subscription().remove(sub.getQualifiedTopicName(), sub.getName()); if (response.getStatus() == OK.getStatusCode()) { @@ -77,7 +77,7 @@ private void removeSubscriptions() { }); } - private void removeTopics() { + protected void removeTopics(HermesEndpoints management, Waiter wait) { management.query().queryTopics("{\"query\": {}}").forEach(topic -> { Response response = management.topic().remove(topic.getQualifiedName()); if (response.getStatus() == OK.getStatusCode()) { diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/MetricsTest.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/MetricsTest.java index 6ae36a291b..67171e38fa 100644 --- a/integration/src/integration/java/pl/allegro/tech/hermes/integration/MetricsTest.java +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/MetricsTest.java @@ -13,7 +13,8 @@ import pl.allegro.tech.hermes.api.TopicName; import pl.allegro.tech.hermes.frontend.config.GraphiteProperties; import pl.allegro.tech.hermes.integration.env.SharedServices; -import pl.allegro.tech.hermes.integration.helper.GraphiteEndpoint; +import pl.allegro.tech.hermes.integration.helper.PrometheusEndpoint; +import pl.allegro.tech.hermes.integration.helper.PrometheusEndpoint.PrometheusTopicResponse; import pl.allegro.tech.hermes.integration.helper.graphite.GraphiteMockServer; import pl.allegro.tech.hermes.integration.shame.Unreliable; import pl.allegro.tech.hermes.test.helper.endpoint.RemoteServiceEndpoint; @@ -27,13 +28,13 @@ import static java.lang.Integer.MAX_VALUE; import static org.assertj.core.api.Assertions.assertThat; import static pl.allegro.tech.hermes.api.BatchSubscriptionPolicy.Builder.batchSubscriptionPolicy; -import static pl.allegro.tech.hermes.integration.helper.GraphiteEndpoint.subscriptionMetricsStub; +import static pl.allegro.tech.hermes.integration.helper.PrometheusEndpoint.PrometheusSubscriptionResponseBuilder.builder; import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscription; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.randomTopic; public class MetricsTest extends IntegrationTest { - private GraphiteEndpoint graphiteEndpoint; + private PrometheusEndpoint prometheusEndpoint; private RemoteServiceEndpoint remoteService; @@ -43,7 +44,7 @@ public class MetricsTest extends IntegrationTest { @BeforeMethod public void initializeAlways() { - this.graphiteEndpoint = new GraphiteEndpoint(SharedServices.services().graphiteHttpMock()); + this.prometheusEndpoint = new PrometheusEndpoint(SharedServices.services().prometheusHttpMock()); this.remoteService = new RemoteServiceEndpoint(SharedServices.services().serviceMock()); this.graphiteServer = SharedServices.services().graphiteMock(); this.graphiteProperties = new GraphiteProperties(); @@ -55,7 +56,7 @@ public void shouldIncreaseTopicMetricsAfterMessageHasBeenPublished() { // given Topic topic = operations.buildTopic(randomTopic("group", "topic_metrics").build()); operations.createSubscription(topic, "subscription", remoteService.getUrl()); - graphiteEndpoint.returnMetricForTopic(topic.getName().getGroupName(), topic.getName().getName(), 10, 15); + prometheusEndpoint.returnTopicMetrics(topic, new PrometheusTopicResponse(10, 15, 0)); remoteService.expectMessages(TestMessage.simple().body()); assertThat(publisher.publish(topic.getQualifiedName(), TestMessage.simple().body()).getStatus()) @@ -67,8 +68,8 @@ public void shouldIncreaseTopicMetricsAfterMessageHasBeenPublished() { TopicMetrics metrics = management.topic().getMetrics(topic.getQualifiedName()); // then - assertThat(metrics.getRate()).isEqualTo(MetricDecimalValue.of("10")); - assertThat(metrics.getDeliveryRate()).isEqualTo(MetricDecimalValue.of("15")); + assertThat(metrics.getRate()).isEqualTo(MetricDecimalValue.of("10.0")); + assertThat(metrics.getDeliveryRate()).isEqualTo(MetricDecimalValue.of("15.0")); assertThat(metrics.getPublished()).isEqualTo(1); assertThat(metrics.getVolume()).isGreaterThan(1); }); @@ -80,8 +81,7 @@ public void shouldIncreaseSubscriptionDeliveredMetricsAfterMessageDelivered() { // given Topic topic = operations.buildTopic(randomTopic("pl.group", "topic").build()); operations.createSubscription(topic, "subscription", remoteService.getUrl()); - graphiteEndpoint.returnMetric( - subscriptionMetricsStub("pl_group." + topic.getName().getName() + ".subscription").withRate(15).build()); + prometheusEndpoint.returnSubscriptionMetrics(topic, "subscription", builder().withRate(15).build()); remoteService.expectMessages(TestMessage.simple().body()); assertThat(publisher.publish(topic.getQualifiedName(), TestMessage.simple().body()).getStatus()) @@ -93,7 +93,7 @@ public void shouldIncreaseSubscriptionDeliveredMetricsAfterMessageDelivered() { SubscriptionMetrics metrics = management.subscription().getMetrics(topic.getQualifiedName(), "subscription"); // then - assertThat(metrics.getRate()).isEqualTo(MetricDecimalValue.of("15")); + assertThat(metrics.getRate()).isEqualTo(MetricDecimalValue.of("15.0")); assertThat(metrics.getDelivered()).isEqualTo(1); assertThat(metrics.getDiscarded()).isEqualTo(0); assertThat(metrics.getVolume()).isGreaterThan(1); @@ -122,18 +122,16 @@ public void shouldNotCreateNewSubscriptionWhenAskedForNonExistingMetrics() { public void shouldReadSubscriptionDeliveryRate() { // given Topic topic = operations.buildTopic("pl.allegro.tech.hermes", "topic"); - operations.createSubscription(topic, "pl.allegro.tech.hermes.subscription", remoteService.getUrl()); - graphiteEndpoint.returnMetric( - subscriptionMetricsStub("pl_allegro_tech_hermes.topic.pl_allegro_tech_hermes_subscription").withRate(15).build() - ); + String subscriptionName = "pl.allegro.tech.hermes.subscription"; + operations.createSubscription(topic, subscriptionName, remoteService.getUrl()); + prometheusEndpoint.returnSubscriptionMetrics(topic, subscriptionName, builder().withRate(15).build()); wait.until(() -> { // when - SubscriptionMetrics metrics = management.subscription().getMetrics("pl.allegro.tech.hermes.topic", - "pl.allegro.tech.hermes.subscription"); + SubscriptionMetrics metrics = management.subscription().getMetrics("pl.allegro.tech.hermes.topic", subscriptionName); // then - assertThat(metrics.getRate()).isEqualTo(MetricDecimalValue.of("15")); + assertThat(metrics.getRate()).isEqualTo(MetricDecimalValue.of("15.0")); }); } diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/EnvironmentAware.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/EnvironmentAware.java index e7b77f3294..2a205a3bde 100644 --- a/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/EnvironmentAware.java +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/EnvironmentAware.java @@ -30,6 +30,8 @@ public interface EnvironmentAware { int GRAPHITE_HTTP_SERVER_PORT = 18089; + int PROMETHEUS_HTTP_SERVER_PORT = 18090; + int GRAPHITE_SERVER_PORT = 18023; int OAUTH_SERVER_PORT = 19999; diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/HermesIntegrationEnvironment.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/HermesIntegrationEnvironment.java index 2c9eea9efd..08b31395a0 100644 --- a/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/HermesIntegrationEnvironment.java +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/HermesIntegrationEnvironment.java @@ -67,6 +67,7 @@ public class HermesIntegrationEnvironment implements EnvironmentAware { STARTERS.put(GraphiteMockStarter.class, new GraphiteMockStarter(GRAPHITE_SERVER_PORT)); STARTERS.put(WireMockStarter.class, new WireMockStarter(HTTP_ENDPOINT_PORT)); STARTERS.put(GraphiteHttpMockStarter.class, new GraphiteHttpMockStarter()); + STARTERS.put(PrometheusHttpMockStarter.class, new PrometheusHttpMockStarter()); STARTERS.put(OAuthServerMockStarter.class, new OAuthServerMockStarter()); STARTERS.put(AuditEventMockStarter.class, new AuditEventMockStarter()); STARTERS.put(JmsStarter.class, new JmsStarter()); diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/JmsStarter.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/JmsStarter.java index 07d16e9073..11b7581436 100644 --- a/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/JmsStarter.java +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/JmsStarter.java @@ -23,5 +23,4 @@ public void stop() throws Exception { public EmbeddedJMS instance() { return jmsServer; } - } diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/PrometheusHttpMockStarter.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/PrometheusHttpMockStarter.java new file mode 100644 index 0000000000..f38e595616 --- /dev/null +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/PrometheusHttpMockStarter.java @@ -0,0 +1,10 @@ +package pl.allegro.tech.hermes.integration.env; + +import pl.allegro.tech.hermes.test.helper.environment.WireMockStarter; + +public class PrometheusHttpMockStarter extends WireMockStarter implements EnvironmentAware { + + public PrometheusHttpMockStarter() { + super(PROMETHEUS_HTTP_SERVER_PORT); + } +} diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/SharedServices.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/SharedServices.java index 43fb85a2d8..7d054fb4c9 100644 --- a/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/SharedServices.java +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/env/SharedServices.java @@ -52,8 +52,8 @@ public WireMockServer graphiteHttpMock() { return ((WireMockStarter) starters.get(GraphiteHttpMockStarter.class)).instance(); } - public WireMockServer oauthMock() { - return ((WireMockStarter) starters.get(OAuthServerMockStarter.class)).instance(); + public WireMockServer prometheusHttpMock() { + return ((WireMockStarter) starters.get(PrometheusHttpMockStarter.class)).instance(); } public WireMockServer auditEventMock() { diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/helper/GraphiteEndpoint.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/helper/GraphiteEndpoint.java index b12867906f..f3ea7c0461 100644 --- a/integration/src/integration/java/pl/allegro/tech/hermes/integration/helper/GraphiteEndpoint.java +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/helper/GraphiteEndpoint.java @@ -4,7 +4,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.github.tomakehurst.wiremock.WireMockServer; import com.github.tomakehurst.wiremock.client.WireMock; -import pl.allegro.tech.hermes.integration.env.EnvironmentAware; import java.util.ArrayList; import java.util.List; @@ -16,7 +15,7 @@ import static java.util.Arrays.asList; import static java.util.Collections.singletonList; -public class GraphiteEndpoint implements EnvironmentAware { +public class GraphiteEndpoint { private static final String TIMESTAMP = "1396860420"; @@ -50,15 +49,6 @@ public void returnMetricForTopic(String group, String topic, int rate, int deliv .withBody(response))); } - public void returnServerErrorForAllTopics() { - graphiteListener.register(get(urlMatching(TOPIC_URL_PATTERN)) - .willReturn(aResponse() - .withStatus(500) - .withHeader("Content-Type", "application/json") - ) - ); - } - public void returnMetric(SubscriptionMetricsStubDefinition metricsStubDefinition) { graphiteListener.register(get(urlMatching(metricsStubDefinition.toUrlPattern())) .willReturn(aResponse() @@ -76,45 +66,23 @@ public void returnMetricWithDelay(SubscriptionMetricsStubDefinition metricsStubD .withBody(metricsStubDefinition.toBody()))); } - private static class GraphiteStubResponse { - private final String target; - private final List> datapoints; - - private GraphiteStubResponse(String target, List> datapoints) { - this.target = target; - this.datapoints = datapoints; - } - - public String getTarget() { - return target; - } - - public List> getDatapoints() { - return datapoints; - } + private record GraphiteStubResponse(String target, List> datapoints) { } - private static class SubscriptionMetricsStubDefinition { - private final String subscription; - private final List responseBody; - - private SubscriptionMetricsStubDefinition(String subscription, List responseBody) { - this.subscription = subscription; - this.responseBody = responseBody; - } + private record SubscriptionMetricsStubDefinition(String subscription, List responseBody) { private String toUrlPattern() { - return "/.*sumSeries%28stats.tech.hermes\\.consumer\\.%2A\\.meter\\." + subscription + "\\.m1_rate%29.*"; - } + return "/.*sumSeries%28stats.tech.hermes\\.consumer\\.%2A\\.meter\\." + subscription + "\\.m1_rate%29.*"; + } - private String toBody() { - try { - return new ObjectMapper().writeValueAsString(responseBody); - } catch (JsonProcessingException e) { - throw new RuntimeException(e); + private String toBody() { + try { + return new ObjectMapper().writeValueAsString(responseBody); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } } } - } public static class SubscriptionMetricsStubDefinitionBuilder { private final String subscription; @@ -130,12 +98,6 @@ public SubscriptionMetricsStubDefinitionBuilder withRate(int rate) { return this; } - public SubscriptionMetricsStubDefinitionBuilder withThroughput(int rate) { - String target = "sumSeries(stats.tech.hermes.consumer.*.throughput." + subscription + ".m1_rate)"; - response.add(new GraphiteStubResponse(target, dataPointOf(rate))); - return this; - } - public SubscriptionMetricsStubDefinitionBuilder withStatusRate(int httpStatus, int rate) { String statusFamily = httpStatusFamily(httpStatus); String target = "sumSeries(stats.tech.hermes.consumer.*.status." + subscription + "." + statusFamily + ".m1_rate)"; diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/helper/PrometheusEndpoint.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/helper/PrometheusEndpoint.java new file mode 100644 index 0000000000..dd6b645f76 --- /dev/null +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/helper/PrometheusEndpoint.java @@ -0,0 +1,209 @@ +package pl.allegro.tech.hermes.integration.helper; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.github.tomakehurst.wiremock.WireMockServer; +import com.github.tomakehurst.wiremock.client.WireMock; +import pl.allegro.tech.hermes.api.Topic; +import pl.allegro.tech.hermes.integration.env.EnvironmentAware; + +import java.util.ArrayList; +import java.util.List; + +import static com.github.tomakehurst.wiremock.client.WireMock.aResponse; +import static com.github.tomakehurst.wiremock.client.WireMock.get; +import static com.github.tomakehurst.wiremock.client.WireMock.urlMatching; + +public class PrometheusEndpoint { + + private static final String TIMESTAMP = "1396860420"; + + private static final String TOPIC_REQUESTS_TOTAL = "hermes_frontend_topic_requests_total"; + private static final String TOPIC_DELIVERED_TOTAL = "hermes_consumers_subscription_delivered_total"; + private static final String TOPIC_THROUGHPUT_TOTAL = "hermes_frontend_topic_throughput_bytes_total"; + + private static final String SUBSCRIPTION_DELIVERED = "hermes_consumers_subscription_delivered_total"; + private static final String SUBSCRIPTION_THROUGHPUT = "hermes_consumers_subscription_throughput_bytes_total"; + private static final String SUBSCRIPTION_STATUS_CODES = "hermes_consumers_subscription_http_status_codes_total"; + + private static final String TOPIC_QUERY_PATTERN = ".*hermes_frontend_topic_requests_total" + + ".*hermes_consumers_subscription_delivered_total.*" + + ".*hermes_frontend_topic_throughput_bytes_total.*GROUP.*TOPIC.*"; + + private static final String SUBSCRIPTION_QUERY_PATTERN = ".*hermes_consumers_subscription_delivered_total" + + ".*hermes_consumers_subscription_timeouts_total" + + ".*hermes_consumers_subscription_throughput_bytes_total" + + ".*hermes_consumers_subscription_other_errors_total" + + ".*hermes_consumers_subscription_batches_total" + + ".*hermes_consumers_subscription_http_status_codes_total.*GROUP.*TOPIC.*SUBSCRIPTION.*"; + + private final ObjectMapper objectMapper; + private final WireMock prometheusListener; + + public PrometheusEndpoint(WireMockServer prometheus) { + this.prometheusListener = new WireMock("localhost", prometheus.port()); + this.objectMapper = new ObjectMapper(); + } + + public void returnTopicMetrics(Topic topic, PrometheusTopicResponse topicStub) { + String response = generateTopicsMetricsResponse(topicStub.topicRate, topicStub.deliveredRate, topicStub.throughput); + String query = TOPIC_QUERY_PATTERN + .replaceAll("GROUP", topic.getName().getGroupName()) + .replaceAll("TOPIC", topic.getName().getName()); + prometheusListener.register(get(urlMatching(query)) + .willReturn(aResponse() + .withStatus(200) + .withHeader("Content-Type", "application/json") + .withBody(response))); + } + + public void returnSubscriptionMetrics(Topic topic, String subscription, PrometheusSubscriptionResponse stub) { + String response = generateSubscriptionResponse(stub); + String query = SUBSCRIPTION_QUERY_PATTERN + .replaceAll("GROUP", topic.getName().getGroupName()) + .replaceAll("TOPIC", topic.getName().getName()) + .replaceAll("SUBSCRIPTION", subscription); + prometheusListener.register(get(urlMatching(query)) + .willReturn(aResponse() + .withStatus(200) + .withHeader("Content-Type", "application/json") + .withBody(response))); + } + + public void returnSubscriptionMetricsWithDelay(Topic topic, String subscription, PrometheusSubscriptionResponse stub, + int prometheusDelay) { + String response = generateSubscriptionResponse(stub); + String query = SUBSCRIPTION_QUERY_PATTERN + .replaceAll("GROUP", topic.getName().getGroupName()) + .replaceAll("TOPIC", topic.getName().getName()) + .replaceAll("SUBSCRIPTION", subscription); + prometheusListener.register(get(urlMatching(query)) + .willReturn(aResponse() + .withStatus(200) + .withFixedDelay(prometheusDelay) + .withHeader("Content-Type", "application/json") + .withBody(response))); + } + + public void returnServerErrorForAllTopics() { + String query = TOPIC_QUERY_PATTERN + .replaceAll("GROUP", "") + .replaceAll("TOPIC", ""); + prometheusListener.register(get(urlMatching(query)) + .willReturn(aResponse() + .withStatus(500) + .withHeader("Content-Type", "application/json"))); + } + + private String generateTopicsMetricsResponse(int rate, int deliveryRate, int throughput) { + return writeToString( + new PrometheusResponse( + "success", + new PrometheusResponse.Data( + "vector", + List.of( + new PrometheusResponse.Result( + new PrometheusResponse.MetricName(TOPIC_REQUESTS_TOTAL, null), + List.of(TIMESTAMP, String.valueOf(rate))), + new PrometheusResponse.Result( + new PrometheusResponse.MetricName(TOPIC_DELIVERED_TOTAL, null), + List.of(TIMESTAMP, String.valueOf(deliveryRate))), + new PrometheusResponse.Result( + new PrometheusResponse.MetricName(TOPIC_THROUGHPUT_TOTAL, null), + List.of(TIMESTAMP, String.valueOf(throughput)))) + ))); + } + + private String generateSubscriptionResponse(PrometheusSubscriptionResponse stub) { + List results = new ArrayList<>(); + results.add( + new PrometheusResponse.Result( + new PrometheusResponse.MetricName(SUBSCRIPTION_DELIVERED, null), + List.of(TIMESTAMP, String.valueOf(stub.rate))) + ); + results.add( + new PrometheusResponse.Result( + new PrometheusResponse.MetricName(SUBSCRIPTION_THROUGHPUT, null), + List.of(TIMESTAMP, String.valueOf(stub.throughput)) + ) + ); + stub.statusCodes().forEach(s -> results.add( + new PrometheusResponse.Result( + new PrometheusResponse.MetricName(SUBSCRIPTION_STATUS_CODES, s.code()), + List.of(TIMESTAMP, String.valueOf(s.rate))))); + PrometheusResponse response = new PrometheusResponse( + "success", new PrometheusResponse.Data("vector", results)); + return writeToString(response); + } + + String writeToString(Object o) { + try { + return objectMapper.writeValueAsString(o); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } + + public record PrometheusTopicResponse(int topicRate, int deliveredRate, int throughput) { + } + + public record PrometheusSubscriptionResponse(int rate, int throughput, List statusCodes) { + } + + public record SubscriptionStatusCode(String code, int rate) { + } + + public static class PrometheusSubscriptionResponseBuilder { + private int rate = 0; + private int throughput = 0; + private final List statusCodes = new ArrayList<>(); + + private PrometheusSubscriptionResponseBuilder() { + } + + public static PrometheusSubscriptionResponseBuilder builder() { + return new PrometheusSubscriptionResponseBuilder(); + } + + public PrometheusSubscriptionResponseBuilder withRate(int rate) { + this.rate = rate; + return this; + } + + public PrometheusSubscriptionResponseBuilder withThroughput(int throughput) { + this.throughput = throughput; + return this; + } + + public PrometheusSubscriptionResponseBuilder withRatedStatusCode(String statusCode, int rate) { + this.statusCodes.add(new SubscriptionStatusCode(statusCode, rate)); + return this; + } + + public PrometheusSubscriptionResponse build() { + return new PrometheusSubscriptionResponse(rate, throughput, statusCodes); + } + } + + record PrometheusResponse(@JsonProperty("status") String status, + @JsonProperty("data") Data data) { + + record Data(@JsonProperty("resultType") String resultType, + @JsonProperty("result") List results) { + } + + record Result( + @JsonProperty("metric") MetricName metricName, + @JsonProperty("value") List values) { + } + + @JsonInclude(JsonInclude.Include.NON_NULL) + record MetricName( + @JsonProperty(value = "__name__") String name, + @JsonProperty(value = "status_code") String statusCode + ) { + } + } +} diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/management/ListUnhealthySubscriptionsForOwnerBasedOnGraphiteTest.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/management/ListUnhealthySubscriptionsForOwnerBasedOnGraphiteTest.java new file mode 100644 index 0000000000..c8a8c2c799 --- /dev/null +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/management/ListUnhealthySubscriptionsForOwnerBasedOnGraphiteTest.java @@ -0,0 +1,387 @@ +package pl.allegro.tech.hermes.integration.management; + +import com.google.common.collect.ImmutableSet; +import jakarta.ws.rs.InternalServerErrorException; +import jakarta.ws.rs.client.Client; +import jakarta.ws.rs.client.ClientBuilder; +import jakarta.ws.rs.core.GenericType; +import org.javers.common.collections.Lists; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import pl.allegro.tech.hermes.api.MonitoringDetails; +import pl.allegro.tech.hermes.api.MonitoringDetails.Severity; +import pl.allegro.tech.hermes.api.OwnerId; +import pl.allegro.tech.hermes.api.Subscription; +import pl.allegro.tech.hermes.api.Topic; +import pl.allegro.tech.hermes.api.UnhealthySubscription; +import pl.allegro.tech.hermes.integration.IntegrationTest; +import pl.allegro.tech.hermes.integration.env.SharedServices; +import pl.allegro.tech.hermes.integration.helper.GraphiteEndpoint; +import pl.allegro.tech.hermes.integration.helper.Waiter; +import pl.allegro.tech.hermes.integration.setup.HermesManagementInstance; +import pl.allegro.tech.hermes.test.helper.endpoint.HermesAPIOperations; +import pl.allegro.tech.hermes.test.helper.endpoint.HermesEndpoints; +import pl.allegro.tech.hermes.test.helper.util.Ports; + +import java.util.List; + +import static jakarta.ws.rs.core.MediaType.TEXT_PLAIN; +import static org.assertj.core.api.AssertionsForClassTypes.catchThrowable; +import static pl.allegro.tech.hermes.api.SubscriptionHealthProblem.malfunctioning; +import static pl.allegro.tech.hermes.integration.env.SharedServices.services; +import static pl.allegro.tech.hermes.integration.helper.GraphiteEndpoint.subscriptionMetricsStub; +import static pl.allegro.tech.hermes.integration.test.HermesAssertions.assertThat; +import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscription; +import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.randomTopic; +import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.topic; + +public class ListUnhealthySubscriptionsForOwnerBasedOnGraphiteTest extends IntegrationTest { + + private final Client httpClient = ClientBuilder.newClient(); + private GraphiteEndpoint graphiteEndpoint; + private HermesAPIOperations apiOperations; + private String managementUrl; + private HermesEndpoints customManagement; + private Waiter hermesWaiter; + + @BeforeClass + public void initializeOnce() { + int managementPort = Ports.nextAvailable(); + managementUrl = "http://localhost:" + managementPort + "/"; + HermesManagementInstance hermesManagement = HermesManagementInstance.starter() + .port(managementPort) + .withGraphiteExternalStorageEnabled() + .addKafkaCluster(DC1, kafkaClusterOne.getBootstrapServersForExternalClients()) + .addZookeeperCluster(DC1, hermesZookeeperOne.getConnectionString()) + .schemaRegistry(schemaRegistry.getUrl()) + .replicationFactor(kafkaClusterOne.getAllBrokers().size()) + .uncleanLeaderElectionEnabled(false) + .start(); + apiOperations = hermesManagement.operations(); + customManagement = new HermesEndpoints(managementUrl, CONSUMER_ENDPOINT_URL); + hermesWaiter = new Waiter(customManagement, services().zookeeper(), brokerOperations, PRIMARY_KAFKA_CLUSTER_NAME, KAFKA_NAMESPACE); + } + + @BeforeMethod + public void initializeAlways() { + graphiteEndpoint = new GraphiteEndpoint(SharedServices.services().graphiteHttpMock()); + } + + @AfterMethod + public void cleanup() { + removeSubscriptions(customManagement, hermesWaiter); + removeTopics(customManagement, hermesWaiter); + } + + @Test + public void shouldNotListHealthySubscriptions() { + // given + Topic topic = apiOperations.buildTopic(randomTopic("group", "topic").build()); + createSubscriptionForOwner(topic, "s1", "Team A"); + createSubscriptionForOwner(topic, "s2", "Team B"); + + // then + assertThat(listUnhealthySubscriptionsForOwner("Team A")).isEmpty(); + assertThat(listUnhealthySubscriptionsForOwnerAsPlainText("Team A")).isEmpty(); + } + + @Test + public void shouldReturnOnlyUnhealthySubscriptionOfSingleOwner() { + // given + Topic topic = apiOperations.buildTopic(topic("groupSecond", "topicSecond").build()); + createSubscriptionForOwner(topic, "ownedSubscription1", "Team A"); + final Subscription subscription = createSubscriptionForOwner(topic, "ownedSubscription2", "Team A"); + createSubscriptionForOwner(topic, "ownedSubscription3", "Team B"); + + graphiteEndpoint.returnMetricForTopic(topic.getName().getGroupName(), topic.getName().getName(), 100, 50); + graphiteEndpoint.returnMetric( + subscriptionMetricsStub(subName(topic, "ownedSubscription1")).withRate(100).withStatusRate(500, 0).build() + ); + graphiteEndpoint.returnMetric( + subscriptionMetricsStub(subName(topic, "ownedSubscription2")).withRate(50).withStatusRate(500, 11).build() + ); + graphiteEndpoint.returnMetric( + subscriptionMetricsStub(subName(topic, "ownedSubscription3")).withRate(100).withStatusRate(500, 0).build() + ); + + // then + assertThat(listUnhealthySubscriptionsForOwner("Team A")).containsOnly( + new UnhealthySubscription("ownedSubscription2", topic.getQualifiedName(), Severity.IMPORTANT, + ImmutableSet.of(malfunctioning(11, subscription.getQualifiedName().toString()))) + ); + + assertThat(listUnhealthySubscriptionsForOwnerAsPlainText("Team A")).isEqualTo( + "ownedSubscription2 - Consuming service returns a lot of 5xx codes for subscription " + + subscription.getQualifiedName().toString() + ", currently 11 5xx/s" + ); + } + + @Test + public void shouldReturnOnlySpecifiedUnhealthySubscription() { + // given + Topic topic1 = apiOperations.buildTopic(topic("group", "topic1").build()); + Topic topic2 = apiOperations.buildTopic(topic("group", "topic2").build()); + Topic topic3 = apiOperations.buildTopic(topic("group", "topic3").build()); + + createSubscriptionForOwner(topic1, "ownedSubscription1", "Team A"); + final Subscription subscription2 = createSubscriptionForOwner(topic2, "ownedSubscription2", "Team A"); + final Subscription subscription3 = createSubscriptionForOwner(topic3, "ownedSubscription3", "Team A"); + createSubscriptionForOwner(topic1, "ownedSubscription4", "Team B"); + final Subscription subscription5 = createSubscriptionForOwner(topic2, "ownedSubscription5", "Team B"); + final Subscription subscription6 = createSubscriptionForOwner(topic3, "ownedSubscription6", "Team B"); + + graphiteEndpoint.returnMetricForTopic(topic1.getName().getGroupName(), topic1.getName().getName(), 100, 50); + graphiteEndpoint.returnMetricForTopic(topic2.getName().getGroupName(), topic2.getName().getName(), 100, 50); + graphiteEndpoint.returnMetricForTopic(topic3.getName().getGroupName(), topic3.getName().getName(), 100, 50); + + graphiteEndpoint.returnMetric( + subscriptionMetricsStub(subName(topic1, "ownedSubscription1")).withRate(100).withStatusRate(500, 0).build() + ); + graphiteEndpoint.returnMetric( + subscriptionMetricsStub(subName(topic2, "ownedSubscription2")).withRate(50).withStatusRate(500, 11).build() + ); + graphiteEndpoint.returnMetric( + subscriptionMetricsStub(subName(topic3, "ownedSubscription3")).withRate(50).withStatusRate(500, 11).build() + ); + + graphiteEndpoint.returnMetric( + subscriptionMetricsStub(subName(topic1, "ownedSubscription4")).withRate(100).withStatusRate(500, 0).build() + ); + graphiteEndpoint.returnMetric( + subscriptionMetricsStub(subName(topic2, "ownedSubscription5")).withRate(50).withStatusRate(500, 11).build() + ); + graphiteEndpoint.returnMetric( + subscriptionMetricsStub(subName(topic3, "ownedSubscription6")).withRate(50).withStatusRate(500, 11).build() + ); + + // then + assertThat(listUnhealthySubscriptionsForOwner("Team A", Lists.asList(), Lists.asList("group.topic2"))).containsOnly( + new UnhealthySubscription("ownedSubscription2", "group.topic2", Severity.IMPORTANT, + ImmutableSet.of(malfunctioning(11, subscription2.getQualifiedName().toString()))) + ); + assertThat(listUnhealthySubscriptionsForOwnerAsPlainText("Team A", Lists.asList(), Lists.asList("group.topic2"))).isEqualTo( + "ownedSubscription2 - Consuming service returns a lot of 5xx codes for subscription " + + subscription2.getQualifiedName().toString() + ", currently 11 5xx/s" + ); + assertThat(listAllUnhealthySubscriptions(Lists.asList(), Lists.asList("group.topic2"))).containsOnly( + new UnhealthySubscription("ownedSubscription2", "group.topic2", Severity.IMPORTANT, + ImmutableSet.of(malfunctioning(11, subscription2.getQualifiedName().toString()))), + new UnhealthySubscription("ownedSubscription5", "group.topic2", Severity.IMPORTANT, + ImmutableSet.of(malfunctioning(11, subscription5.getQualifiedName().toString()))) + ); + assertThat(listAllUnhealthySubscriptionsAsPlainText(Lists.asList(), Lists.asList("group.topic2"))).isEqualTo( + "ownedSubscription2 - Consuming service returns a lot of 5xx codes for subscription " + + subscription2.getQualifiedName().toString() + ", currently 11 5xx/s\r\n" + + "ownedSubscription5 - Consuming service returns a lot of 5xx codes for subscription " + + subscription5.getQualifiedName().toString() + ", currently 11 5xx/s" + ); + assertThat(listAllUnhealthySubscriptionsAsPlainText(Lists.asList("ownedSubscription2"), Lists.asList("group.topic2"))).isEqualTo( + "ownedSubscription2 - Consuming service returns a lot of 5xx codes for subscription " + + subscription2.getQualifiedName().toString() + ", currently 11 5xx/s" + ); + assertThat(listAllUnhealthySubscriptions()).containsOnly( + new UnhealthySubscription("ownedSubscription2", "group.topic2", Severity.IMPORTANT, + ImmutableSet.of(malfunctioning(11, subscription2.getQualifiedName().toString()))), + new UnhealthySubscription("ownedSubscription3", "group.topic3", Severity.IMPORTANT, + ImmutableSet.of(malfunctioning(11, subscription3.getQualifiedName().toString()))), + new UnhealthySubscription("ownedSubscription5", "group.topic2", Severity.IMPORTANT, + ImmutableSet.of(malfunctioning(11, subscription5.getQualifiedName().toString()))), + new UnhealthySubscription("ownedSubscription6", "group.topic3", Severity.IMPORTANT, + ImmutableSet.of(malfunctioning(11, subscription6.getQualifiedName().toString()))) + ); + assertThat(listAllUnhealthySubscriptions(Lists.asList("ownedSubscription2", "ownedSubscription3"), + Lists.asList("group.topic2", "group.topic3"))).containsOnly( + new UnhealthySubscription("ownedSubscription2", "group.topic2", Severity.IMPORTANT, + ImmutableSet.of(malfunctioning(11, subscription2.getQualifiedName().toString()))), + new UnhealthySubscription("ownedSubscription3", "group.topic3", Severity.IMPORTANT, + ImmutableSet.of(malfunctioning(11, subscription3.getQualifiedName().toString()))) + ); + } + + @Test + public void shouldReportAllUnhealthySubscriptionsForEmptyOwnerSource() { + // given + Topic topic = apiOperations.buildTopic(randomTopic("groupFirst", "topicFirst").build()); + createSubscriptionForOwner(topic, "ownedSubscription1", "Team A"); + final Subscription subscription = createSubscriptionForOwner(topic, "ownedSubscription2", "Team A"); + + graphiteEndpoint.returnMetricForTopic(topic.getName().getGroupName(), topic.getName().getName(), 100, 50); + graphiteEndpoint.returnMetric( + subscriptionMetricsStub(subName(topic, "ownedSubscription1")).withRate(100).withStatusRate(500, 0).build() + ); + graphiteEndpoint.returnMetric( + subscriptionMetricsStub(subName(topic, "ownedSubscription2")).withRate(50).withStatusRate(500, 11).build() + ); + + // then + assertThat(listAllUnhealthySubscriptions()).containsOnly( + new UnhealthySubscription("ownedSubscription2", topic.getQualifiedName(), Severity.IMPORTANT, + ImmutableSet.of(malfunctioning(11, subscription.getQualifiedName().toString()))) + ); + assertThat(listAllUnhealthySubscriptionsAsPlainText()).isEqualTo( + "ownedSubscription2 - Consuming service returns a lot of 5xx codes for subscription " + + subscription.getQualifiedName().toString() + ", currently 11 5xx/s" + ); + } + + @Test + public void shouldReportUnhealthySubscriptionsDisrespectingSeverity() { + // given + Topic topic = apiOperations.buildTopic(randomTopic("group", "topic").build()); + final Subscription subscription1 = createSubscriptionForOwner(topic, "ownedSubscription1", "Team A", Severity.CRITICAL); + final Subscription subscription2 = createSubscriptionForOwner(topic, "ownedSubscription2", "Team A", Severity.IMPORTANT); + final Subscription subscription3 = createSubscriptionForOwner(topic, "ownedSubscription3", "Team A", Severity.NON_IMPORTANT); + + graphiteEndpoint.returnMetricForTopic(topic.getName().getGroupName(), topic.getName().getName(), 100, 50); + graphiteEndpoint.returnMetric( + subscriptionMetricsStub(subName(topic, "ownedSubscription1")).withRate(50).withStatusRate(500, 11).build() + ); + graphiteEndpoint.returnMetric( + subscriptionMetricsStub(subName(topic, "ownedSubscription2")).withRate(50).withStatusRate(500, 11).build() + ); + graphiteEndpoint.returnMetric( + subscriptionMetricsStub(subName(topic, "ownedSubscription3")).withRate(50).withStatusRate(500, 11).build() + ); + + // then + assertThat(listUnhealthySubscriptionsDisrespectingSeverity("Team A")).contains( + new UnhealthySubscription("ownedSubscription1", topic.getQualifiedName(), Severity.CRITICAL, + ImmutableSet.of(malfunctioning(11, subscription1.getQualifiedName().toString()))), + new UnhealthySubscription("ownedSubscription2", topic.getQualifiedName(), Severity.IMPORTANT, + ImmutableSet.of(malfunctioning(11, subscription2.getQualifiedName().toString()))), + new UnhealthySubscription("ownedSubscription3", topic.getQualifiedName(), Severity.NON_IMPORTANT, + ImmutableSet.of(malfunctioning(11, subscription3.getQualifiedName().toString()))) + ); + assertThat(listUnhealthySubscriptionsDisrespectingSeverityAsPlainText("Team A")).isEqualTo( + "ownedSubscription1 - Consuming service returns a lot of 5xx codes for subscription " + + subscription1.getQualifiedName().toString() + ", currently 11 5xx/s\r\n" + + "ownedSubscription2 - Consuming service returns a lot of 5xx codes for subscription " + + subscription2.getQualifiedName().toString() + ", currently 11 5xx/s\r\n" + + "ownedSubscription3 - Consuming service returns a lot of 5xx codes for subscription " + + subscription3.getQualifiedName().toString() + ", currently 11 5xx/s" + ); + } + + @Test + public void shouldTimeoutUnhealthySubscriptionsRequest() { + // given + Topic topic = apiOperations.buildTopic(randomTopic("group", "topic").build()); + createSubscriptionForOwner(topic, "ownedSubscription1", "Team A", Severity.CRITICAL); + int graphiteDelay = 1000; + graphiteEndpoint.returnMetricWithDelay(subscriptionMetricsStub(subName(topic, "ownedSubscription1")) + .withRate(50) + .withStatusRate(200, 11).build(), graphiteDelay); + + // when + long start = System.currentTimeMillis(); + Throwable thrown = catchThrowable(() -> listUnhealthyAsPlainText("Plaintext", "Team A", true, Lists.asList(), Lists.asList())); + long end = System.currentTimeMillis(); + + // then + assertThat(thrown).isInstanceOf(InternalServerErrorException.class); + assertThat(end - start < graphiteDelay); + + } + + @Test + public void shouldReportSuspendedSubscriptionAsHealthy() { + // given + Topic topic = apiOperations.buildTopic(randomTopic("group", "topic").build()); + Subscription s = createSubscriptionForOwner(topic, "subscription1", "Team A"); + + // when + s.setState(Subscription.State.SUSPENDED); + + // then + assertThat(listUnhealthySubscriptionsForOwner("Team A")).isEmpty(); + assertThat(listUnhealthySubscriptionsForOwnerAsPlainText("Team A")).isEmpty(); + } + + private Subscription createSubscriptionForOwner(Topic topic, String subscriptionName, String ownerId) { + return createSubscriptionForOwner(topic, subscriptionName, ownerId, Severity.IMPORTANT); + } + + private Subscription createSubscriptionForOwner(Topic topic, String subscriptionName, String ownerId, Severity severity) { + Subscription subscription = subscription(topic, subscriptionName) + .withEndpoint(HTTP_ENDPOINT_URL) + .withOwner(ownerId(ownerId)) + .withMonitoringDetails(new MonitoringDetails(severity, "")) + .build(); + + apiOperations.createSubscription(topic, subscription); + return subscription; + } + + private OwnerId ownerId(String ownerId) { + return new OwnerId("Plaintext", ownerId); + } + + private List listUnhealthySubscriptionsForOwner(String ownerId) { + return listUnhealthy("Plaintext", ownerId, true, Lists.asList(), Lists.asList()); + } + + private List listUnhealthySubscriptionsForOwner(String ownerId, List subscriptionNames, + List qualifiedTopicNames) { + return listUnhealthy("Plaintext", ownerId, true, subscriptionNames, qualifiedTopicNames); + } + + private List listUnhealthySubscriptionsDisrespectingSeverity(String ownerId) { + return listUnhealthy("Plaintext", ownerId, false, Lists.asList(), Lists.asList()); + } + + private List listAllUnhealthySubscriptions() { + return listUnhealthy(null, null, true, Lists.asList(), Lists.asList()); + } + + private List listAllUnhealthySubscriptions(List subscriptionNames, List qualifiedTopicNames) { + return listUnhealthy(null, null, true, subscriptionNames, qualifiedTopicNames); + } + + private List listUnhealthy(String ownerSourceName, String ownerId, boolean respectMonitoringSeverity, + List subscriptionNames, List qualifiedTopicNames) { + return customManagement.unhealthyEndpoint() + .listUnhealthy(ownerSourceName, ownerId, respectMonitoringSeverity, subscriptionNames, qualifiedTopicNames) + .readEntity(new GenericType<>() { + }); + } + + private String listUnhealthySubscriptionsForOwnerAsPlainText(String ownerId) { + return listUnhealthyAsPlainText("Plaintext", ownerId, true, Lists.asList(), Lists.asList()); + } + + private String listUnhealthySubscriptionsForOwnerAsPlainText(String ownerId, List subscriptionNames, + List qualifiedTopicNames) { + return listUnhealthyAsPlainText("Plaintext", ownerId, true, subscriptionNames, qualifiedTopicNames); + } + + private String listUnhealthySubscriptionsDisrespectingSeverityAsPlainText(String ownerId) { + return listUnhealthyAsPlainText("Plaintext", ownerId, false, Lists.asList(), Lists.asList()); + } + + private String listAllUnhealthySubscriptionsAsPlainText() { + return listUnhealthyAsPlainText(null, null, true, Lists.asList(), Lists.asList()); + } + + private String listAllUnhealthySubscriptionsAsPlainText(List subscriptionNames, List qualifiedTopicNames) { + return listUnhealthyAsPlainText(null, null, true, subscriptionNames, qualifiedTopicNames); + } + + private String listUnhealthyAsPlainText(String ownerSourceName, String ownerId, boolean respectMonitoringSeverity, + List subscriptionNames, List qualifiedTopicNames) { + return httpClient.target(managementUrl) + .path("unhealthy") + .queryParam("ownerSourceName", ownerSourceName) + .queryParam("ownerId", ownerId) + .queryParam("respectMonitoringSeverity", respectMonitoringSeverity) + .queryParam("subscriptionNames", (Object[]) subscriptionNames.toArray(new String[0])) + .queryParam("qualifiedTopicNames", (Object[]) qualifiedTopicNames.toArray(new String[0])) + .request(TEXT_PLAIN) + .get(String.class); + } + + private String subName(Topic topic, String rawSubscriptionName) { + return topic.getQualifiedName() + "." + rawSubscriptionName; + } +} diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/management/ListUnhealthySubscriptionsForOwnerTest.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/management/ListUnhealthySubscriptionsForOwnerTest.java index 6324f7732f..73fba11bdc 100644 --- a/integration/src/integration/java/pl/allegro/tech/hermes/integration/management/ListUnhealthySubscriptionsForOwnerTest.java +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/management/ListUnhealthySubscriptionsForOwnerTest.java @@ -15,7 +15,8 @@ import pl.allegro.tech.hermes.api.UnhealthySubscription; import pl.allegro.tech.hermes.integration.IntegrationTest; import pl.allegro.tech.hermes.integration.env.SharedServices; -import pl.allegro.tech.hermes.integration.helper.GraphiteEndpoint; +import pl.allegro.tech.hermes.integration.helper.PrometheusEndpoint; +import pl.allegro.tech.hermes.integration.helper.PrometheusEndpoint.PrometheusTopicResponse; import java.util.List; @@ -23,7 +24,7 @@ import static org.assertj.core.api.Assertions.catchThrowable; import static pl.allegro.tech.hermes.api.MonitoringDetails.Severity; import static pl.allegro.tech.hermes.api.SubscriptionHealthProblem.malfunctioning; -import static pl.allegro.tech.hermes.integration.helper.GraphiteEndpoint.subscriptionMetricsStub; +import static pl.allegro.tech.hermes.integration.helper.PrometheusEndpoint.PrometheusSubscriptionResponseBuilder.builder; import static pl.allegro.tech.hermes.integration.test.HermesAssertions.assertThat; import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscription; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.randomTopic; @@ -31,12 +32,12 @@ public class ListUnhealthySubscriptionsForOwnerTest extends IntegrationTest { - private GraphiteEndpoint graphiteEndpoint; - private Client httpClient = ClientBuilder.newClient(); + private PrometheusEndpoint prometheusEndpoint; + private final Client httpClient = ClientBuilder.newClient(); @BeforeMethod public void initializeAlways() { - graphiteEndpoint = new GraphiteEndpoint(SharedServices.services().graphiteHttpMock()); + prometheusEndpoint = new PrometheusEndpoint(SharedServices.services().prometheusHttpMock()); } @Test @@ -58,17 +59,14 @@ public void shouldReturnOnlyUnhealthySubscriptionOfSingleOwner() { createSubscriptionForOwner(topic, "ownedSubscription1", "Team A"); final Subscription subscription = createSubscriptionForOwner(topic, "ownedSubscription2", "Team A"); createSubscriptionForOwner(topic, "ownedSubscription3", "Team B"); - - graphiteEndpoint.returnMetricForTopic(topic.getName().getGroupName(), topic.getName().getName(), 100, 50); - graphiteEndpoint.returnMetric( - subscriptionMetricsStub(subName(topic, "ownedSubscription1")).withRate(100).withStatusRate(500, 0).build() - ); - graphiteEndpoint.returnMetric( - subscriptionMetricsStub(subName(topic, "ownedSubscription2")).withRate(50).withStatusRate(500, 11).build() - ); - graphiteEndpoint.returnMetric( - subscriptionMetricsStub(subName(topic, "ownedSubscription3")).withRate(100).withStatusRate(500, 0).build() - ); + PrometheusTopicResponse topicStub = new PrometheusTopicResponse(100, 50, 0); + prometheusEndpoint.returnTopicMetrics(topic, topicStub); + prometheusEndpoint.returnSubscriptionMetrics(topic, "ownedSubscription1", + builder().withRate(100).withRatedStatusCode("500", 0).build()); + prometheusEndpoint.returnSubscriptionMetrics(topic, "ownedSubscription2", + builder().withRate(50).withRatedStatusCode("500", 11).build()); + prometheusEndpoint.returnSubscriptionMetrics(topic, "ownedSubscription3", + builder().withRate(100).withRatedStatusCode("500", 0).build()); // then assertThat(listUnhealthySubscriptionsForOwner("Team A")).containsOnly( @@ -95,29 +93,22 @@ public void shouldReturnOnlySpecifiedUnhealthySubscription() { final Subscription subscription5 = createSubscriptionForOwner(topic2, "ownedSubscription5", "Team B"); final Subscription subscription6 = createSubscriptionForOwner(topic3, "ownedSubscription6", "Team B"); - graphiteEndpoint.returnMetricForTopic(topic1.getName().getGroupName(), topic1.getName().getName(), 100, 50); - graphiteEndpoint.returnMetricForTopic(topic2.getName().getGroupName(), topic2.getName().getName(), 100, 50); - graphiteEndpoint.returnMetricForTopic(topic3.getName().getGroupName(), topic3.getName().getName(), 100, 50); - - graphiteEndpoint.returnMetric( - subscriptionMetricsStub(subName(topic1, "ownedSubscription1")).withRate(100).withStatusRate(500, 0).build() - ); - graphiteEndpoint.returnMetric( - subscriptionMetricsStub(subName(topic2, "ownedSubscription2")).withRate(50).withStatusRate(500, 11).build() - ); - graphiteEndpoint.returnMetric( - subscriptionMetricsStub(subName(topic3, "ownedSubscription3")).withRate(50).withStatusRate(500, 11).build() - ); - - graphiteEndpoint.returnMetric( - subscriptionMetricsStub(subName(topic1, "ownedSubscription4")).withRate(100).withStatusRate(500, 0).build() - ); - graphiteEndpoint.returnMetric( - subscriptionMetricsStub(subName(topic2, "ownedSubscription5")).withRate(50).withStatusRate(500, 11).build() - ); - graphiteEndpoint.returnMetric( - subscriptionMetricsStub(subName(topic3, "ownedSubscription6")).withRate(50).withStatusRate(500, 11).build() - ); + prometheusEndpoint.returnTopicMetrics(topic1, new PrometheusTopicResponse(100, 50, 0)); + prometheusEndpoint.returnTopicMetrics(topic2, new PrometheusTopicResponse(100, 50, 0)); + prometheusEndpoint.returnTopicMetrics(topic3, new PrometheusTopicResponse(100, 50, 0)); + + prometheusEndpoint.returnSubscriptionMetrics(topic1, "ownedSubscription1", + builder().withRate(100).withRatedStatusCode("500", 0).build()); + prometheusEndpoint.returnSubscriptionMetrics(topic2, "ownedSubscription2", + builder().withRate(50).withRatedStatusCode("500", 11).build()); + prometheusEndpoint.returnSubscriptionMetrics(topic3, "ownedSubscription3", + builder().withRate(50).withRatedStatusCode("500", 11).build()); + prometheusEndpoint.returnSubscriptionMetrics(topic1, "ownedSubscription4", + builder().withRate(100).withRatedStatusCode("500", 0).build()); + prometheusEndpoint.returnSubscriptionMetrics(topic2, "ownedSubscription5", + builder().withRate(50).withRatedStatusCode("500", 11).build()); + prometheusEndpoint.returnSubscriptionMetrics(topic3, "ownedSubscription6", + builder().withRate(50).withRatedStatusCode("500", 11).build()); // then assertThat(listUnhealthySubscriptionsForOwner("Team A", Lists.asList(), Lists.asList("group.topic2"))).containsOnly( @@ -170,13 +161,11 @@ public void shouldReportAllUnhealthySubscriptionsForEmptyOwnerSource() { createSubscriptionForOwner(topic, "ownedSubscription1", "Team A"); final Subscription subscription = createSubscriptionForOwner(topic, "ownedSubscription2", "Team A"); - graphiteEndpoint.returnMetricForTopic(topic.getName().getGroupName(), topic.getName().getName(), 100, 50); - graphiteEndpoint.returnMetric( - subscriptionMetricsStub(subName(topic, "ownedSubscription1")).withRate(100).withStatusRate(500, 0).build() - ); - graphiteEndpoint.returnMetric( - subscriptionMetricsStub(subName(topic, "ownedSubscription2")).withRate(50).withStatusRate(500, 11).build() - ); + prometheusEndpoint.returnTopicMetrics(topic, new PrometheusTopicResponse(100, 50, 0)); + prometheusEndpoint.returnSubscriptionMetrics(topic, "ownedSubscription1", + builder().withRate(100).withRatedStatusCode("500", 0).build()); + prometheusEndpoint.returnSubscriptionMetrics(topic, "ownedSubscription2", + builder().withRate(50).withRatedStatusCode("500", 11).build()); // then assertThat(listAllUnhealthySubscriptions()).containsOnly( @@ -197,16 +186,13 @@ public void shouldReportUnhealthySubscriptionsDisrespectingSeverity() { final Subscription subscription2 = createSubscriptionForOwner(topic, "ownedSubscription2", "Team A", Severity.IMPORTANT); final Subscription subscription3 = createSubscriptionForOwner(topic, "ownedSubscription3", "Team A", Severity.NON_IMPORTANT); - graphiteEndpoint.returnMetricForTopic(topic.getName().getGroupName(), topic.getName().getName(), 100, 50); - graphiteEndpoint.returnMetric( - subscriptionMetricsStub(subName(topic, "ownedSubscription1")).withRate(50).withStatusRate(500, 11).build() - ); - graphiteEndpoint.returnMetric( - subscriptionMetricsStub(subName(topic, "ownedSubscription2")).withRate(50).withStatusRate(500, 11).build() - ); - graphiteEndpoint.returnMetric( - subscriptionMetricsStub(subName(topic, "ownedSubscription3")).withRate(50).withStatusRate(500, 11).build() - ); + prometheusEndpoint.returnTopicMetrics(topic, new PrometheusTopicResponse(100, 50, 0)); + prometheusEndpoint.returnSubscriptionMetrics(topic, "ownedSubscription1", + builder().withRate(50).withRatedStatusCode("500", 11).build()); + prometheusEndpoint.returnSubscriptionMetrics(topic, "ownedSubscription2", + builder().withRate(50).withRatedStatusCode("500", 11).build()); + prometheusEndpoint.returnSubscriptionMetrics(topic, "ownedSubscription3", + builder().withRate(50).withRatedStatusCode("500", 11).build()); // then assertThat(listUnhealthySubscriptionsDisrespectingSeverity("Team A")).contains( @@ -232,10 +218,10 @@ public void shouldTimeoutUnhealthySubscriptionsRequest() { // given Topic topic = operations.buildTopic(randomTopic("group", "topic").build()); createSubscriptionForOwner(topic, "ownedSubscription1", "Team A", Severity.CRITICAL); - int graphiteDelay = 1000; - graphiteEndpoint.returnMetricWithDelay(subscriptionMetricsStub(subName(topic, "ownedSubscription1")) - .withRate(50) - .withStatusRate(200, 11).build(), graphiteDelay); + int prometheusDelay = 1000; + + prometheusEndpoint.returnSubscriptionMetricsWithDelay(topic, "ownedSubscription1", + builder().withRate(50).withRatedStatusCode("500", 11).build(), prometheusDelay); // when long start = System.currentTimeMillis(); @@ -244,7 +230,7 @@ public void shouldTimeoutUnhealthySubscriptionsRequest() { // then assertThat(thrown).isInstanceOf(InternalServerErrorException.class); - assertThat(end - start < graphiteDelay); + assertThat(end - start < prometheusDelay); } @@ -312,7 +298,7 @@ private List listUnhealthy(String ownerSourceName, String List subscriptionNames, List qualifiedTopicNames) { return management.unhealthyEndpoint() .listUnhealthy(ownerSourceName, ownerId, respectMonitoringSeverity, subscriptionNames, qualifiedTopicNames) - .readEntity(new GenericType>() { + .readEntity(new GenericType<>() { }); } @@ -349,8 +335,4 @@ private String listUnhealthyAsPlainText(String ownerSourceName, String ownerId, .request(TEXT_PLAIN) .get(String.class); } - - private String subName(Topic topic, String rawSubscriptionName) { - return topic.getQualifiedName() + "." + rawSubscriptionName; - } } diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/management/QueryEndpointTest.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/management/QueryEndpointTest.java index 53baa3e8ff..4cfd56932d 100644 --- a/integration/src/integration/java/pl/allegro/tech/hermes/integration/management/QueryEndpointTest.java +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/management/QueryEndpointTest.java @@ -14,7 +14,7 @@ import pl.allegro.tech.hermes.api.TrackingMode; import pl.allegro.tech.hermes.integration.IntegrationTest; import pl.allegro.tech.hermes.integration.env.SharedServices; -import pl.allegro.tech.hermes.integration.helper.GraphiteEndpoint; +import pl.allegro.tech.hermes.integration.helper.PrometheusEndpoint; import pl.allegro.tech.hermes.test.helper.avro.AvroUserSchemaLoader; import pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder; import pl.allegro.tech.hermes.test.helper.endpoint.RemoteServiceEndpoint; @@ -28,7 +28,7 @@ import static pl.allegro.tech.hermes.api.ContentType.JSON; import static pl.allegro.tech.hermes.api.SubscriptionPolicy.Builder.subscriptionPolicy; import static pl.allegro.tech.hermes.api.TopicWithSchema.topicWithSchema; -import static pl.allegro.tech.hermes.integration.helper.GraphiteEndpoint.subscriptionMetricsStub; +import static pl.allegro.tech.hermes.integration.helper.PrometheusEndpoint.PrometheusSubscriptionResponseBuilder.builder; import static pl.allegro.tech.hermes.integration.test.HermesAssertions.assertThat; import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscription; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.randomTopic; @@ -40,12 +40,12 @@ public class QueryEndpointTest extends IntegrationTest { private RemoteServiceEndpoint remoteService; - private GraphiteEndpoint graphiteEndpoint; + private PrometheusEndpoint prometheusEndpoint; @BeforeClass public void initialize() { remoteService = new RemoteServiceEndpoint(SharedServices.services().serviceMock()); - graphiteEndpoint = new GraphiteEndpoint(SharedServices.services().graphiteHttpMock()); + prometheusEndpoint = new PrometheusEndpoint(SharedServices.services().prometheusHttpMock()); } @AfterClass @@ -234,16 +234,10 @@ public void shouldQuerySubscriptionsMetrics() { String queryGetSubscriptionsMetricsWithLagNegative = "{\"query\": {\"lag\": {\"lt\": 0}}}"; String queryGetSubscriptionsMetricsWithVolume = "{\"query\": {\"volume\": {\"gt\": -1}}}"; - graphiteEndpoint.returnMetric(subscriptionMetricsStub("subscriptionsMetricsTestGroup1.topic.subscription1") - .withRate(100) - .withThroughput(0) - .build() - ); - graphiteEndpoint.returnMetric(subscriptionMetricsStub("subscriptionsMetricsTestGroup2.topic.subscription2") - .withRate(40) - .withThroughput(10) - .build() - ); + prometheusEndpoint.returnSubscriptionMetrics(topic1, "subscription1", builder() + .withRate(100).withThroughput(0).build()); + prometheusEndpoint.returnSubscriptionMetrics(topic2, "subscription2", builder() + .withRate(40).withThroughput(10).build()); wait.until(() -> { // when @@ -276,11 +270,12 @@ public void shouldHandleUnavailableSubscriptionsMetrics() { String queryGetAllSubscriptionsMetrics = "{\"query\": {}}"; String queryGetSubscriptionsMetricsWithPositiveRate = "{\"query\": {\"rate\": {\"gt\": 0}}}"; - int graphiteResponseDelay = 10 * 60 * 1000; - graphiteEndpoint.returnMetricWithDelay(subscriptionMetricsStub("unavailableMetricsGroup.topic.subscription") + int prometheusResponseDelay = 10 * 60 * 1000; + prometheusEndpoint.returnSubscriptionMetricsWithDelay(topic, "subscription", + builder() .withRate(100) .build(), - graphiteResponseDelay + prometheusResponseDelay ); wait.until(() -> { diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/management/SubscriptionManagementTest.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/management/SubscriptionManagementTest.java index e0ea8667e2..16c8afcd7f 100644 --- a/integration/src/integration/java/pl/allegro/tech/hermes/integration/management/SubscriptionManagementTest.java +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/management/SubscriptionManagementTest.java @@ -27,7 +27,8 @@ import pl.allegro.tech.hermes.consumers.config.KafkaProperties; import pl.allegro.tech.hermes.integration.IntegrationTest; import pl.allegro.tech.hermes.integration.env.SharedServices; -import pl.allegro.tech.hermes.integration.helper.GraphiteEndpoint; +import pl.allegro.tech.hermes.integration.helper.PrometheusEndpoint; +import pl.allegro.tech.hermes.integration.helper.PrometheusEndpoint.PrometheusTopicResponse; import pl.allegro.tech.hermes.integration.shame.Unreliable; import pl.allegro.tech.hermes.management.TestSecurityProvider; import pl.allegro.tech.hermes.test.helper.endpoint.BrokerOperations.ConsumerGroupOffset; @@ -40,7 +41,6 @@ import java.util.stream.Stream; import static com.jayway.awaitility.Awaitility.await; -import static com.jayway.awaitility.Awaitility.waitAtMost; import static jakarta.ws.rs.client.ClientBuilder.newClient; import static jakarta.ws.rs.core.Response.Status.BAD_REQUEST; import static jakarta.ws.rs.core.Response.Status.CREATED; @@ -51,7 +51,7 @@ import static pl.allegro.tech.hermes.api.SubscriptionHealth.Status.UNHEALTHY; import static pl.allegro.tech.hermes.api.SubscriptionHealthProblem.malfunctioning; import static pl.allegro.tech.hermes.client.HermesClientBuilder.hermesClient; -import static pl.allegro.tech.hermes.integration.helper.GraphiteEndpoint.subscriptionMetricsStub; +import static pl.allegro.tech.hermes.integration.helper.PrometheusEndpoint.PrometheusSubscriptionResponseBuilder.builder; import static pl.allegro.tech.hermes.integration.test.HermesAssertions.assertThat; import static pl.allegro.tech.hermes.test.helper.builder.SubscriptionBuilder.subscription; import static pl.allegro.tech.hermes.test.helper.builder.TopicBuilder.randomTopic; @@ -64,13 +64,13 @@ public class SubscriptionManagementTest extends IntegrationTest { private RemoteServiceEndpoint remoteService; private HermesClient client; - private GraphiteEndpoint graphiteEndpoint; + private PrometheusEndpoint prometheusEndpoint; @BeforeMethod public void initializeAlways() { remoteService = new RemoteServiceEndpoint(SharedServices.services().serviceMock()); client = hermesClient(new JerseyHermesSender(newClient())).withURI(create("http://localhost:" + FRONTEND_PORT)).build(); - graphiteEndpoint = new GraphiteEndpoint(SharedServices.services().graphiteHttpMock()); + prometheusEndpoint = new PrometheusEndpoint(SharedServices.services().prometheusHttpMock()); } @AfterMethod @@ -410,8 +410,8 @@ public void shouldReturnHealthyStatusForAHealthySubscription() { // and operations.buildTopic(topic); operations.createSubscription(topic, subscriptionName, remoteService.getUrl()); - graphiteEndpoint.returnMetricForTopic(topic.getName().getGroupName(), topic.getName().getName(), 100, 100); - graphiteEndpoint.returnMetric(subscriptionMetricsStub(topic.getQualifiedName() + ".subscription").withRate(100).build()); + prometheusEndpoint.returnTopicMetrics(topic, new PrometheusTopicResponse(100, 100, 0)); + prometheusEndpoint.returnSubscriptionMetrics(topic, subscriptionName, builder().withRate(100).build()); // when SubscriptionHealth subscriptionHealth = management.subscription().getHealth(topic.getQualifiedName(), subscriptionName); @@ -429,10 +429,9 @@ public void shouldReturnUnhealthyStatusWithAProblemForMalfunctioningSubscription // and operations.buildTopic(topic); operations.createSubscription(topic, subscriptionName, remoteService.getUrl()); - graphiteEndpoint.returnMetricForTopic(topic.getName().getGroupName(), topic.getName().getName(), 100, 50); - graphiteEndpoint.returnMetric(subscriptionMetricsStub(topic.getQualifiedName() + ".subscription") - .withRate(50) - .withStatusRate(500, 11).build()); + prometheusEndpoint.returnTopicMetrics(topic, new PrometheusTopicResponse(100, 50, 0)); + prometheusEndpoint.returnSubscriptionMetrics(topic, "subscription", + builder().withRate(50).withRatedStatusCode("500", 11).build()); // when SubscriptionHealth subscriptionHealth = management.subscription().getHealth(topic.getQualifiedName(), subscriptionName); @@ -443,17 +442,17 @@ public void shouldReturnUnhealthyStatusWithAProblemForMalfunctioningSubscription } @Test - public void shouldReturnNoDataStatusWhenGraphiteRespondsWithAnError() { + public void shouldReturnNoDataStatusWhenPrometheusRespondsWithAnError() { // given - String topicName = "topic"; Topic topic = randomTopic("healthNoData", "topic").build(); String subscriptionName = "subscription"; // and operations.buildTopic(topic); operations.createSubscription(topic, subscriptionName, remoteService.getUrl()); - graphiteEndpoint.returnServerErrorForAllTopics(); - graphiteEndpoint.returnMetric(subscriptionMetricsStub(topic.getQualifiedName() + ".subscription").withRate(100).build()); + prometheusEndpoint.returnServerErrorForAllTopics(); + prometheusEndpoint.returnSubscriptionMetrics(topic, "subscription", + builder().withRate(100).build()); // when SubscriptionHealth subscriptionHealth = management.subscription().getHealth(topic.getQualifiedName(), subscriptionName); diff --git a/integration/src/integration/java/pl/allegro/tech/hermes/integration/setup/HermesManagementInstance.java b/integration/src/integration/java/pl/allegro/tech/hermes/integration/setup/HermesManagementInstance.java index e3586a4a5c..efe817f15f 100644 --- a/integration/src/integration/java/pl/allegro/tech/hermes/integration/setup/HermesManagementInstance.java +++ b/integration/src/integration/java/pl/allegro/tech/hermes/integration/setup/HermesManagementInstance.java @@ -4,7 +4,6 @@ import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.ExponentialBackoffRetry; -import org.slf4j.Logger; import pl.allegro.tech.hermes.integration.helper.Waiter; import pl.allegro.tech.hermes.management.HermesManagement; import pl.allegro.tech.hermes.test.helper.endpoint.BrokerOperations; @@ -18,13 +17,10 @@ import java.util.stream.Collectors; import static com.jayway.awaitility.Awaitility.waitAtMost; -import static org.slf4j.LoggerFactory.getLogger; import static pl.allegro.tech.hermes.test.helper.endpoint.TimeoutAdjuster.adjust; public class HermesManagementInstance { - private static final Logger logger = getLogger(HermesManagementInstance.class); - private final HermesAPIOperations operations; private HermesManagementInstance(HermesAPIOperations operations) { @@ -49,6 +45,7 @@ public static class Starter { private boolean uncleanLeaderElectionEnabled = false; private String schemaRegistry; private boolean avroContentTypeMetadataRequired = true; + private boolean graphiteExternalMetricsStorage = false; public Starter port(int port) { this.port = port; @@ -81,6 +78,11 @@ public Starter addKafkaCluster(String dc, String connectionString) { return this; } + public Starter withGraphiteExternalStorageEnabled() { + this.graphiteExternalMetricsStorage = true; + return this; + } + public HermesManagementInstance start() { try { startManagement(); @@ -119,6 +121,9 @@ private void startManagement() { args.add("--topic.uncleanLeaderElectionEnabled=" + uncleanLeaderElectionEnabled); args.add("--topic.avroContentTypeMetadataRequired=" + avroContentTypeMetadataRequired); args.add("--schema.repository.serverUrl=" + schemaRegistry); + + args.add("--graphite.client.enabled=" + graphiteExternalMetricsStorage); + args.add("--prometheus.client.enabled=" + !graphiteExternalMetricsStorage); HermesManagement.main(args.toArray(new String[0])); } diff --git a/integration/src/test/resources/application.yaml b/integration/src/test/resources/application.yaml index 2d8c3331a8..1635892834 100644 --- a/integration/src/test/resources/application.yaml +++ b/integration/src/test/resources/application.yaml @@ -45,11 +45,15 @@ storage: auth: oauthServerUrl: http://localhost:19999 -metrics: - graphiteHttpUri: http://localhost:18089/ - graphite: client: + enabled: false + cacheTtlSeconds: 1 + externalMonitoringUrl: http://localhost:18089/ + +prometheus: + client: + enabled: true cacheTtlSeconds: 1 spring: